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 2017/11/11 12:02:42 UTC

[01/11] carbondata git commit: [CARBONDATA-1690][DATAMAP] Clear datamap when renaming table [Forced Update!]

Repository: carbondata
Updated Branches:
  refs/heads/pre-aggregate 344ee0ac5 -> ee6bd2f86 (forced update)


[CARBONDATA-1690][DATAMAP] Clear datamap when renaming table

This closes #1479


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

Branch: refs/heads/pre-aggregate
Commit: a91c6e4f975885d30850ff168ab06c8c5b04e6f2
Parents: 4aa0f49
Author: xuchuanyin <xu...@hust.edu.cn>
Authored: Thu Nov 9 21:25:06 2017 +0800
Committer: ravipesala <ra...@gmail.com>
Committed: Fri Nov 10 16:22:06 2017 +0530

----------------------------------------------------------------------
 .../schema/CarbonAlterTableRenameCommand.scala  |  7 ++++--
 .../AlterTableValidationTestCase.scala          | 24 ++++++++++++++++++++
 2 files changed, 29 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/a91c6e4f/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala
index c000488..5936355 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala
@@ -25,6 +25,7 @@ import org.apache.spark.util.AlterTableUtil
 
 import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
 import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datamap.DataMapStoreManager
 import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.locks.{ICarbonLock, LockUsage}
 import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier}
@@ -83,9 +84,11 @@ private[sql] case class CarbonAlterTableRenameCommand(
       val tableMeta = metastore.lookupRelation(Some(oldDatabaseName), oldTableName)(sparkSession)
         .asInstanceOf[CarbonRelation].tableMeta
       carbonTable = tableMeta.carbonTable
+      // invalid data map for the old table, see CARBON-1690
+      val oldTableIdentifier = AbsoluteTableIdentifier.fromTablePath(tableMeta.tablePath)
+      DataMapStoreManager.getInstance().clearDataMap(oldTableIdentifier)
       // get the latest carbon table and check for column existence
-      val carbonTablePath = CarbonStorePath.
-        getCarbonTablePath(AbsoluteTableIdentifier.fromTablePath(tableMeta.tablePath))
+      val carbonTablePath = CarbonStorePath.getCarbonTablePath(oldTableIdentifier)
       val tableMetadataFile = carbonTablePath.getPath
       val tableInfo: org.apache.carbondata.format.TableInfo =
         metastore.getThriftTableInfo(carbonTablePath)(sparkSession)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a91c6e4f/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 9cfb77a..6f618fe 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
@@ -446,6 +446,30 @@ class AlterTableValidationTestCase extends Spark2QueryTest with BeforeAndAfterAl
     sql("alter table Default.uniqdata rename to uniqdata1")
     checkAnswer(sql("select * from Default.uniqdata1"), Row(1,"hello"))
   }
+
+  // test query before&after renaming, see CARBONDATA-1690
+  test("RenameTable_query_before_and_after_renaming") {
+    try {
+      sql(s"""create table test1 (name string, id int) stored by 'carbondata'""").collect
+      sql(s"""create table test2 (name string, id int) stored by 'carbondata'""").collect
+      sql(s"""insert into test1 select 'xx1',1""").collect
+      sql(s"""insert into test2 select 'xx2',2""").collect
+      // query before rename
+      checkAnswer(sql(s"""select * from test1"""), Seq(Row("xx1", 1)))
+      sql(s"""alter table test1 RENAME TO test3""").collect
+      sql(s"""alter table test2 RENAME TO test1""").collect
+      // query after rename
+      checkAnswer(sql(s"""select * from test1"""), Seq(Row("xx2", 2)))
+    } catch {
+      case e: Exception =>
+        assert(false)
+    } finally {
+      sql(s"""drop table if exists test1""").collect
+      sql(s"""drop table if exists test3""").collect
+      sql(s"""drop table if exists test2""").collect
+    }
+  }
+
   test("describe formatted for default sort_columns pre and post alter") {
     sql("CREATE TABLE defaultSortColumnsWithAlter (empno int, empname String, designation String,role String, doj Timestamp) STORED BY 'org.apache.carbondata.format' " +
         "tblproperties('DICTIONARY_INCLUDE'='empno','DICTIONARY_EXCLUDE'='role')")


[06/11] carbondata git commit: [DOCS] Fix license header

Posted by ra...@apache.org.
[DOCS] Fix license header

This closes #1487


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

Branch: refs/heads/pre-aggregate
Commit: 9f6c8e6cef99cadc44b32af8dcf1ea17f34b7861
Parents: c6ca640
Author: chenliang613 <ch...@huawei.com>
Authored: Sat Nov 11 16:00:09 2017 +0800
Committer: chenliang613 <ch...@huawei.com>
Committed: Sat Nov 11 16:06:57 2017 +0800

----------------------------------------------------------------------
 README.md                                       | 32 ++++++++++----------
 build/README.md                                 | 24 +++++++--------
 integration/hive/hive-guide.md                  | 24 +++++++--------
 integration/presto/README.md                    | 26 ++++++++--------
 .../spark/sql/CarbonCatalystOperators.scala     |  0
 5 files changed, 50 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/9f6c8e6c/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index f2b4e15..ed55de0 100644
--- a/README.md
+++ b/README.md
@@ -1,19 +1,19 @@
-/*
- * 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.
- */
+<!--
+    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.
+-->
 
 <img src="/docs/images/CarbonData_logo.png" width="200" height="40">
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/9f6c8e6c/build/README.md
----------------------------------------------------------------------
diff --git a/build/README.md b/build/README.md
index 50f6ce2..ab53877 100644
--- a/build/README.md
+++ b/build/README.md
@@ -1,20 +1,18 @@
 <!--
-    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
+    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.
+    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.
 -->
 
 # Building CarbonData

http://git-wip-us.apache.org/repos/asf/carbondata/blob/9f6c8e6c/integration/hive/hive-guide.md
----------------------------------------------------------------------
diff --git a/integration/hive/hive-guide.md b/integration/hive/hive-guide.md
index b3848b5..c38a539 100644
--- a/integration/hive/hive-guide.md
+++ b/integration/hive/hive-guide.md
@@ -1,20 +1,18 @@
 <!--
-    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
+    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.
+    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.
 -->
 
 # Quick Start

http://git-wip-us.apache.org/repos/asf/carbondata/blob/9f6c8e6c/integration/presto/README.md
----------------------------------------------------------------------
diff --git a/integration/presto/README.md b/integration/presto/README.md
index dc14cb0..d264954 100644
--- a/integration/presto/README.md
+++ b/integration/presto/README.md
@@ -1,26 +1,24 @@
 <!--
-    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
+    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.
+    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.
 -->
 
 Please follow the below steps to query carbondata in presto
 
 ### Config presto server
-* Download presto server (0.166 is suggested and supported) : https://repo1.maven.org/maven2/com/facebook/presto/presto-server/
+* Download presto server (0.186 is suggested and supported) : https://repo1.maven.org/maven2/com/facebook/presto/presto-server/
 * Finish presto configuration following https://prestodb.io/docs/current/installation/deployment.html.
   A configuration example:
   ```

http://git-wip-us.apache.org/repos/asf/carbondata/blob/9f6c8e6c/integration/spark/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala
deleted file mode 100644
index e69de29..0000000


[04/11] carbondata git commit: [CARBONDATA-1686] Presto Version Upgrade to 0.186

Posted by ra...@apache.org.
[CARBONDATA-1686] Presto Version Upgrade to 0.186

This closes #1477


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

Branch: refs/heads/pre-aggregate
Commit: ccb65603e611a15331b2cd0f1eef9e0ffd129b4c
Parents: 933e30c
Author: Bhavya <bh...@knoldus.com>
Authored: Thu Nov 9 12:55:50 2017 +0530
Committer: chenliang613 <ch...@huawei.com>
Committed: Sat Nov 11 08:41:18 2017 +0800

----------------------------------------------------------------------
 integration/presto/pom.xml                      | 41 +++++++++++++-----
 .../presto/CarbondataColumnConstraint.java      |  3 +-
 .../presto/CarbondataColumnHandle.java          |  3 +-
 .../carbondata/presto/CarbondataConnector.java  |  2 +-
 .../carbondata/presto/CarbondataPageSource.java |  4 --
 .../presto/CarbondataRecordCursor.java          |  5 ---
 .../presto/CarbondataTableLayoutHandle.java     |  3 +-
 .../presto/CarbondataTransactionHandle.java     | 44 +++++++++++++++++++-
 .../readers/DecimalSliceStreamReader.java       |  6 +--
 .../carbondata/presto/server/PrestoServer.scala | 10 ++---
 10 files changed, 84 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/ccb65603/integration/presto/pom.xml
----------------------------------------------------------------------
diff --git a/integration/presto/pom.xml b/integration/presto/pom.xml
index 13d351d..5179284 100644
--- a/integration/presto/pom.xml
+++ b/integration/presto/pom.xml
@@ -31,7 +31,7 @@
   <packaging>presto-plugin</packaging>
 
   <properties>
-    <presto.version>0.166</presto.version>
+    <presto.version>0.186</presto.version>
     <dev.path>${basedir}/../../dev</dev.path>
   </properties>
 
@@ -143,7 +143,6 @@
           <groupId>io.dropwizard.metrics</groupId>
           <artifactId>metrics-graphite</artifactId>
         </exclusion>
-
         <exclusion>
           <groupId>com.google.code.findbugs</groupId>
           <artifactId>jsr305</artifactId>
@@ -413,7 +412,7 @@
     <dependency>
       <groupId>io.airlift</groupId>
       <artifactId>slice</artifactId>
-      <version>0.27</version>
+      <version>0.31</version>
       <scope>provided</scope>
       <exclusions>
         <exclusion>
@@ -441,6 +440,34 @@
       <scope>test</scope>
       <version>2.1.0</version>
     </dependency>
+    <dependency>
+      <groupId>com.google.code.findbugs</groupId>
+      <artifactId>jsr305</artifactId>
+      <version>3.0.2</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.google.code.findbugs</groupId>
+      <artifactId>jsr305</artifactId>
+      <version>3.0.2</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.glassfish.hk2</groupId>
+      <artifactId>hk2-api</artifactId>
+      <version>2.5.0-b42</version>
+    </dependency>
+    <dependency>
+      <groupId>org.glassfish.hk2</groupId>
+      <artifactId>hk2-locator</artifactId>
+      <version>2.5.0-b42</version>
+    </dependency>
+    <dependency>
+      <groupId>org.glassfish.hk2</groupId>
+      <artifactId>hk2-utils</artifactId>
+      <version>2.5.0-b42</version>
+    </dependency>
+
   </dependencies>
 
   <build>
@@ -517,14 +544,6 @@
         </executions>
       </plugin>
       <plugin>
-        <artifactId>maven-compiler-plugin</artifactId>
-        <configuration>
-          <source>1.8</source>
-          <target>1.8</target>
-        </configuration>
-      </plugin>
-
-      <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-enforcer-plugin</artifactId>
         <version>1.4.1</version>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ccb65603/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataColumnConstraint.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataColumnConstraint.java b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataColumnConstraint.java
index 82c7c78..020e7b2 100755
--- a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataColumnConstraint.java
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataColumnConstraint.java
@@ -25,8 +25,7 @@ import com.fasterxml.jackson.annotation.JsonSetter;
 import java.util.Objects;
 import java.util.Optional;
 
-//import static com.google.common.base.MoreObjects.toStringHelper;
-import static com.google.common.base.Objects.toStringHelper;
+import static com.google.common.base.MoreObjects.toStringHelper;
 import static java.util.Objects.requireNonNull;
 
 /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ccb65603/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataColumnHandle.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataColumnHandle.java b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataColumnHandle.java
index fb9a0ba..4ec145d 100755
--- a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataColumnHandle.java
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataColumnHandle.java
@@ -25,8 +25,7 @@ import com.fasterxml.jackson.annotation.JsonProperty;
 
 import java.util.Objects;
 
-//import static com.google.common.base.MoreObjects.toStringHelper;
-import static com.google.common.base.Objects.toStringHelper;
+import static com.google.common.base.MoreObjects.toStringHelper;
 import static java.util.Objects.requireNonNull;
 
 public class CarbondataColumnHandle implements ColumnHandle {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ccb65603/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataConnector.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataConnector.java b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataConnector.java
index 406ed93..25917ac 100755
--- a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataConnector.java
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataConnector.java
@@ -51,7 +51,7 @@ public class CarbondataConnector implements Connector {
   @Override public ConnectorTransactionHandle beginTransaction(IsolationLevel isolationLevel,
       boolean readOnly) {
     checkConnectorSupports(READ_COMMITTED, isolationLevel);
-    return CarbondataTransactionHandle.INSTANCE;
+    return new CarbondataTransactionHandle();
   }
 
   @Override public ConnectorMetadata getMetadata(ConnectorTransactionHandle transactionHandle) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ccb65603/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataPageSource.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataPageSource.java b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataPageSource.java
index fd65230..390565b 100644
--- a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataPageSource.java
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataPageSource.java
@@ -78,10 +78,6 @@ class CarbondataPageSource implements ConnectorPageSource {
     this.readers = createStreamReaders();
   }
 
-  @Override public long getTotalBytes() {
-    return sizeOfData;
-  }
-
   @Override public long getCompletedBytes() {
     return sizeOfData;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ccb65603/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordCursor.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordCursor.java b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordCursor.java
index 4663903..c614fa9 100755
--- a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordCursor.java
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordCursor.java
@@ -22,7 +22,6 @@ import java.math.BigInteger;
 import java.sql.Timestamp;
 import java.util.List;
 
-import org.apache.carbondata.common.CarbonIterator;
 import org.apache.carbondata.core.cache.dictionary.Dictionary;
 import org.apache.carbondata.core.metadata.datatype.DataType;
 
@@ -73,10 +72,6 @@ public class CarbondataRecordCursor implements RecordCursor {
     this.totalBytes = 0;
   }
 
-  @Override public long getTotalBytes() {
-    return totalBytes;
-  }
-
   @Override public long getCompletedBytes() {
     return totalBytes;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ccb65603/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataTableLayoutHandle.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataTableLayoutHandle.java b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataTableLayoutHandle.java
index bf6318f..fbaa1d2 100755
--- a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataTableLayoutHandle.java
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataTableLayoutHandle.java
@@ -25,8 +25,7 @@ import com.fasterxml.jackson.annotation.JsonProperty;
 
 import java.util.Objects;
 
-//import static com.google.common.base.MoreObjects.toStringHelper;
-import static com.google.common.base.Objects.toStringHelper;
+import static com.google.common.base.MoreObjects.toStringHelper;
 import static java.util.Objects.requireNonNull;
 
 public class CarbondataTableLayoutHandle implements ConnectorTableLayoutHandle {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ccb65603/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataTransactionHandle.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataTransactionHandle.java b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataTransactionHandle.java
index e95c490..7939d23 100755
--- a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataTransactionHandle.java
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataTransactionHandle.java
@@ -17,8 +17,48 @@
 
 package org.apache.carbondata.presto;
 
+import java.util.Objects;
+import java.util.UUID;
+
 import com.facebook.presto.spi.connector.ConnectorTransactionHandle;
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import static com.google.common.base.MoreObjects.toStringHelper;
+import static java.util.Objects.requireNonNull;
+
+public class CarbondataTransactionHandle implements ConnectorTransactionHandle {
+  private final UUID uuid;
+
+  public CarbondataTransactionHandle() {
+    this(UUID.randomUUID());
+  }
+
+  @JsonCreator public CarbondataTransactionHandle(@JsonProperty("uuid") UUID uuid) {
+    this.uuid = requireNonNull(uuid, "uuid is null");
+  }
+
+  @JsonProperty public UUID getUuid() {
+    return uuid;
+  }
+
+  @Override public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if ((obj == null) || (getClass() != obj.getClass())) {
+      return false;
+    }
+
+    return Objects.equals(uuid, ((CarbondataTransactionHandle) obj).uuid);
+  }
+
+  @Override public int hashCode() {
+    return Objects.hash(uuid);
+  }
+
+  @Override public String toString() {
+    return toStringHelper(this).add("uuid", uuid).toString();
+  }
 
-public enum CarbondataTransactionHandle implements ConnectorTransactionHandle {
-  INSTANCE
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ccb65603/integration/presto/src/main/java/org/apache/carbondata/presto/readers/DecimalSliceStreamReader.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/readers/DecimalSliceStreamReader.java b/integration/presto/src/main/java/org/apache/carbondata/presto/readers/DecimalSliceStreamReader.java
index 6612ab0..2f84bf4 100644
--- a/integration/presto/src/main/java/org/apache/carbondata/presto/readers/DecimalSliceStreamReader.java
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/readers/DecimalSliceStreamReader.java
@@ -187,9 +187,9 @@ public class DecimalSliceStreamReader  extends AbstractStreamReader {
         builder.appendNull();
       } else {
         if (isShortDecimal(type)) {
-          long rescaledDecimal = Decimals
-              .rescale(columnVector.getDecimal(i, precision, scale).toLong(),
-                  columnVector.getDecimal(i, precision, scale).scale(), scale);
+          BigDecimal decimalValue = columnVector.getDecimal(i, precision, scale).toJavaBigDecimal();
+          long rescaledDecimal = Decimals.rescale(decimalValue.unscaledValue().longValue(),
+              decimalValue.scale(), scale);
           type.writeLong(builder, rescaledDecimal);
         } else {
           Slice slice =

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ccb65603/integration/presto/src/test/scala/org/apache/carbondata/presto/server/PrestoServer.scala
----------------------------------------------------------------------
diff --git a/integration/presto/src/test/scala/org/apache/carbondata/presto/server/PrestoServer.scala b/integration/presto/src/test/scala/org/apache/carbondata/presto/server/PrestoServer.scala
index cdf8833..d6d3617 100644
--- a/integration/presto/src/test/scala/org/apache/carbondata/presto/server/PrestoServer.scala
+++ b/integration/presto/src/test/scala/org/apache/carbondata/presto/server/PrestoServer.scala
@@ -18,7 +18,7 @@ package org.apache.carbondata.presto.server
 
 import java.sql.{Connection, DriverManager, ResultSet}
 import java.util
-import java.util.{Locale, Optional}
+import java.util.{Locale, Optional, Properties}
 
 import scala.collection.JavaConverters._
 import scala.util.{Failure, Success, Try}
@@ -119,14 +119,14 @@ object PrestoServer {
     val JDBC_DRIVER = "com.facebook.presto.jdbc.PrestoDriver"
     val DB_URL = "jdbc:presto://localhost:8086/carbondata/testdb"
 
+    val properties = new Properties
     // The database Credentials
-    val USER = "username"
-    val PASS = "password"
-
+    properties.setProperty("user", "test");
+  
     // STEP 2: Register JDBC driver
     Class.forName(JDBC_DRIVER)
     // STEP 3: Open a connection
-    DriverManager.getConnection(DB_URL, USER, PASS)
+    DriverManager.getConnection(DB_URL, properties)
   }
 
   /**


[10/11] carbondata git commit: [CARBONDATA-1517]- Pre Aggregate Create Table Support

Posted by ra...@apache.org.
[CARBONDATA-1517]- Pre Aggregate Create Table Support

Support CTAS in carbon and support creating aggregation tables using CTAS and update aggregation table information to main table schema.

This closes #1433


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

Branch: refs/heads/pre-aggregate
Commit: 1d560c069a6f1c2cac80b78280e794eec67bcb5b
Parents: 037f630
Author: kumarvishal <ku...@gmail.com>
Authored: Sun Oct 15 18:05:55 2017 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Sat Nov 11 17:32:02 2017 +0530

----------------------------------------------------------------------
 .../table/column/ParentColumnTableRelation.java |  71 +++
 .../ThriftWrapperSchemaConverterImplTest.java   |  28 +-
 .../preaggregate/TestPreAggCreateCommand.scala  | 148 +++++++
 .../carbondata/spark/util/CommonUtil.scala      |   9 +
 .../spark/sql/catalyst/CarbonDDLSqlParser.scala |  12 +-
 .../command/carbonTableSchemaCommon.scala       | 170 +++++---
 .../command/management/LoadTableCommand.scala   |   2 +-
 .../CreatePreAggregateTableCommand.scala        | 136 ++++++
 .../preaaggregate/PreAggregateUtil.scala        | 431 +++++++++++++++++++
 .../schema/CarbonAlterTableRenameCommand.scala  |   2 +-
 .../spark/sql/hive/CarbonFileMetastore.scala    |  41 +-
 .../spark/sql/hive/CarbonHiveMetaStore.scala    |  72 +++-
 .../apache/spark/sql/hive/CarbonMetaStore.scala |  21 +-
 .../sql/parser/CarbonSpark2SqlParser.scala      |   2 +-
 .../spark/sql/parser/CarbonSparkSqlParser.scala |  33 +-
 .../org/apache/spark/util/AlterTableUtil.scala  |  10 +-
 16 files changed, 1100 insertions(+), 88 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/1d560c06/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/ParentColumnTableRelation.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/ParentColumnTableRelation.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/ParentColumnTableRelation.java
new file mode 100644
index 0000000..425d0f2
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/ParentColumnTableRelation.java
@@ -0,0 +1,71 @@
+/*
+ * 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.core.metadata.schema.table.column;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.Serializable;
+
+import org.apache.carbondata.core.metadata.schema.table.RelationIdentifier;
+import org.apache.carbondata.core.metadata.schema.table.Writable;
+
+/**
+ * To maintain the relation of child column to parent table column
+ */
+public class ParentColumnTableRelation implements Serializable, Writable {
+
+  private RelationIdentifier relationIdentifier;
+  /**
+   * parent column id
+   */
+  private String columnId;
+
+  private String columnName;
+
+  public ParentColumnTableRelation(RelationIdentifier relationIdentifier, String columId,
+      String columnName) {
+    this.relationIdentifier = relationIdentifier;
+    this.columnId = columId;
+    this.columnName = columnName;
+  }
+
+  public RelationIdentifier getRelationIdentifier() {
+    return relationIdentifier;
+  }
+
+  public String getColumnId() {
+    return columnId;
+  }
+
+  public String getColumnName() {
+    return columnName;
+  }
+
+  @Override public void write(DataOutput out) throws IOException {
+    relationIdentifier.write(out);
+    out.writeUTF(columnId);
+    out.writeUTF(columnName);
+  }
+
+  @Override public void readFields(DataInput in) throws IOException {
+    this.relationIdentifier = new RelationIdentifier(null, null, null);
+    relationIdentifier.readFields(in);
+    this.columnId = in.readUTF();
+    this.columnName = in.readUTF();
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1d560c06/core/src/test/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImplTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImplTest.java b/core/src/test/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImplTest.java
index b8e4eea..d4bb344 100644
--- a/core/src/test/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImplTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImplTest.java
@@ -29,6 +29,7 @@ import org.apache.carbondata.core.metadata.schema.SchemaEvolutionEntry;
 import org.apache.carbondata.core.metadata.schema.table.TableInfo;
 import org.apache.carbondata.core.metadata.schema.table.TableSchema;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
+import org.apache.carbondata.format.DataMapSchema;
 
 import mockit.Mock;
 import mockit.MockUp;
@@ -82,6 +83,7 @@ public class ThriftWrapperSchemaConverterImplTest {
         new org.apache.carbondata.format.ColumnSchema(org.apache.carbondata.format.DataType.STRING,
             "columnName", "1", true, encoders, true);
     thriftColumnSchema.setSchemaOrdinal(1);
+    thriftColumnSchema.setAggregate_function("");
     thriftColumnSchemas = new ArrayList<org.apache.carbondata.format.ColumnSchema>();
     thriftColumnSchemas.add(thriftColumnSchema);
     thriftSchemaEvolutionEntries = new ArrayList<>();
@@ -419,6 +421,7 @@ public class ThriftWrapperSchemaConverterImplTest {
             new org.apache.carbondata.format.ColumnSchema(org.apache.carbondata.format.DataType.BOOLEAN,
                     "columnName", "1", true, encoders, true);
     thriftColumnSchema.setSchemaOrdinal(1);
+    thriftColumnSchema.setAggregate_function("");
     ColumnSchema wrapperColumnSchema = new ColumnSchema();
 
     new MockUp<ColumnSchema>() {
@@ -481,6 +484,8 @@ public class ThriftWrapperSchemaConverterImplTest {
       @Mock public String getColumnReferenceId() {
         return "1";
       }
+
+      @Mock public String getAggFunction() {return "" ;}
     };
 
     org.apache.carbondata.format.ColumnSchema actualResult =
@@ -494,7 +499,7 @@ public class ThriftWrapperSchemaConverterImplTest {
         new org.apache.carbondata.format.ColumnSchema(org.apache.carbondata.format.DataType.STRING,
             "columnName", "1", true, encoders, true);
     thriftColumnSchema.setSchemaOrdinal(1);
-
+    thriftColumnSchema.setAggregate_function("");
     new MockUp<ColumnSchema>() {
       @Mock public List<Encoding> getEncodingList() {
         return encodings;
@@ -569,7 +574,7 @@ public class ThriftWrapperSchemaConverterImplTest {
         new org.apache.carbondata.format.ColumnSchema(org.apache.carbondata.format.DataType.INT,
             "columnName", "1", true, encoders, true);
     thriftColumnSchema.setSchemaOrdinal(1);
-
+    thriftColumnSchema.setAggregate_function("");
     new MockUp<ColumnSchema>() {
       @Mock public List<Encoding> getEncodingList() {
         return encodings;
@@ -643,6 +648,7 @@ public class ThriftWrapperSchemaConverterImplTest {
         new org.apache.carbondata.format.ColumnSchema(org.apache.carbondata.format.DataType.SHORT,
             "columnName", "1", true, encoders, true);
     thriftColumnSchema.setSchemaOrdinal(1);
+    thriftColumnSchema.setAggregate_function("");
     new MockUp<ColumnSchema>() {
       @Mock public List<Encoding> getEncodingList() {
         return encodings;
@@ -716,7 +722,7 @@ public class ThriftWrapperSchemaConverterImplTest {
         new org.apache.carbondata.format.ColumnSchema(org.apache.carbondata.format.DataType.LONG,
             "columnName", "1", true, encoders, true);
     thriftColumnSchema.setSchemaOrdinal(1);
-
+    thriftColumnSchema.setAggregate_function("");
     new MockUp<ColumnSchema>() {
       @Mock public List<Encoding> getEncodingList() {
         return encodings;
@@ -790,6 +796,7 @@ public class ThriftWrapperSchemaConverterImplTest {
         new org.apache.carbondata.format.ColumnSchema(org.apache.carbondata.format.DataType.DOUBLE,
             "columnName", "1", true, encoders, true);
     thriftColumnSchema.setSchemaOrdinal(1);
+    thriftColumnSchema.setAggregate_function("");
 
     new MockUp<ColumnSchema>() {
       @Mock public List<Encoding> getEncodingList() {
@@ -864,6 +871,7 @@ public class ThriftWrapperSchemaConverterImplTest {
         new org.apache.carbondata.format.ColumnSchema(org.apache.carbondata.format.DataType.DECIMAL,
             "columnName", "1", true, encoders, true);
     thriftColumnSchema.setSchemaOrdinal(1);
+    thriftColumnSchema.setAggregate_function("");
     new MockUp<ColumnSchema>() {
       @Mock public List<Encoding> getEncodingList() {
         return encodings;
@@ -924,6 +932,10 @@ public class ThriftWrapperSchemaConverterImplTest {
       @Mock public String getColumnReferenceId() {
         return "1";
       }
+
+      @Mock public String getAggFunction() {
+        return "";
+      }
     };
 
     ColumnSchema wrapperColumnSchema = new ColumnSchema();
@@ -938,6 +950,7 @@ public class ThriftWrapperSchemaConverterImplTest {
             org.apache.carbondata.format.DataType.TIMESTAMP, "columnName", "1", true, encoders,
             true);
     thriftColumnSchema.setSchemaOrdinal(1);
+    thriftColumnSchema.setAggregate_function("");
 
     new MockUp<ColumnSchema>() {
       @Mock public List<Encoding> getEncodingList() {
@@ -1012,7 +1025,7 @@ public class ThriftWrapperSchemaConverterImplTest {
         new org.apache.carbondata.format.ColumnSchema(org.apache.carbondata.format.DataType.ARRAY,
             "columnName", "1", true, encoders, true);
     thriftColumnSchema.setSchemaOrdinal(1);
-
+    thriftColumnSchema.setAggregate_function("");
     new MockUp<ColumnSchema>() {
       @Mock public List<Encoding> getEncodingList() {
         return encodings;
@@ -1086,6 +1099,7 @@ public class ThriftWrapperSchemaConverterImplTest {
         new org.apache.carbondata.format.ColumnSchema(org.apache.carbondata.format.DataType.STRUCT,
             "columnName", "1", true, encoders, true);
     thriftColumnSchema.setSchemaOrdinal(1);
+    thriftColumnSchema.setAggregate_function("");
 
     new MockUp<ColumnSchema>() {
       @Mock public List<Encoding> getEncodingList() {
@@ -1166,6 +1180,7 @@ public class ThriftWrapperSchemaConverterImplTest {
             encoders,
             true);
     thriftColumnSchema.setSchemaOrdinal(1);
+    thriftColumnSchema.setAggregate_function("");
 
     new MockUp<ColumnSchema>() {
       @Mock public List<Encoding> getEncodingList() {
@@ -1316,7 +1331,7 @@ public class ThriftWrapperSchemaConverterImplTest {
         new org.apache.carbondata.format.ColumnSchema(org.apache.carbondata.format.DataType.BOOLEAN,
             "columnName", "1", true, encoders, true);
     thriftColumnSchema.setSchemaOrdinal(1);
-
+    thriftColumnSchema.setAggregate_function("");
     ColumnSchema wrapperColumnSchema = new ColumnSchema();
     org.apache.carbondata.format.ColumnSchema actualResult =
         thriftWrapperSchemaConverter.fromWrapperToExternalColumnSchema(wrapperColumnSchema);
@@ -1504,6 +1519,8 @@ public class ThriftWrapperSchemaConverterImplTest {
       @Mock public String getColumnReferenceId() {
         return "1";
       }
+
+      @Mock public String getAggFunction() { return "";}
     };
 
     new MockUp<TableInfo>() {
@@ -1540,6 +1557,7 @@ public class ThriftWrapperSchemaConverterImplTest {
     org.apache.carbondata.format.TableInfo expectedResult =
         new org.apache.carbondata.format.TableInfo(thriftFactTable, new ArrayList<org.apache
             .carbondata.format.TableSchema>());
+    expectedResult.setDataMapSchemas(new ArrayList<DataMapSchema>());
     assertEquals(expectedResult, actualResult);
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1d560c06/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggCreateCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggCreateCommand.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggCreateCommand.scala
new file mode 100644
index 0000000..6120e88
--- /dev/null
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggCreateCommand.scala
@@ -0,0 +1,148 @@
+package org.apache.carbondata.integration.spark.testsuite.preaggregate
+
+import org.apache.spark.sql.test.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+class TestPreAggCreateCommand extends QueryTest with BeforeAndAfterAll {
+
+  override def beforeAll {
+    sql("drop table if exists PreAggMain")
+    sql("drop table if exists PreAggMain1")
+    sql("drop table if exists PreAggMain2")
+    sql("create table preaggMain (a string, b string, c string) stored by 'carbondata'")
+    sql("create table preaggMain1 (a string, b string, c string) stored by 'carbondata' tblProperties('DICTIONARY_INCLUDE' = 'a')")
+    sql("create table preaggMain2 (a string, b string, c string) stored by 'carbondata'")
+  }
+
+
+  test("test pre agg create table One") {
+    sql("create table preagg1 stored BY 'carbondata' tblproperties('parent'='PreAggMain') as select a,sum(b) from PreAggMain group by a")
+    checkExistence(sql("DESCRIBE FORMATTED preagg1"), true, "preaggmain_a")
+    checkExistence(sql("DESCRIBE FORMATTED preagg1"), true, "preaggmain_b_sum")
+    sql("drop table preagg1")
+  }
+
+  test("test pre agg create table Two") {
+    sql("create table preagg2 stored BY 'carbondata' tblproperties('parent'='PreAggMain') as select a as a1,sum(b) from PreAggMain group by a")
+    checkExistence(sql("DESCRIBE FORMATTED preagg2"), true, "preaggmain_a")
+    checkExistence(sql("DESCRIBE FORMATTED preagg2"), true, "preaggmain_b_sum")
+    sql("drop table preagg2")
+  }
+
+  test("test pre agg create table Three") {
+    sql("create table preagg3 stored BY 'carbondata' tblproperties('parent'='PreAggMain') as select a,sum(b) as sum from PreAggMain group by a")
+    checkExistence(sql("DESCRIBE FORMATTED preagg3"), true, "preaggmain_a")
+    checkExistence(sql("DESCRIBE FORMATTED preagg3"), true, "preaggmain_b_sum")
+    sql("drop table preagg3")
+  }
+
+  test("test pre agg create table four") {
+    sql("create table preagg4 stored BY 'carbondata' tblproperties('parent'='PreAggMain') as select a as a1,sum(b) as sum from PreAggMain group by a")
+    checkExistence(sql("DESCRIBE FORMATTED preagg4"), true, "preaggmain_a")
+    checkExistence(sql("DESCRIBE FORMATTED preagg4"), true, "preaggmain_b_sum")
+    sql("drop table preagg4")
+  }
+
+
+  test("test pre agg create table five") {
+    sql("create table preagg11 stored BY 'carbondata' tblproperties('parent'='PreAggMain1') as select a,sum(b) from PreAggMain1 group by a")
+    checkExistence(sql("DESCRIBE FORMATTED preagg11"), true, "preaggmain1_a")
+    checkExistence(sql("DESCRIBE FORMATTED preagg11"), true, "preaggmain1_b_sum")
+    checkExistence(sql("DESCRIBE FORMATTED preagg11"), true, "DICTIONARY")
+    sql("drop table preagg11")
+  }
+
+  test("test pre agg create table six") {
+    sql("create table preagg12 stored BY 'carbondata' tblproperties('parent'='PreAggMain1') as select a as a1,sum(b) from PreAggMain1 group by a")
+    checkExistence(sql("DESCRIBE FORMATTED preagg12"), true, "preaggmain1_a")
+    checkExistence(sql("DESCRIBE FORMATTED preagg12"), true, "preaggmain1_b_sum")
+    checkExistence(sql("DESCRIBE FORMATTED preagg12"), true, "DICTIONARY")
+    sql("drop table preagg12")
+  }
+
+  test("test pre agg create table seven") {
+    sql("create table preagg13 stored BY 'carbondata' tblproperties('parent'='PreAggMain1') as select a,sum(b) as sum from PreAggMain1 group by a")
+    checkExistence(sql("DESCRIBE FORMATTED preagg13"), true, "preaggmain1_a")
+    checkExistence(sql("DESCRIBE FORMATTED preagg13"), true, "preaggmain1_b_sum")
+    checkExistence(sql("DESCRIBE FORMATTED preagg13"), true, "DICTIONARY")
+    sql("drop table preagg13")
+  }
+
+  test("test pre agg create table eight") {
+    sql("create table preagg14 stored BY 'carbondata' tblproperties('parent'='PreAggMain1') as select a as a1,sum(b) as sum from PreAggMain1 group by a")
+    checkExistence(sql("DESCRIBE FORMATTED preagg14"), true, "preaggmain1_a")
+    checkExistence(sql("DESCRIBE FORMATTED preagg14"), true, "preaggmain1_b_sum")
+    checkExistence(sql("DESCRIBE FORMATTED preagg14"), true, "DICTIONARY")
+    sql("drop table preagg14")
+  }
+
+
+  test("test pre agg create table nine") {
+    sql("create table preagg15 stored BY 'carbondata' tblproperties('parent'='PreAggMain2') as select a,avg(b) from PreAggMain2 group by a")
+    checkExistence(sql("DESCRIBE FORMATTED preagg15"), true, "preaggmain2_a")
+    checkExistence(sql("DESCRIBE FORMATTED preagg15"), true, "preaggmain2_b_sum")
+    checkExistence(sql("DESCRIBE FORMATTED preagg15"), true, "preaggmain2_b_count")
+    sql("drop table preagg15")
+  }
+
+  test("test pre agg create table ten") {
+    sql("create table preagg16 stored BY 'carbondata' tblproperties('parent'='PreAggMain2') as select a as a1,max(b) from PreAggMain2 group by a")
+    checkExistence(sql("DESCRIBE FORMATTED preagg16"), true, "preaggmain2_a")
+    checkExistence(sql("DESCRIBE FORMATTED preagg16"), true, "preaggmain2_b_max")
+    sql("drop table preagg16")
+  }
+
+  test("test pre agg create table eleven") {
+    sql("create table preagg17 stored BY 'carbondata' tblproperties('parent'='PreAggMain2') as select a,min(b) from PreAggMain2 group by a")
+    checkExistence(sql("DESCRIBE FORMATTED preagg17"), true, "preaggmain2_a")
+    checkExistence(sql("DESCRIBE FORMATTED preagg17"), true, "preaggmain2_b_min")
+    sql("drop table preagg17")
+  }
+
+  test("test pre agg create table twelve") {
+    sql("create table preagg18 stored BY 'carbondata' tblproperties('parent'='PreAggMain2') as select a as a1,count(b) from PreAggMain2 group by a")
+    checkExistence(sql("DESCRIBE FORMATTED preagg18"), true, "preaggmain2_a")
+    checkExistence(sql("DESCRIBE FORMATTED preagg18"), true, "preaggmain2_b_count")
+    sql("drop table preagg18")
+  }
+
+  test("test pre agg create table thirteen") {
+    try {
+      sql(
+        "create table preagg19 stored BY 'carbondata' tblproperties('parent'='PreAggMain2') as select a as a1,count(distinct b) from PreAggMain2 group by a")
+      assert(false)
+    } catch {
+      case _: Exception =>
+        assert(true)
+    }
+  }
+
+  test("test pre agg create table fourteen") {
+    try {
+      sql(
+        "create table preagg20 stored BY 'carbondata' tblproperties('parent'='PreAggMain2') as select a as a1,sum(distinct b) from PreAggMain2 group by a")
+      assert(false)
+    } catch {
+      case _: Exception =>
+        assert(true)
+    }
+  }
+
+  test("test pre agg create table fifteen") {
+    try {
+      sql(
+        "create table preagg21 stored BY 'carbondata' tblproperties('parent'='PreAggMain2') as select a as a1,sum(b) from PreAggMain2 where a='vishal' group by a")
+      assert(false)
+    } catch {
+      case _: Exception =>
+        assert(true)
+    }
+  }
+
+
+  override def afterAll {
+    sql("drop table if exists PreAggMain")
+    sql("drop table if exists PreAggMain1")
+    sql("drop table if exists PreAggMain2")
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1d560c06/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
index 27ebf42..84294ff 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
@@ -20,6 +20,7 @@ package org.apache.carbondata.spark.util
 
 import java.text.SimpleDateFormat
 import java.util
+import java.util.regex.{Matcher, Pattern}
 
 import scala.collection.JavaConverters._
 import scala.collection.mutable.Map
@@ -834,6 +835,14 @@ object CommonUtil {
     }
   }
 
+  def getScaleAndPrecision(dataType: String): (Int, Int) = {
+    val m: Matcher = Pattern.compile("^decimal\\(([^)]+)\\)").matcher(dataType)
+    m.find()
+    val matchedString: String = m.group(1)
+    val scaleAndPrecision = matchedString.split(",")
+    (Integer.parseInt(scaleAndPrecision(0).trim), Integer.parseInt(scaleAndPrecision(1).trim))
+  }
+
   /**
    * Merge the carbonindex files with in the segment to carbonindexmerge file inside same segment
    */

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1d560c06/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
index ee51954..bb80bce 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
@@ -221,16 +221,6 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
     dimensions ++ complexDimensions
   }
 
-
-
-  def getScaleAndPrecision(dataType: String): (Int, Int) = {
-    val m: Matcher = Pattern.compile("^decimal\\(([^)]+)\\)").matcher(dataType)
-    m.find()
-    val matchedString: String = m.group(1)
-    val scaleAndPrecision = matchedString.split(",")
-    (Integer.parseInt(scaleAndPrecision(0).trim), Integer.parseInt(scaleAndPrecision(1).trim))
-  }
-
   /**
    * This will prepate the Model from the Tree details.
    *
@@ -1074,7 +1064,7 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
       // if it is present then extracting the precision and scale. resetting the data type
       // with Decimal.
       case _ if dataType.startsWith("decimal") =>
-        val (precision, scale) = getScaleAndPrecision(dataType)
+        val (precision, scale) = CommonUtil.getScaleAndPrecision(dataType)
         Field(field.column,
           Some("Decimal"),
           field.name,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1d560c06/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala
index 1188b59..37ba8a5 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala
@@ -33,8 +33,8 @@ import org.apache.carbondata.core.metadata.CarbonTableIdentifier
 import org.apache.carbondata.core.metadata.datatype.{DataType, DataTypes, DecimalType}
 import org.apache.carbondata.core.metadata.encoder.Encoding
 import org.apache.carbondata.core.metadata.schema._
-import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, TableInfo, TableSchema}
-import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema
+import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, RelationIdentifier, TableInfo, TableSchema}
+import org.apache.carbondata.core.metadata.schema.table.column.{ColumnSchema, ParentColumnTableRelation}
 import org.apache.carbondata.core.service.CarbonCommonFactory
 import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentUpdateStatusManager}
 import org.apache.carbondata.core.util.DataTypeUtil
@@ -61,13 +61,30 @@ case class TableModel(
     util.List[ColumnProperty]]] = None,
     bucketFields: Option[BucketFields],
     partitionInfo: Option[PartitionInfo],
-    tableComment: Option[String] = None)
+    tableComment: Option[String] = None,
+    var parentTable: Option[CarbonTable] = None,
+    var dataMapRelation: Option[scala.collection.mutable.LinkedHashMap[Field, DataMapField]] = None)
 
 case class Field(column: String, var dataType: Option[String], name: Option[String],
     children: Option[List[Field]], parent: String = null,
     storeType: Option[String] = Some("columnar"),
     var schemaOrdinal: Int = -1,
-    var precision: Int = 0, var scale: Int = 0, var rawSchema: String = "")
+    var precision: Int = 0, var scale: Int = 0, var rawSchema: String = "") {
+  override def equals(o: Any) : Boolean = o match {
+    case that: Field =>
+      that.column.equalsIgnoreCase(this.column)
+    case _ => false
+  }
+  override def hashCode : Int = column.hashCode
+}
+
+case class DataMapField(aggregateFunction: String = "",
+    columnTableRelation: Option[ColumnTableRelation] = None) {
+}
+
+case class ColumnTableRelation(parentColumnName: String, parentColumnId: String,
+    parentTableName: String, parentDatabaseName: String, parentTableId: String) {
+}
 
 case class ColumnProperty(key: String, value: String)
 
@@ -358,15 +375,13 @@ class TableNewProcessor(cm: TableModel) {
       fields.foreach(field => {
         val encoders = new java.util.ArrayList[Encoding]()
         encoders.add(Encoding.DICTIONARY)
-        val columnSchema: ColumnSchema = TableNewProcessor.createColumnSchema(
-          field,
+        val columnSchema: ColumnSchema = getColumnSchema(
+          DataTypeConverterUtil.convertToCarbonType(field.dataType.getOrElse("")),
+          field.name.getOrElse(field.column),
           encoders,
-          isDimensionCol = true,
-          field.precision,
-          field.scale,
-          field.schemaOrdinal,
-          cm.highcardinalitydims.getOrElse(Seq()),
-          cm.databaseName)
+          true,
+          field,
+          cm.dataMapRelation)
         allColumns ++= Seq(columnSchema)
         if (field.children.get != null) {
           columnSchema.setNumberOfChild(field.children.get.size)
@@ -377,6 +392,56 @@ class TableNewProcessor(cm: TableModel) {
     allColumns
   }
 
+  def getColumnSchema(
+      dataType: DataType,
+      colName: String,
+      encoders: java.util.List[Encoding],
+      isDimensionCol: Boolean,
+      field: Field,
+      map: Option[scala.collection.mutable.LinkedHashMap[Field, DataMapField]]) : ColumnSchema = {
+    val columnSchema = new ColumnSchema()
+    columnSchema.setDataType(dataType)
+    columnSchema.setColumnName(colName)
+    val isParentColumnRelation = map.isDefined && map.get.get(field).isDefined
+    if(!isParentColumnRelation) {
+      val highCardinalityDims = cm.highcardinalitydims.getOrElse(Seq())
+      if (highCardinalityDims.contains(colName)) {
+        encoders.remove(Encoding.DICTIONARY)
+      }
+    if (dataType == DataTypes.DATE) {
+        encoders.add(Encoding.DIRECT_DICTIONARY)
+      }
+    if (dataType == DataTypes.TIMESTAMP && !highCardinalityDims.contains(colName)) {
+        encoders.add(Encoding.DIRECT_DICTIONARY)
+      }
+    }
+    columnSchema.setEncodingList(encoders)
+    val colUniqueIdGenerator = CarbonCommonFactory.getColumnUniqueIdGenerator
+    val columnUniqueId = colUniqueIdGenerator.generateUniqueId(cm.databaseName,
+      columnSchema)
+    columnSchema.setColumnUniqueId(columnUniqueId)
+    columnSchema.setColumnReferenceId(columnUniqueId)
+    columnSchema.setDimensionColumn(isDimensionCol)
+    columnSchema.setPrecision(field.precision)
+    columnSchema.setScale(field.scale)
+    columnSchema.setSchemaOrdinal(field.schemaOrdinal)
+    columnSchema.setSortColumn(false)
+    if(isParentColumnRelation) {
+      val dataMapField = map.get.get(field).get
+      columnSchema.setAggFunction(dataMapField.aggregateFunction);
+        val relation = dataMapField.columnTableRelation.get
+        val parentColumnTableRelationList = new util.ArrayList[ParentColumnTableRelation]
+        val relationIdentifier = new RelationIdentifier(
+          relation.parentDatabaseName, relation.parentTableName, relation.parentTableId)
+        val parentColumnTableRelation = new ParentColumnTableRelation(
+          relationIdentifier, relation.parentColumnId, relation.parentColumnName)
+        parentColumnTableRelationList.add(parentColumnTableRelation)
+        columnSchema.setParentColumnTableRelations(parentColumnTableRelationList)
+    }
+    // TODO: Need to fill RowGroupID, converted type
+    // & Number of Children after DDL finalization
+    columnSchema
+  }
 
   // process create dml fields and create wrapper TableInfo object
   def process: TableInfo = {
@@ -388,17 +453,22 @@ class TableNewProcessor(cm: TableModel) {
     // Sort columns should be at the begin of all columns
     cm.sortKeyDims.get.foreach { keyDim =>
       val field = cm.dimCols.find(keyDim equals _.column).get
-      val encoders = new java.util.ArrayList[Encoding]()
-      encoders.add(Encoding.DICTIONARY)
-      val columnSchema: ColumnSchema = TableNewProcessor.createColumnSchema(
-        field,
+      val encoders = if (cm.parentTable.isDefined && cm.dataMapRelation.get.get(field).isDefined) {
+        cm.parentTable.get.getColumnByName(
+          cm.parentTable.get.getFactTableName,
+          cm.dataMapRelation.get.get(field).get.columnTableRelation.get.parentColumnName).getEncoder
+      } else {
+        val encoders = new java.util.ArrayList[Encoding]()
+        encoders.add(Encoding.DICTIONARY)
+        encoders
+      }
+      val columnSchema = getColumnSchema(
+        DataTypeConverterUtil.convertToCarbonType(field.dataType.getOrElse("")),
+        field.name.getOrElse(field.column),
         encoders,
-        isDimensionCol = true,
-        field.precision,
-        field.scale,
-        field.schemaOrdinal,
-        cm.highcardinalitydims.getOrElse(Seq()),
-        cm.databaseName)
+        true,
+        field,
+        cm.dataMapRelation)
       columnSchema.setSortColumn(true)
       allColumns :+= columnSchema
       index = index + 1
@@ -407,17 +477,24 @@ class TableNewProcessor(cm: TableModel) {
     cm.dimCols.foreach { field =>
       val sortField = cm.sortKeyDims.get.find(field.column equals _)
       if (sortField.isEmpty) {
-        val encoders = new java.util.ArrayList[Encoding]()
-        encoders.add(Encoding.DICTIONARY)
-        val columnSchema: ColumnSchema = TableNewProcessor.createColumnSchema(
-          field,
+        val encoders = if (cm.parentTable.isDefined &&
+                           cm.dataMapRelation.get.get(field).isDefined) {
+          cm.parentTable.get.getColumnByName(
+            cm.parentTable.get.getFactTableName,
+            cm.dataMapRelation.get.get(field).get.
+              columnTableRelation.get.parentColumnName).getEncoder
+        } else {
+          val encoders = new java.util.ArrayList[Encoding]()
+          encoders.add(Encoding.DICTIONARY)
+          encoders
+        }
+        val columnSchema = getColumnSchema(
+          DataTypeConverterUtil.convertToCarbonType(field.dataType.getOrElse("")),
+          field.name.getOrElse(field.column),
           encoders,
-          isDimensionCol = true,
-          field.precision,
-          field.scale,
-          field.schemaOrdinal,
-          cm.highcardinalitydims.getOrElse(Seq()),
-          cm.databaseName)
+          true,
+          field,
+          cm.dataMapRelation)
         allColumns :+= columnSchema
         index = index + 1
         if (field.children.isDefined && field.children.get != null) {
@@ -429,15 +506,13 @@ class TableNewProcessor(cm: TableModel) {
 
     cm.msrCols.foreach { field =>
       val encoders = new java.util.ArrayList[Encoding]()
-      val columnSchema: ColumnSchema = TableNewProcessor.createColumnSchema(
-        field,
+      val columnSchema = getColumnSchema(
+        DataTypeConverterUtil.convertToCarbonType(field.dataType.getOrElse("")),
+        field.name.getOrElse(field.column),
         encoders,
-        isDimensionCol = false,
-        field.precision,
-        field.scale,
-        field.schemaOrdinal,
-        cm.highcardinalitydims.getOrElse(Seq()),
-        cm.databaseName)
+        false,
+        field,
+        cm.dataMapRelation)
       allColumns :+= columnSchema
       index = index + 1
       measureCount += 1
@@ -486,15 +561,13 @@ class TableNewProcessor(cm: TableModel) {
         Some(CarbonCommonConstants.DEFAULT_INVISIBLE_DUMMY_MEASURE),
         None
       )
-      val columnSchema: ColumnSchema = TableNewProcessor.createColumnSchema(
-        field,
+      val columnSchema: ColumnSchema = getColumnSchema(
+        DataTypes.DOUBLE,
+        CarbonCommonConstants.DEFAULT_INVISIBLE_DUMMY_MEASURE,
         encoders,
-        isDimensionCol = false,
-        field.precision,
-        field.scale,
-        -1,
-        cm.highcardinalitydims.getOrElse(Seq()),
-        cm.databaseName)
+        false,
+        field,
+        cm.dataMapRelation)
       columnSchema.setInvisible(true)
       allColumns :+= columnSchema
     }
@@ -503,6 +576,7 @@ class TableNewProcessor(cm: TableModel) {
 
     val tableInfo = new TableInfo()
     val tableSchema = new TableSchema()
+
     val schemaEvol = new SchemaEvolution()
     schemaEvol.setSchemaEvolutionEntryList(new util.ArrayList[SchemaEvolutionEntry]())
     tableSchema.setTableId(UUID.randomUUID().toString)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1d560c06/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/LoadTableCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/LoadTableCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/LoadTableCommand.scala
index bda6829..222c30d 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/LoadTableCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/LoadTableCommand.scala
@@ -364,7 +364,7 @@ case class LoadTableCommand(
     entry.setTime_stamp(System.currentTimeMillis())
 
     // write TableInfo
-    metastore.updateTableSchema(carbonTablePath.getCarbonTableIdentifier,
+    metastore.updateTableSchemaForAlter(carbonTablePath.getCarbonTableIdentifier,
       carbonTablePath.getCarbonTableIdentifier,
       tableInfo, entry, carbonTablePath.getPath)(sparkSession)
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1d560c06/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala
new file mode 100644
index 0000000..ca384f9
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala
@@ -0,0 +1,136 @@
+/*
+ * 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.spark.sql.execution.command.preaaggregate
+
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.execution.command._
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.exception.InvalidConfigurationException
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
+import org.apache.carbondata.core.metadata.schema.table.{RelationIdentifier, TableInfo}
+import org.apache.carbondata.core.util.CarbonUtil
+
+/**
+ * Below command class will be used to create pre-aggregate table
+ * and updating the parent table about the child table information
+ * Failure case:
+ * 1. failed to create pre aggregate table.
+ * 2. failed to update main table
+ *
+ * @param cm
+ * @param dataFrame
+ * @param createDSTable
+ * @param queryString
+ */
+case class CreatePreAggregateTableCommand(
+    cm: TableModel,
+    dataFrame: DataFrame,
+    createDSTable: Boolean = true,
+    queryString: String,
+    fieldRelationMap: scala.collection.mutable.LinkedHashMap[Field, DataMapField])
+  extends RunnableCommand with SchemaProcessCommand {
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    processSchema(sparkSession)
+  }
+
+  override def processSchema(sparkSession: SparkSession): Seq[Row] = {
+    val storePath = CarbonEnv.getInstance(sparkSession).storePath
+    CarbonEnv.getInstance(sparkSession).carbonMetastore.
+      checkSchemasModifiedTimeAndReloadTables(storePath)
+    val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+    cm.databaseName = GetDB.getDatabaseName(cm.databaseNameOp, sparkSession)
+    val tbName = cm.tableName
+    val dbName = cm.databaseName
+    LOGGER.audit(s"Creating Table with Database name [$dbName] and Table name [$tbName]")
+    // getting the parent table
+    val parentTable = PreAggregateUtil.getParentCarbonTable(dataFrame.logicalPlan)
+    // getting the table name
+    val parentTableName = parentTable.getFactTableName
+    // getting the db name of parent table
+    val parentDbName = parentTable.getDatabaseName
+    // updating the relation identifier, this will be stored in child table
+    // which can be used during dropping of pre-aggreate table as parent table will
+    // also get updated
+    cm.parentTable = Some(parentTable)
+    cm.dataMapRelation = Some(fieldRelationMap)
+    val tableInfo: TableInfo = TableNewProcessor(cm)
+    // Add validation for sort scope when create table
+    val sortScope = tableInfo.getFactTable.getTableProperties
+      .getOrDefault("sort_scope", CarbonCommonConstants.LOAD_SORT_SCOPE_DEFAULT)
+    if (!CarbonUtil.isValidSortOption(sortScope)) {
+      throw new InvalidConfigurationException(
+        s"Passing invalid SORT_SCOPE '$sortScope', valid SORT_SCOPE are 'NO_SORT', 'BATCH_SORT'," +
+        s" 'LOCAL_SORT' and 'GLOBAL_SORT' ")
+    }
+
+    if (tableInfo.getFactTable.getListOfColumns.size <= 0) {
+      sys.error("No Dimensions found. Table should have at least one dimesnion !")
+    }
+
+    if (sparkSession.sessionState.catalog.listTables(dbName)
+      .exists(_.table.equalsIgnoreCase(tbName))) {
+      if (!cm.ifNotExistsSet) {
+        LOGGER.audit(
+          s"Table creation with Database name [$dbName] and Table name [$tbName] failed. " +
+          s"Table [$tbName] already exists under database [$dbName]")
+        sys.error(s"Table [$tbName] already exists under database [$dbName]")
+      }
+    } else {
+      val tableIdentifier = AbsoluteTableIdentifier.from(storePath, dbName, tbName)
+      // Add Database to catalog and persist
+      val catalog = CarbonEnv.getInstance(sparkSession).carbonMetastore
+      val tablePath = tableIdentifier.getTablePath
+      val carbonSchemaString = catalog.generateTableSchemaString(tableInfo, tablePath)
+      if (createDSTable) {
+        try {
+          val fields = new Array[Field](cm.dimCols.size + cm.msrCols.size)
+          cm.dimCols.foreach(f => fields(f.schemaOrdinal) = f)
+          cm.msrCols.foreach(f => fields(f.schemaOrdinal) = f)
+          sparkSession.sql(
+            s"""CREATE TABLE $dbName.$tbName
+               |(${ fields.map(f => f.rawSchema).mkString(",") })
+               |USING org.apache.spark.sql.CarbonSource""".stripMargin +
+            s""" OPTIONS (tableName "$tbName", dbName "$dbName", tablePath """.stripMargin +
+            s""""$tablePath"$carbonSchemaString) """)
+          // child schema object which will be updated on parent table about the
+          val childSchema = tableInfo.getFactTable
+            .buildChildSchema("", tableInfo.getDatabaseName, queryString, "AGGREGATION")
+          // upadting the parent table about child table
+          PreAggregateUtil.updateMainTable(parentDbName, parentTableName, childSchema, sparkSession)
+        } catch {
+          case e: Exception =>
+            val identifier: TableIdentifier = TableIdentifier(tbName, Some(dbName))
+            // call the drop table to delete the created table.
+            CarbonEnv.getInstance(sparkSession).carbonMetastore
+              .dropTable(tablePath, identifier)(sparkSession)
+            LOGGER.audit(s"Table creation with Database name [$dbName] " +
+                         s"and Table name [$tbName] failed")
+            throw e
+        }
+      }
+
+      LOGGER.audit(s"Table created with Database name [$dbName] and Table name [$tbName]")
+    }
+    Seq.empty
+  }
+}
+
+

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1d560c06/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala
new file mode 100644
index 0000000..c4b6783
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala
@@ -0,0 +1,431 @@
+/*
+ * 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.spark.sql.execution.command.preaaggregate
+
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.SparkConf
+import org.apache.spark.sql.{CarbonDatasourceHadoopRelation, CarbonEnv, SparkSession}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, Cast}
+import org.apache.spark.sql.catalyst.expressions.aggregate._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.execution.command.{ColumnTableRelation, DataMapField, Field}
+import org.apache.spark.sql.execution.datasources.LogicalRelation
+import org.apache.spark.sql.hive.{CarbonRelation, CarbonSessionState}
+import org.apache.spark.sql.hive.HiveExternalCatalog.{DATASOURCE_SCHEMA_NUMPARTS, DATASOURCE_SCHEMA_PART_PREFIX}
+import org.apache.spark.sql.types.DataType
+
+import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.locks.{CarbonLockUtil, ICarbonLock, LockUsage}
+import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl
+import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, DataMapSchema}
+import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.format.TableInfo
+import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
+import org.apache.carbondata.spark.util.CommonUtil
+
+/**
+ * Utility class for keeping all the utility method for pre-aggregate
+ */
+object PreAggregateUtil {
+
+  private val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+
+  def getParentCarbonTable(plan: LogicalPlan): CarbonTable = {
+    plan match {
+      case Aggregate(_, aExp, SubqueryAlias(_, l: LogicalRelation, _))
+        if l.relation.isInstanceOf[CarbonDatasourceHadoopRelation] =>
+        l.relation.asInstanceOf[CarbonDatasourceHadoopRelation].carbonRelation.metaData.carbonTable
+      case _ => throw new MalformedCarbonCommandException("table does not exist")
+    }
+  }
+
+  /**
+   * Below method will be used to validate the select plan
+   * and get the required fields from select plan
+   * Currently only aggregate query is support any other type of query will
+   * fail
+   * @param plan
+   * @param selectStmt
+   * @return list of fields
+   */
+  def validateActualSelectPlanAndGetAttrubites(plan: LogicalPlan,
+      selectStmt: String): scala.collection.mutable.LinkedHashMap[Field, DataMapField] = {
+    val fieldToDataMapFieldMap = scala.collection.mutable.LinkedHashMap.empty[Field, DataMapField]
+    plan match {
+      case Aggregate(_, aExp, SubqueryAlias(_, l: LogicalRelation, _))
+        if l.relation.isInstanceOf[CarbonDatasourceHadoopRelation] =>
+        val carbonTable = l.relation.asInstanceOf[CarbonDatasourceHadoopRelation].carbonRelation
+          .metaData.carbonTable
+        val parentTableName = carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier
+          .getTableName
+        val parentDatabaseName = carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier
+          .getDatabaseName
+        val parentTableId = carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier
+          .getTableId
+        if (!carbonTable.getTableInfo.getParentRelationIdentifiers.isEmpty) {
+          throw new MalformedCarbonCommandException(
+            "Pre Aggregation is not supported on Pre-Aggregated Table")
+        }
+        aExp.map {
+          case Alias(attr: AggregateExpression, _) =>
+            if (attr.isDistinct) {
+              throw new MalformedCarbonCommandException(
+                "Distinct is not supported On Pre Aggregation")
+            }
+            fieldToDataMapFieldMap ++= ((validateAggregateFunctionAndGetFields(carbonTable,
+              attr.aggregateFunction,
+              parentTableName,
+              parentDatabaseName,
+              parentTableId)))
+          case attr: AttributeReference =>
+            fieldToDataMapFieldMap += getField(attr.name,
+              attr.dataType,
+              parentColumnId = carbonTable.getColumnByName(parentTableName, attr.name).getColumnId,
+              parentTableName = parentTableName,
+              parentDatabaseName = parentDatabaseName, parentTableId = parentTableId)
+          case Alias(attr: AttributeReference, _) =>
+            fieldToDataMapFieldMap += getField(attr.name,
+              attr.dataType,
+              parentColumnId = carbonTable.getColumnByName(parentTableName, attr.name).getColumnId,
+              parentTableName = parentTableName,
+              parentDatabaseName = parentDatabaseName, parentTableId = parentTableId)
+          case _ =>
+            throw new MalformedCarbonCommandException(s"Unsupported Select Statement:${
+              selectStmt } ")
+        }
+        Some(carbonTable)
+      case _ =>
+        throw new MalformedCarbonCommandException(s"Unsupported Select Statement:${ selectStmt } ")
+    }
+    fieldToDataMapFieldMap
+  }
+
+  /**
+   * Below method will be used to validate about the aggregate function
+   * which is applied on select query.
+   * Currently sum, max, min, count, avg is supported
+   * in case of any other aggregate function it will throw error
+   * In case of avg it will return two fields one for count
+   * and other of sum of that column to support rollup
+   * @param carbonTable
+   * @param aggFunctions
+   * @param parentTableName
+   * @param parentDatabaseName
+   * @param parentTableId
+   * @return list of fields
+   */
+  def validateAggregateFunctionAndGetFields(carbonTable: CarbonTable,
+      aggFunctions: AggregateFunction,
+      parentTableName: String,
+      parentDatabaseName: String,
+      parentTableId: String) : scala.collection.mutable.ListBuffer[(Field, DataMapField)] = {
+    val list = scala.collection.mutable.ListBuffer.empty[(Field, DataMapField)]
+    aggFunctions match {
+      case sum@Sum(attr: AttributeReference) =>
+        list += getField(attr.name,
+          attr.dataType,
+          sum.prettyName,
+          carbonTable.getColumnByName(parentTableName, attr.name).getColumnId,
+          parentTableName,
+          parentDatabaseName, parentTableId = parentTableId)
+      case sum@Sum(Cast(attr: AttributeReference, changeDataType: DataType)) =>
+        list += getField(attr.name,
+          changeDataType,
+          sum.prettyName,
+          carbonTable.getColumnByName(parentTableName, attr.name).getColumnId,
+          parentTableName,
+          parentDatabaseName, parentTableId = parentTableId)
+      case count@Count(Seq(attr: AttributeReference)) =>
+        list += getField(attr.name,
+          attr.dataType,
+          count.prettyName,
+          carbonTable.getColumnByName(parentTableName, attr.name).getColumnId,
+          parentTableName,
+          parentDatabaseName, parentTableId = parentTableId)
+      case min@Min(attr: AttributeReference) =>
+        list += getField(attr.name,
+          attr.dataType,
+          min.prettyName,
+          carbonTable.getColumnByName(parentTableName, attr.name).getColumnId,
+          parentTableName,
+          parentDatabaseName, parentTableId = parentTableId)
+      case min@Min(Cast(attr: AttributeReference, changeDataType: DataType)) =>
+        list += getField(attr.name,
+          changeDataType,
+          min.prettyName,
+          carbonTable.getColumnByName(parentTableName, attr.name).getColumnId,
+          parentTableName,
+          parentDatabaseName, parentTableId = parentTableId)
+      case max@Max(attr: AttributeReference) =>
+        list += getField(attr.name,
+          attr.dataType,
+          max.prettyName,
+          carbonTable.getColumnByName(parentTableName, attr.name).getColumnId,
+          parentTableName,
+          parentDatabaseName, parentTableId = parentTableId)
+      case max@Max(Cast(attr: AttributeReference, changeDataType: DataType)) =>
+        list += getField(attr.name,
+          changeDataType,
+          max.prettyName,
+          carbonTable.getColumnByName(parentTableName, attr.name).getColumnId,
+          parentTableName,
+          parentDatabaseName, parentTableId = parentTableId)
+      case Average(attr: AttributeReference) =>
+        getField(attr.name,
+          attr.dataType,
+          "sum",
+          carbonTable.getColumnByName(parentTableName, attr.name).getColumnId,
+          parentTableName,
+          parentDatabaseName, parentTableId = parentTableId)
+        list += getField(attr.name,
+          attr.dataType,
+          "count",
+          carbonTable.getColumnByName(parentTableName, attr.name).getColumnId,
+          parentTableName,
+          parentDatabaseName, parentTableId = parentTableId)
+      case Average(Cast(attr: AttributeReference, changeDataType: DataType)) =>
+        list += getField(attr.name,
+          changeDataType,
+          "sum",
+          carbonTable.getColumnByName(parentTableName, attr.name).getColumnId,
+          parentTableName,
+          parentDatabaseName, parentTableId = parentTableId)
+        list += getField(attr.name,
+          changeDataType,
+          "count",
+          carbonTable.getColumnByName(parentTableName, attr.name).getColumnId,
+          parentTableName,
+          parentDatabaseName, parentTableId = parentTableId)
+      case _ =>
+        throw new MalformedCarbonCommandException("Un-Supported Aggregation Type")
+    }
+  }
+
+  /**
+   * Below method will be used to get the fields object for pre aggregate table
+   * @param columnName
+   * @param dataType
+   * @param aggregateType
+   * @param parentColumnId
+   * @param parentTableName
+   * @param parentDatabaseName
+   * @param parentTableId
+   * @return fields object
+   */
+  def getField(columnName: String,
+      dataType: DataType,
+      aggregateType: String = "",
+      parentColumnId: String,
+      parentTableName: String,
+      parentDatabaseName: String,
+      parentTableId: String): (Field, DataMapField) = {
+    val actualColumnName = if (aggregateType.equals("")) {
+      parentTableName + '_' + columnName
+    } else {
+      parentTableName + '_' + columnName + '_' + aggregateType
+    }
+    val rawSchema = '`' + actualColumnName + '`' + ' ' + dataType.typeName
+    val columnTableRelation = ColumnTableRelation(parentColumnName = columnName,
+      parentColumnId = parentColumnId,
+      parentTableName = parentTableName,
+      parentDatabaseName = parentDatabaseName, parentTableId = parentTableId)
+    val dataMapField = DataMapField(aggregateType, Some(columnTableRelation))
+    if (dataType.typeName.startsWith("decimal")) {
+      val (precision, scale) = CommonUtil.getScaleAndPrecision(dataType.catalogString)
+      (Field(column = actualColumnName,
+        dataType = Some(dataType.typeName),
+        name = Some(actualColumnName),
+        children = None,
+        precision = precision,
+        scale = scale,
+        rawSchema = rawSchema), dataMapField)
+    }
+    else {
+      (Field(column = actualColumnName,
+        dataType = Some(dataType.typeName),
+        name = Some(actualColumnName),
+        children = None,
+        rawSchema = rawSchema), dataMapField)
+    }
+  }
+
+  /**
+   * Below method will be used to update the main table about the pre aggregate table information
+   * in case of any exption it will throw error so pre aggregate table creation will fail
+   * @param dbName
+   * @param tableName
+   * @param childSchema
+   * @param sparkSession
+   */
+  def updateMainTable(dbName: String, tableName: String,
+      childSchema: DataMapSchema, sparkSession: SparkSession): Unit = {
+    val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+    val locksToBeAcquired = List(LockUsage.METADATA_LOCK,
+      LockUsage.DROP_TABLE_LOCK)
+    var locks = List.empty[ICarbonLock]
+    var carbonTable: CarbonTable = null
+    var numberOfCurrentChild: Int = 0
+    try {
+      val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
+      carbonTable = metastore
+        .lookupRelation(Some(dbName), tableName)(sparkSession).asInstanceOf[CarbonRelation]
+        .tableMeta.carbonTable
+      locks = acquireLock(dbName, tableName, locksToBeAcquired, carbonTable)
+      // get the latest carbon table and check for column existence
+      // read the latest schema file
+      val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getStorePath,
+        carbonTable.getCarbonTableIdentifier)
+      val thriftTableInfo: TableInfo = metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
+      val schemaConverter = new ThriftWrapperSchemaConverterImpl()
+      val wrapperTableInfo = schemaConverter
+        .fromExternalToWrapperTableInfo(thriftTableInfo,
+          dbName,
+          tableName,
+          carbonTable.getStorePath)
+      numberOfCurrentChild = wrapperTableInfo.getDataMapSchemaList.size
+      wrapperTableInfo.getDataMapSchemaList.add(childSchema)
+      val thriftTable = schemaConverter
+        .fromWrapperToExternalTableInfo(wrapperTableInfo, dbName, tableName)
+      updateSchemaInfo(carbonTable,
+        thriftTable)(sparkSession,
+        sparkSession.sessionState.asInstanceOf[CarbonSessionState])
+      LOGGER.info(s"Pre Aggeragte Parent table updated is successful for table $dbName.$tableName")
+    } catch {
+      case e: Exception =>
+        LOGGER.error(e, "Pre Aggregate Parent table update failed reverting changes")
+        revertMainTableChanges(dbName, tableName, numberOfCurrentChild)(sparkSession)
+        throw e
+    } finally {
+      // release lock after command execution completion
+      releaseLocks(locks)
+    }
+    Seq.empty
+  }
+
+  /**
+   * Below method will be used to update the main table schema
+   * @param carbonTable
+   * @param thriftTable
+   * @param sparkSession
+   * @param sessionState
+   */
+  def updateSchemaInfo(carbonTable: CarbonTable,
+      thriftTable: TableInfo)(sparkSession: SparkSession,
+      sessionState: CarbonSessionState): Unit = {
+    val dbName = carbonTable.getDatabaseName
+    val tableName = carbonTable.getFactTableName
+    CarbonEnv.getInstance(sparkSession).carbonMetastore
+      .updateTableSchemaForDataMap(carbonTable.getCarbonTableIdentifier,
+        carbonTable.getCarbonTableIdentifier,
+        thriftTable,
+        carbonTable.getAbsoluteTableIdentifier.getTablePath)(sparkSession)
+    val tableIdentifier = TableIdentifier(tableName, Some(dbName))
+    sparkSession.catalog.refreshTable(tableIdentifier.quotedString)
+  }
+
+  /**
+   * This method will split schema string into multiple parts of configured size and
+   * registers the parts as keys in tableProperties which will be read by spark to prepare
+   * Carbon Table fields
+   *
+   * @param sparkConf
+   * @param schemaJsonString
+   * @return
+   */
+  private def prepareSchemaJson(sparkConf: SparkConf,
+      schemaJsonString: String): String = {
+    val threshold = sparkConf
+      .getInt(CarbonCommonConstants.SPARK_SCHEMA_STRING_LENGTH_THRESHOLD,
+        CarbonCommonConstants.SPARK_SCHEMA_STRING_LENGTH_THRESHOLD_DEFAULT)
+    // Split the JSON string.
+    val parts = schemaJsonString.grouped(threshold).toSeq
+    var schemaParts: Seq[String] = Seq.empty
+    schemaParts = schemaParts :+ s"'$DATASOURCE_SCHEMA_NUMPARTS'='${ parts.size }'"
+    parts.zipWithIndex.foreach { case (part, index) =>
+      schemaParts = schemaParts :+ s"'$DATASOURCE_SCHEMA_PART_PREFIX$index'='$part'"
+    }
+    schemaParts.mkString(",")
+  }
+
+  /**
+   * Validates that the table exists and acquires meta lock on it.
+   *
+   * @param dbName
+   * @param tableName
+   * @return
+   */
+  def acquireLock(dbName: String,
+      tableName: String,
+      locksToBeAcquired: List[String],
+      table: CarbonTable): List[ICarbonLock] = {
+    // acquire the lock first
+    val acquiredLocks = ListBuffer[ICarbonLock]()
+    try {
+      locksToBeAcquired.foreach { lock =>
+        acquiredLocks += CarbonLockUtil.getLockObject(table.getCarbonTableIdentifier, lock)
+      }
+      acquiredLocks.toList
+    } catch {
+      case e: Exception =>
+        releaseLocks(acquiredLocks.toList)
+        throw e
+    }
+  }
+
+  /**
+   * This method will release the locks acquired for an operation
+   *
+   * @param locks
+   */
+  def releaseLocks(locks: List[ICarbonLock]): Unit = {
+    locks.foreach { carbonLock =>
+      if (carbonLock.unlock()) {
+        LOGGER.info("Pre agg table lock released successfully")
+      } else {
+        LOGGER.error("Unable to release lock during Pre agg table cretion")
+      }
+    }
+  }
+
+  /**
+   * This method reverts the changes to the schema if add column command fails.
+   *
+   * @param dbName
+   * @param tableName
+   * @param numberOfChildSchema
+   * @param sparkSession
+   */
+  def revertMainTableChanges(dbName: String, tableName: String, numberOfChildSchema: Int)
+    (sparkSession: SparkSession): Unit = {
+    val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
+    val carbonTable = metastore
+      .lookupRelation(Some(dbName), tableName)(sparkSession).asInstanceOf[CarbonRelation].tableMeta
+      .carbonTable
+    carbonTable.getTableLastUpdatedTime
+    val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getStorePath,
+      carbonTable.getCarbonTableIdentifier)
+    val thriftTable: TableInfo = metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
+    if (thriftTable.dataMapSchemas.size > numberOfChildSchema) {
+      metastore
+        .revertTableSchemaForPreAggCreationFailure(carbonTable.getCarbonTableIdentifier,
+          thriftTable, carbonTable.getAbsoluteTableIdentifier.getTablePath)(sparkSession)
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1d560c06/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala
index 5936355..e0617d6 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala
@@ -109,7 +109,7 @@ private[sql] case class CarbonAlterTableRenameCommand(
       }
       val newTableIdentifier = new CarbonTableIdentifier(oldDatabaseName,
         newTableName, carbonTable.getCarbonTableIdentifier.getTableId)
-      val newTablePath = metastore.updateTableSchema(newTableIdentifier,
+      val newTablePath = metastore.updateTableSchemaForAlter(newTableIdentifier,
         carbonTable.getCarbonTableIdentifier,
         tableInfo,
         schemaEvolutionEntry,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1d560c06/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala
index 9822d8f..51c7f3b 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala
@@ -43,6 +43,7 @@ import org.apache.carbondata.core.util.CarbonUtil
 import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
 import org.apache.carbondata.core.writer.ThriftWriter
 import org.apache.carbondata.events.{LookupRelationPostEvent, OperationContext, OperationListenerBus}
+import org.apache.carbondata.format
 import org.apache.carbondata.format.{SchemaEvolutionEntry, TableInfo}
 import org.apache.carbondata.processing.merger.TableMeta
 import org.apache.carbondata.spark.util.CarbonSparkUtil
@@ -111,6 +112,22 @@ class CarbonFileMetastore extends CarbonMetaStore {
     }
   }
 
+  /**
+   * This method will overwrite the existing schema and update it with the given details
+   *
+   * @param newTableIdentifier
+   * @param thriftTableInfo
+   * @param carbonStorePath
+   * @param sparkSession
+   */
+  def updateTableSchemaForDataMap(newTableIdentifier: CarbonTableIdentifier,
+      oldTableIdentifier: CarbonTableIdentifier,
+      thriftTableInfo: org.apache.carbondata.format.TableInfo,
+      carbonStorePath: String)(sparkSession: SparkSession): String = {
+    updateTableSchemaForAlter(newTableIdentifier,
+      oldTableIdentifier, thriftTableInfo, null, carbonStorePath) (sparkSession)
+  }
+
   def lookupRelation(dbName: Option[String], tableName: String)
     (sparkSession: SparkSession): LogicalPlan = {
     lookupRelation(TableIdentifier(tableName, dbName))(sparkSession)
@@ -214,7 +231,7 @@ class CarbonFileMetastore extends CarbonMetaStore {
    * @param tablePath
    * @param sparkSession
    */
-  def updateTableSchema(newTableIdentifier: CarbonTableIdentifier,
+  def updateTableSchemaForAlter(newTableIdentifier: CarbonTableIdentifier,
       oldTableIdentifier: CarbonTableIdentifier,
       thriftTableInfo: org.apache.carbondata.format.TableInfo,
       schemaEvolutionEntry: SchemaEvolutionEntry,
@@ -251,7 +268,7 @@ class CarbonFileMetastore extends CarbonMetaStore {
    * @param tablePath
    * @param sparkSession
    */
-  def revertTableSchema(carbonTableIdentifier: CarbonTableIdentifier,
+  def revertTableSchemaInAlterFailure(carbonTableIdentifier: CarbonTableIdentifier,
       thriftTableInfo: org.apache.carbondata.format.TableInfo,
       tablePath: String)(sparkSession: SparkSession): String = {
     val tableIdentifier = AbsoluteTableIdentifier.fromTablePath(tablePath)
@@ -271,7 +288,27 @@ class CarbonFileMetastore extends CarbonMetaStore {
     path
   }
 
+  override def revertTableSchemaForPreAggCreationFailure(carbonTableIdentifier:
+  CarbonTableIdentifier,
+      thriftTableInfo: org.apache.carbondata.format.TableInfo,
+      tablePath: String)(sparkSession: SparkSession): String = {
+    val tableIdentifier = AbsoluteTableIdentifier.fromTablePath(tablePath)
+    val schemaConverter = new ThriftWrapperSchemaConverterImpl
+    val wrapperTableInfo = schemaConverter
+      .fromExternalToWrapperTableInfo(thriftTableInfo,
+        carbonTableIdentifier.getDatabaseName,
+        carbonTableIdentifier.getTableName,
+        tableIdentifier.getStorePath)
+    val childSchemaList = wrapperTableInfo.getDataMapSchemaList
+    childSchemaList.remove(childSchemaList.size() - 1)
+    wrapperTableInfo.setStorePath(tableIdentifier.getStorePath)
+    val path = createSchemaThriftFile(wrapperTableInfo,
+      thriftTableInfo,
+      tableIdentifier.getCarbonTableIdentifier)
+    addTableCache(wrapperTableInfo, tableIdentifier)
+    path
 
+  }
 
   /**
    *

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1d560c06/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonHiveMetaStore.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonHiveMetaStore.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonHiveMetaStore.scala
index 76241a6..c64b7bb 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonHiveMetaStore.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonHiveMetaStore.scala
@@ -115,7 +115,7 @@ class CarbonHiveMetaStore extends CarbonFileMetastore {
    * @param schemaEvolutionEntry
    * @param sparkSession
    */
-  override def updateTableSchema(newTableIdentifier: CarbonTableIdentifier,
+  override def updateTableSchemaForAlter(newTableIdentifier: CarbonTableIdentifier,
       oldTableIdentifier: CarbonTableIdentifier,
       thriftTableInfo: format.TableInfo,
       schemaEvolutionEntry: SchemaEvolutionEntry,
@@ -126,7 +126,7 @@ class CarbonHiveMetaStore extends CarbonFileMetastore {
     if (schemaEvolutionEntry != null) {
       thriftTableInfo.fact_table.schema_evolution.schema_evolution_history.add(schemaEvolutionEntry)
     }
-    updateHiveMetaStore(newTableIdentifier,
+    updateHiveMetaStoreForAlter(newTableIdentifier,
       oldTableIdentifier,
       thriftTableInfo,
       identifier.getStorePath,
@@ -134,7 +134,29 @@ class CarbonHiveMetaStore extends CarbonFileMetastore {
       schemaConverter)
   }
 
-  private def updateHiveMetaStore(newTableIdentifier: CarbonTableIdentifier,
+  /**
+   * This method will overwrite the existing schema and update it with the given details
+   *
+   * @param newTableIdentifier
+   * @param thriftTableInfo
+   * @param carbonStorePath
+   * @param sparkSession
+   */
+  override def updateTableSchemaForDataMap(newTableIdentifier: CarbonTableIdentifier,
+      oldTableIdentifier: CarbonTableIdentifier,
+      thriftTableInfo: org.apache.carbondata.format.TableInfo,
+      carbonStorePath: String)(sparkSession: SparkSession): String = {
+    val schemaConverter = new ThriftWrapperSchemaConverterImpl
+    val identifier = AbsoluteTableIdentifier.fromTablePath(carbonStorePath)
+    updateHiveMetaStoreForDataMap(newTableIdentifier,
+      oldTableIdentifier,
+      thriftTableInfo,
+      identifier.getStorePath,
+      sparkSession,
+      schemaConverter)
+  }
+
+  private def updateHiveMetaStoreForAlter(newTableIdentifier: CarbonTableIdentifier,
       oldTableIdentifier: CarbonTableIdentifier,
       thriftTableInfo: format.TableInfo,
       carbonStorePath: String,
@@ -161,6 +183,30 @@ class CarbonHiveMetaStore extends CarbonFileMetastore {
     CarbonStorePath.getCarbonTablePath(carbonStorePath, newTableIdentifier).getPath
   }
 
+  private def updateHiveMetaStoreForDataMap(newTableIdentifier: CarbonTableIdentifier,
+      oldTableIdentifier: CarbonTableIdentifier,
+      thriftTableInfo: format.TableInfo,
+      carbonStorePath: String,
+      sparkSession: SparkSession,
+      schemaConverter: ThriftWrapperSchemaConverterImpl) = {
+    val wrapperTableInfo = schemaConverter
+      .fromExternalToWrapperTableInfo(thriftTableInfo,
+        newTableIdentifier.getDatabaseName,
+        newTableIdentifier.getTableName,
+        carbonStorePath)
+    wrapperTableInfo.setStorePath(carbonStorePath)
+    val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonStorePath, newTableIdentifier)
+    val schemaMetadataPath =
+      CarbonTablePath.getFolderContainingFile(carbonTablePath.getSchemaFilePath)
+    wrapperTableInfo.setMetaDataFilepath(schemaMetadataPath)
+    val dbName = oldTableIdentifier.getDatabaseName
+    val tableName = oldTableIdentifier.getTableName
+    sparkSession.catalog.refreshTable(TableIdentifier(tableName, Some(dbName)).quotedString)
+    removeTableFromMetadata(dbName, tableName)
+    CarbonMetadata.getInstance().loadTableMetadata(wrapperTableInfo)
+    CarbonStorePath.getCarbonTablePath(carbonStorePath, newTableIdentifier).getPath
+  }
+
   /**
    * This method will is used to remove the evolution entry in case of failure.
    *
@@ -168,7 +214,7 @@ class CarbonHiveMetaStore extends CarbonFileMetastore {
    * @param thriftTableInfo
    * @param sparkSession
    */
-  override def revertTableSchema(carbonTableIdentifier: CarbonTableIdentifier,
+  override def revertTableSchemaInAlterFailure(carbonTableIdentifier: CarbonTableIdentifier,
       thriftTableInfo: format.TableInfo,
       tablePath: String)
     (sparkSession: SparkSession): String = {
@@ -176,7 +222,23 @@ class CarbonHiveMetaStore extends CarbonFileMetastore {
     val identifier = AbsoluteTableIdentifier.fromTablePath(tablePath)
     val evolutionEntries = thriftTableInfo.fact_table.schema_evolution.schema_evolution_history
     evolutionEntries.remove(evolutionEntries.size() - 1)
-    updateHiveMetaStore(carbonTableIdentifier,
+    updateHiveMetaStoreForAlter(carbonTableIdentifier,
+      carbonTableIdentifier,
+      thriftTableInfo,
+      identifier.getStorePath,
+      sparkSession,
+      schemaConverter)
+  }
+
+  override def revertTableSchemaForPreAggCreationFailure(carbonTableIdentifier:
+  CarbonTableIdentifier,
+      thriftTableInfo: org.apache.carbondata.format.TableInfo,
+      tablePath: String)(sparkSession: SparkSession): String = {
+    val schemaConverter = new ThriftWrapperSchemaConverterImpl
+    val identifier = AbsoluteTableIdentifier.fromTablePath(tablePath)
+    val childSchemas = thriftTableInfo.dataMapSchemas
+    childSchemas.remove(childSchemas.size())
+    updateHiveMetaStoreForAlter(carbonTableIdentifier,
       carbonTableIdentifier,
       thriftTableInfo,
       identifier.getStorePath,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1d560c06/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonMetaStore.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonMetaStore.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonMetaStore.scala
index dcb43d1..24996ed 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonMetaStore.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonMetaStore.scala
@@ -66,13 +66,26 @@ trait CarbonMetaStore {
    * @param carbonStorePath
    * @param sparkSession
    */
-  def updateTableSchema(newTableIdentifier: CarbonTableIdentifier,
+  def updateTableSchemaForAlter(newTableIdentifier: CarbonTableIdentifier,
       oldTableIdentifier: CarbonTableIdentifier,
       thriftTableInfo: org.apache.carbondata.format.TableInfo,
       schemaEvolutionEntry: SchemaEvolutionEntry,
       carbonStorePath: String)(sparkSession: SparkSession): String
 
   /**
+   * This method will overwrite the existing schema and update it with the given details
+   *
+   * @param newTableIdentifier
+   * @param thriftTableInfo
+   * @param carbonStorePath
+   * @param sparkSession
+   */
+  def updateTableSchemaForDataMap(newTableIdentifier: CarbonTableIdentifier,
+      oldTableIdentifier: CarbonTableIdentifier,
+      thriftTableInfo: org.apache.carbondata.format.TableInfo,
+      carbonStorePath: String)(sparkSession: SparkSession): String
+
+  /**
    * This method will is used to remove the evolution entry in case of failure.
    *
    * @param carbonTableIdentifier
@@ -80,11 +93,15 @@ trait CarbonMetaStore {
    * @param tablePath
    * @param sparkSession
    */
-  def revertTableSchema(carbonTableIdentifier: CarbonTableIdentifier,
+  def revertTableSchemaInAlterFailure(carbonTableIdentifier: CarbonTableIdentifier,
       thriftTableInfo: org.apache.carbondata.format.TableInfo,
       tablePath: String)
     (sparkSession: SparkSession): String
 
+
+  def revertTableSchemaForPreAggCreationFailure(carbonTableIdentifier: CarbonTableIdentifier,
+      thriftTableInfo: org.apache.carbondata.format.TableInfo,
+      tablePath: String)(sparkSession: SparkSession): String
   /**
    * Prepare Thrift Schema from wrapper TableInfo and write to disk
    */

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1d560c06/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
index fc2ed41..bf21bc8 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
@@ -440,7 +440,7 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
       // so checking the start of the string and taking the precision and scale.
       // resetting the data type with decimal
       if (f.dataType.getOrElse("").startsWith("decimal")) {
-        val (precision, scale) = getScaleAndPrecision(col.dataType.catalogString)
+        val (precision, scale) = CommonUtil.getScaleAndPrecision(col.dataType.catalogString)
         f.precision = precision
         f.scale = scale
         f.dataType = Some("decimal")

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1d560c06/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala
index a53e71f..256477e 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala
@@ -18,14 +18,15 @@ package org.apache.spark.sql.parser
 
 import scala.collection.mutable
 
-import org.apache.spark.sql.{CarbonSession, SparkSession}
+import org.apache.spark.sql.{CarbonSession, DataFrame, Dataset, SparkSession}
 import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.catalyst.parser.{AbstractSqlParser, ParseException, SqlBaseParser}
 import org.apache.spark.sql.catalyst.parser.ParserUtils._
 import org.apache.spark.sql.catalyst.parser.SqlBaseParser.{CreateTableContext, TablePropertyListContext}
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
 import org.apache.spark.sql.execution.SparkSqlAstBuilder
-import org.apache.spark.sql.execution.command.{BucketFields, CarbonCreateTableCommand, PartitionerField, TableModel}
+import org.apache.spark.sql.execution.command._
+import org.apache.spark.sql.execution.command.preaaggregate.{CreatePreAggregateTableCommand, PreAggregateUtil}
 import org.apache.spark.sql.internal.{SQLConf, VariableSubstitution}
 import org.apache.spark.sql.types.StructField
 
@@ -39,7 +40,7 @@ import org.apache.carbondata.spark.util.CommonUtil
  */
 class CarbonSparkSqlParser(conf: SQLConf, sparkSession: SparkSession) extends AbstractSqlParser {
 
-  val astBuilder = new CarbonSqlAstBuilder(conf)
+  val astBuilder = new CarbonSqlAstBuilder(conf, sparkSession: SparkSession)
 
   private val substitutor = new VariableSubstitution(conf)
 
@@ -69,7 +70,8 @@ class CarbonSparkSqlParser(conf: SQLConf, sparkSession: SparkSession) extends Ab
   }
 }
 
-class CarbonSqlAstBuilder(conf: SQLConf) extends SparkSqlAstBuilder(conf) {
+class CarbonSqlAstBuilder(conf: SQLConf, sparkSession: SparkSession)
+  extends SparkSqlAstBuilder(conf) {
 
   val parser = new CarbonSpark2SqlParser
 
@@ -117,8 +119,18 @@ class CarbonSqlAstBuilder(conf: SQLConf) extends SparkSqlAstBuilder(conf) {
       val (partitionByStructFields, partitionFields) =
         validateParitionFields(ctx, colNames, tableProperties)
 
-      val fields = parser.getFields(colsStructFields ++ partitionByStructFields)
-
+      val isAggTable = tableProperties.get("parent").isDefined
+      var fields = parser.getFields(colsStructFields ++ partitionByStructFields)
+      val dfAndFieldRelationTuple = if (isAggTable) {
+        val selectQuery = Option(ctx.query).map(plan).get
+        val df = Dataset.ofRows(sparkSession, selectQuery)
+        val fieldRelationMap = PreAggregateUtil
+          .validateActualSelectPlanAndGetAttrubites(df.logicalPlan, source(ctx.query()))
+        fields = fieldRelationMap.keySet.toSeq
+        Some(df, fieldRelationMap)
+      } else {
+        None
+      }
       // validate bucket fields
       val bucketFields: Option[BucketFields] =
         parser.getBucketFields(tableProperties, fields, options)
@@ -137,7 +149,14 @@ class CarbonSqlAstBuilder(conf: SQLConf) extends SparkSqlAstBuilder(conf) {
         isAlterFlow = false,
         tableComment)
 
-      CarbonCreateTableCommand(tableModel)
+      if(!isAggTable) {
+        CarbonCreateTableCommand(tableModel)
+      } else {
+        CreatePreAggregateTableCommand(tableModel,
+          dfAndFieldRelationTuple.get._1,
+          queryString = source(ctx.query).toString,
+          fieldRelationMap = dfAndFieldRelationTuple.get._2)
+      }
     } else {
       super.visitCreateTable(ctx)
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1d560c06/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala b/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala
index 44f5a36..bda4eeb 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala
@@ -136,7 +136,7 @@ object AlterTableUtil {
     val dbName = carbonTable.getDatabaseName
     val tableName = carbonTable.getFactTableName
     CarbonEnv.getInstance(sparkSession).carbonMetastore
-      .updateTableSchema(carbonTable.getCarbonTableIdentifier,
+      .updateTableSchemaForAlter(carbonTable.getCarbonTableIdentifier,
         carbonTable.getCarbonTableIdentifier,
         thriftTable,
         schemaEvolutionEntry,
@@ -211,7 +211,7 @@ object AlterTableUtil {
           .renameForce(carbonTablePath.getParent.toString + CarbonCommonConstants.FILE_SEPARATOR +
                        oldTableIdentifier.table)
         val tableIdentifier = new CarbonTableIdentifier(database, oldTableIdentifier.table, tableId)
-        metastore.revertTableSchema(tableIdentifier,
+        metastore.revertTableSchemaInAlterFailure(tableIdentifier,
           tableInfo, carbonTablePath.getPath)(sparkSession)
         metastore.removeTableFromMetadata(database, newTableName)
       }
@@ -243,7 +243,7 @@ object AlterTableUtil {
       val addedSchemas = evolutionEntryList.get(evolutionEntryList.size() - 1).added
       thriftTable.fact_table.table_columns.removeAll(addedSchemas)
       metastore
-        .revertTableSchema(carbonTable.getCarbonTableIdentifier,
+        .revertTableSchemaInAlterFailure(carbonTable.getCarbonTableIdentifier,
           thriftTable, carbonTable.getAbsoluteTableIdentifier.getTablePath)(sparkSession)
     }
   }
@@ -278,7 +278,7 @@ object AlterTableUtil {
         }
       }
       metastore
-        .revertTableSchema(carbonTable.getCarbonTableIdentifier,
+        .revertTableSchemaInAlterFailure(carbonTable.getCarbonTableIdentifier,
           thriftTable, carbonTable.getAbsoluteTableIdentifier.getTablePath)(sparkSession)
     }
   }
@@ -316,7 +316,7 @@ object AlterTableUtil {
         }
       }
       metastore
-        .revertTableSchema(carbonTable.getCarbonTableIdentifier,
+        .revertTableSchemaInAlterFailure(carbonTable.getCarbonTableIdentifier,
           thriftTable, carbonTable.getAbsoluteTableIdentifier.getTablePath)(sparkSession)
     }
   }


[11/11] carbondata git commit: [CARBONDATA-1524][CARBONDATA-1525][AggTable] Added support for aggregate table drop

Posted by ra...@apache.org.
[CARBONDATA-1524][CARBONDATA-1525][AggTable] Added support for aggregate table drop

This closes #1443


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

Branch: refs/heads/pre-aggregate
Commit: ee6bd2f86168dce4ebd0b3464a527e509307055d
Parents: 1d560c0
Author: kunal642 <ku...@gmail.com>
Authored: Wed Oct 18 20:09:04 2017 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Sat Nov 11 17:32:02 2017 +0530

----------------------------------------------------------------------
 .../table/column/ParentColumnTableRelation.java |  2 +
 .../preaggregate/TestPreAggregateDrop.scala     | 67 ++++++++++++++++++++
 .../carbondata/events/DropTableEvents.scala     |  6 +-
 .../org/apache/carbondata/events/Events.scala   |  2 +-
 .../command/carbonTableSchemaCommon.scala       |  2 +-
 .../command/CarbonDropTableCommand.scala        | 29 ++++++---
 .../CreatePreAggregateTableCommand.scala        |  4 +-
 .../DropPreAggregateTablePostListener.scala     | 49 ++++++++++++++
 .../preaaggregate/PreAggregateUtil.scala        | 26 ++++----
 .../spark/sql/hive/CarbonFileMetastore.scala    | 53 ++++++++++++++--
 .../spark/sql/hive/CarbonHiveMetaStore.scala    | 13 +++-
 .../spark/sql/hive/CarbonSessionState.scala     | 13 ++++
 12 files changed, 231 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/ee6bd2f8/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/ParentColumnTableRelation.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/ParentColumnTableRelation.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/ParentColumnTableRelation.java
index 425d0f2..28dc12c 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/ParentColumnTableRelation.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/ParentColumnTableRelation.java
@@ -29,6 +29,8 @@ import org.apache.carbondata.core.metadata.schema.table.Writable;
  */
 public class ParentColumnTableRelation implements Serializable, Writable {
 
+  private static final long serialVersionUID = 1321746085997166646L;
+
   private RelationIdentifier relationIdentifier;
   /**
    * parent column id

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ee6bd2f8/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateDrop.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateDrop.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateDrop.scala
new file mode 100644
index 0000000..4dad3e1
--- /dev/null
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateDrop.scala
@@ -0,0 +1,67 @@
+/*
+ * 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.integration.spark.testsuite.preaggregate
+
+import org.apache.spark.sql.test.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+class TestPreAggregateDrop extends QueryTest with BeforeAndAfterAll {
+
+  override def beforeAll {
+    sql("drop table if exists maintable")
+    sql("drop table if exists preagg1")
+    sql("drop table if exists preagg2")
+    sql("create table maintable (a string, b string, c string) stored by 'carbondata'")
+  }
+
+  test("create and drop preaggregate table") {
+    sql(
+      "create table preagg1 stored BY 'carbondata' tblproperties('parent'='maintable') as select" +
+      " a,sum(b) from maintable group by a")
+    sql("drop table if exists preagg1")
+    checkExistence(sql("show tables"), false, "preagg1")
+  }
+
+  test("dropping 1 aggregate table should not drop others") {
+    sql(
+      "create table preagg1 stored BY 'carbondata' tblproperties('parent'='maintable') as select" +
+      " a,sum(b) from maintable group by a")
+    sql(
+      "create table preagg2 stored BY 'carbondata' tblproperties('parent'='maintable') as select" +
+      " a,sum(c) from maintable group by a")
+    sql("drop table if exists preagg2")
+    val showTables = sql("show tables")
+    checkExistence(showTables, false, "preagg2")
+    checkExistence(showTables, true, "preagg1")
+  }
+  
+  test("drop main table and check if preaggreagte is deleted") {
+    sql(
+      "create table preagg2 stored BY 'carbondata' tblproperties('parent'='maintable') as select" +
+      " a,sum(c) from maintable group by a")
+    sql("drop table if exists maintable")
+    checkExistence(sql("show tables"), false, "preagg1", "maintable", "preagg2")
+  }
+
+  override def afterAll() {
+    sql("drop table if exists maintable")
+    sql("drop table if exists preagg1")
+    sql("drop table if exists preagg2")
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ee6bd2f8/integration/spark-common/src/main/scala/org/apache/carbondata/events/DropTableEvents.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/events/DropTableEvents.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/events/DropTableEvents.scala
index ed43de6..ab77fba 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/events/DropTableEvents.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/events/DropTableEvents.scala
@@ -27,7 +27,7 @@ import org.apache.carbondata.core.metadata.schema.table.CarbonTable
  * @param ifExistsSet
  * @param sparkSession
  */
-case class DropTablePreEvent(carbonTable: CarbonTable,
+case class DropTablePreEvent(carbonTable: Option[CarbonTable],
     ifExistsSet: Boolean,
     sparkSession: SparkSession)
   extends Event with DropTableEventInfo
@@ -39,7 +39,7 @@ case class DropTablePreEvent(carbonTable: CarbonTable,
  * @param ifExistsSet
  * @param sparkSession
  */
-case class DropTablePostEvent(carbonTable: CarbonTable,
+case class DropTablePostEvent(carbonTable: Option[CarbonTable],
     ifExistsSet: Boolean,
     sparkSession: SparkSession)
   extends Event with DropTableEventInfo
@@ -51,7 +51,7 @@ case class DropTablePostEvent(carbonTable: CarbonTable,
  * @param ifExistsSet
  * @param sparkSession
  */
-case class DropTableAbortEvent(carbonTable: CarbonTable,
+case class DropTableAbortEvent(carbonTable: Option[CarbonTable],
     ifExistsSet: Boolean,
     sparkSession: SparkSession)
   extends Event with DropTableEventInfo

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ee6bd2f8/integration/spark-common/src/main/scala/org/apache/carbondata/events/Events.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/events/Events.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/events/Events.scala
index 0d923ed..4f8d57e 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/events/Events.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/events/Events.scala
@@ -57,7 +57,7 @@ trait LookupRelationEventInfo {
  * event for drop table
  */
 trait DropTableEventInfo {
-  val carbonTable: CarbonTable
+  val carbonTable: Option[CarbonTable]
   val ifExistsSet: Boolean
 }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ee6bd2f8/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala
index 37ba8a5..759d3d8 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala
@@ -428,7 +428,7 @@ class TableNewProcessor(cm: TableModel) {
     columnSchema.setSortColumn(false)
     if(isParentColumnRelation) {
       val dataMapField = map.get.get(field).get
-      columnSchema.setAggFunction(dataMapField.aggregateFunction);
+      columnSchema.setAggFunction(dataMapField.aggregateFunction)
         val relation = dataMapField.columnTableRelation.get
         val parentColumnTableRelationList = new util.ArrayList[ParentColumnTableRelation]
         val relationIdentifier = new RelationIdentifier(

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ee6bd2f8/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonDropTableCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonDropTableCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonDropTableCommand.scala
index 5905493..a8e6c37 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonDropTableCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonDropTableCommand.scala
@@ -21,12 +21,14 @@ import scala.collection.mutable.ListBuffer
 
 import org.apache.spark.sql.{CarbonEnv, GetDB, Row, SparkSession}
 import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException
 import org.apache.spark.sql.hive.CarbonRelation
 
 import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
 import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.locks.{CarbonLockUtil, ICarbonLock, LockUsage}
 import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonMetadata, CarbonTableIdentifier}
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.util.CarbonUtil
 import org.apache.carbondata.core.util.path.CarbonStorePath
 import org.apache.carbondata.events.{DropTablePostEvent, DropTablePreEvent, OperationContext, OperationListenerBus}
@@ -60,9 +62,20 @@ case class CarbonDropTableCommand(
         lock => carbonLocks += CarbonLockUtil.getLockObject(carbonTableIdentifier, lock)
       }
       LOGGER.audit(s"Deleting table [$tableName] under database [$dbName]")
-
-      // fires the event before dropping main table
-      val carbonTable = CarbonMetadata.getInstance.getCarbonTable(dbName + "_" + tableName)
+      val carbonTable: Option[CarbonTable] =
+        catalog.getTableFromMetadataCache(dbName, tableName) match {
+          case Some(tableMeta) => Some(tableMeta.carbonTable)
+          case None => try {
+            Some(catalog.lookupRelation(identifier)(sparkSession)
+              .asInstanceOf[CarbonRelation].metaData.carbonTable)
+          } catch {
+            case ex: NoSuchTableException =>
+              if (!ifExistsSet) {
+                throw ex
+              }
+              None
+          }
+        }
       val operationContext = new OperationContext
       val dropTablePreEvent: DropTablePreEvent =
         DropTablePreEvent(
@@ -70,23 +83,23 @@ case class CarbonDropTableCommand(
           ifExistsSet,
           sparkSession)
       OperationListenerBus.getInstance.fireEvent(dropTablePreEvent, operationContext)
-
       CarbonEnv.getInstance(sparkSession).carbonMetastore
         .dropTable(tableIdentifier.getTablePath, identifier)(sparkSession)
 
+      // fires the event after dropping main table
       val dropTablePostEvent: DropTablePostEvent =
         DropTablePostEvent(
           carbonTable,
           ifExistsSet,
           sparkSession)
-      OperationListenerBus.getInstance.fireEvent(dropTablePreEvent, operationContext)
-
+      OperationListenerBus.getInstance.fireEvent(dropTablePostEvent, operationContext)
       LOGGER.audit(s"Deleted table [$tableName] under database [$dbName]")
     } catch {
       case ex: Exception =>
         LOGGER.error(ex, s"Dropping table $dbName.$tableName failed")
-        sys.error(s"Dropping table $dbName.$tableName failed: ${ex.getMessage}")
-    } finally {
+        sys.error(s"Dropping table $dbName.$tableName failed: ${ ex.getMessage }")
+    }
+    finally {
       if (carbonLocks.nonEmpty) {
         val unlocked = carbonLocks.forall(_.unlock())
         if (unlocked) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ee6bd2f8/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala
index ca384f9..e42e933 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala
@@ -24,7 +24,7 @@ import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.exception.InvalidConfigurationException
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
-import org.apache.carbondata.core.metadata.schema.table.{RelationIdentifier, TableInfo}
+import org.apache.carbondata.core.metadata.schema.table.TableInfo
 import org.apache.carbondata.core.util.CarbonUtil
 
 /**
@@ -132,5 +132,3 @@ case class CreatePreAggregateTableCommand(
     Seq.empty
   }
 }
-
-

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ee6bd2f8/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/DropPreAggregateTablePostListener.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/DropPreAggregateTablePostListener.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/DropPreAggregateTablePostListener.scala
new file mode 100644
index 0000000..7127c46
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/DropPreAggregateTablePostListener.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.spark.sql.execution.command.preaaggregate
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.execution.command.CarbonDropTableCommand
+
+import org.apache.carbondata.core.metadata.schema.table.DataMapSchema
+import org.apache.carbondata.events.{DropTablePostEvent, Event, OperationContext, OperationEventListener}
+
+class DropPreAggregateTablePostListener extends OperationEventListener {
+
+  /**
+   * Called on a specified event occurrence
+   *
+   * @param event
+   */
+  override def onEvent(event: Event, operationContext: OperationContext): Unit = {
+    val dropPostEvent = event.asInstanceOf[DropTablePostEvent]
+    val carbonTable = dropPostEvent.carbonTable
+    val sparkSession = dropPostEvent.sparkSession
+    if (carbonTable.isDefined && carbonTable.get.getTableInfo.getDataMapSchemaList != null &&
+        !carbonTable.get.getTableInfo.getDataMapSchemaList.isEmpty) {
+      val childSchemas = carbonTable.get.getTableInfo.getDataMapSchemaList
+      for (childSchema: DataMapSchema <- childSchemas.asScala) {
+        CarbonDropTableCommand(ifExistsSet = true,
+          Some(childSchema.getRelationIdentifier.getDatabaseName),
+          childSchema.getRelationIdentifier.getTableName).run(sparkSession)
+      }
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ee6bd2f8/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala
index c4b6783..fd0e543 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala
@@ -59,8 +59,8 @@ object PreAggregateUtil {
   /**
    * Below method will be used to validate the select plan
    * and get the required fields from select plan
-   * Currently only aggregate query is support any other type of query will
-   * fail
+   * Currently only aggregate query is support any other type of query will fail
+   *
    * @param plan
    * @param selectStmt
    * @return list of fields
@@ -89,11 +89,11 @@ object PreAggregateUtil {
               throw new MalformedCarbonCommandException(
                 "Distinct is not supported On Pre Aggregation")
             }
-            fieldToDataMapFieldMap ++= ((validateAggregateFunctionAndGetFields(carbonTable,
+            fieldToDataMapFieldMap ++= (validateAggregateFunctionAndGetFields(carbonTable,
               attr.aggregateFunction,
               parentTableName,
               parentDatabaseName,
-              parentTableId)))
+              parentTableId))
           case attr: AttributeReference =>
             fieldToDataMapFieldMap += getField(attr.name,
               attr.dataType,
@@ -124,6 +124,7 @@ object PreAggregateUtil {
    * in case of any other aggregate function it will throw error
    * In case of avg it will return two fields one for count
    * and other of sum of that column to support rollup
+   *
    * @param carbonTable
    * @param aggFunctions
    * @param parentTableName
@@ -220,6 +221,7 @@ object PreAggregateUtil {
 
   /**
    * Below method will be used to get the fields object for pre aggregate table
+   *
    * @param columnName
    * @param dataType
    * @param aggregateType
@@ -256,8 +258,7 @@ object PreAggregateUtil {
         precision = precision,
         scale = scale,
         rawSchema = rawSchema), dataMapField)
-    }
-    else {
+    } else {
       (Field(column = actualColumnName,
         dataType = Some(dataType.typeName),
         name = Some(actualColumnName),
@@ -268,7 +269,8 @@ object PreAggregateUtil {
 
   /**
    * Below method will be used to update the main table about the pre aggregate table information
-   * in case of any exption it will throw error so pre aggregate table creation will fail
+   * in case of any exception it will throw error so pre aggregate table creation will fail
+   *
    * @param dbName
    * @param tableName
    * @param childSchema
@@ -304,9 +306,8 @@ object PreAggregateUtil {
       val thriftTable = schemaConverter
         .fromWrapperToExternalTableInfo(wrapperTableInfo, dbName, tableName)
       updateSchemaInfo(carbonTable,
-        thriftTable)(sparkSession,
-        sparkSession.sessionState.asInstanceOf[CarbonSessionState])
-      LOGGER.info(s"Pre Aggeragte Parent table updated is successful for table $dbName.$tableName")
+        thriftTable)(sparkSession)
+      LOGGER.info(s"Parent table updated is successful for table $dbName.$tableName")
     } catch {
       case e: Exception =>
         LOGGER.error(e, "Pre Aggregate Parent table update failed reverting changes")
@@ -321,14 +322,13 @@ object PreAggregateUtil {
 
   /**
    * Below method will be used to update the main table schema
+   *
    * @param carbonTable
    * @param thriftTable
    * @param sparkSession
-   * @param sessionState
    */
   def updateSchemaInfo(carbonTable: CarbonTable,
-      thriftTable: TableInfo)(sparkSession: SparkSession,
-      sessionState: CarbonSessionState): Unit = {
+      thriftTable: TableInfo)(sparkSession: SparkSession): Unit = {
     val dbName = carbonTable.getDatabaseName
     val tableName = carbonTable.getFactTableName
     CarbonEnv.getInstance(sparkSession).carbonMetastore

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ee6bd2f8/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala
index 51c7f3b..ac75fa7 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala
@@ -20,12 +20,14 @@ package org.apache.spark.sql.hive
 import java.util.UUID
 import java.util.concurrent.atomic.AtomicLong
 
+import scala.collection.JavaConverters._
 import scala.collection.mutable.ArrayBuffer
 
-import org.apache.spark.sql.{CarbonDatasourceHadoopRelation, SparkSession}
+import org.apache.spark.sql.{CarbonDatasourceHadoopRelation, CarbonEnv, SparkSession}
 import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.catalyst.analysis.NoSuchTableException
 import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}
+import org.apache.spark.sql.execution.command.preaaggregate.PreAggregateUtil
 import org.apache.spark.sql.execution.datasources.LogicalRelation
 
 import org.apache.carbondata.common.logging.LogServiceFactory
@@ -34,11 +36,10 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datamap.DataMapStoreManager
 import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.fileoperations.FileWriteOperation
-import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonMetadata, CarbonTableIdentifier}
+import org.apache.carbondata.core.metadata.{schema, AbsoluteTableIdentifier, CarbonMetadata, CarbonTableIdentifier}
 import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl
-import org.apache.carbondata.core.metadata.schema
 import org.apache.carbondata.core.metadata.schema.table
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, RelationIdentifier}
 import org.apache.carbondata.core.util.CarbonUtil
 import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
 import org.apache.carbondata.core.writer.ThriftWriter
@@ -449,6 +450,41 @@ class CarbonFileMetastore extends CarbonMetaStore {
     }
   }
 
+  protected def updateParentTableInfo(parentRelationIdentifier: RelationIdentifier,
+      childCarbonTable: CarbonTable)(sparkSession: SparkSession): Unit = {
+    val dbName = parentRelationIdentifier.getDatabaseName
+    val tableName = parentRelationIdentifier.getTableName
+    val metaStore = CarbonEnv.getInstance(sparkSession).carbonMetastore
+    try {
+      val tableMeta = metaStore.getTableFromMetadataCache(dbName, tableName)
+      if (tableMeta.isDefined) {
+        val parentCarbonTable = tableMeta.get.carbonTable
+        val childSchemas = parentCarbonTable.getTableInfo.getDataMapSchemaList
+        if (childSchemas == null) {
+          throw UninitializedFieldError("Child schemas is not initialized")
+        }
+        val childSchemaIterator = childSchemas.iterator()
+        while (childSchemaIterator.hasNext) {
+          val childSchema = childSchemaIterator.next()
+          if (childSchema.getChildSchema.equals(childCarbonTable.getTableInfo.getFactTable)) {
+            childSchemaIterator.remove()
+          }
+        }
+        val schemaConverter = new ThriftWrapperSchemaConverterImpl
+        PreAggregateUtil
+          .updateSchemaInfo(parentCarbonTable,
+            schemaConverter
+              .fromWrapperToExternalTableInfo(parentCarbonTable.getTableInfo,
+                dbName,
+                tableName))(sparkSession)
+      }
+    } catch {
+      case ex: Exception =>
+        LOGGER.error(ex, s"Updating parent table $dbName.$tableName failed.")
+        throw ex
+    }
+  }
+
   def dropTable(tablePath: String, tableIdentifier: TableIdentifier)
     (sparkSession: SparkSession) {
     val dbName = tableIdentifier.database.get
@@ -461,6 +497,14 @@ class CarbonFileMetastore extends CarbonMetaStore {
       ManageDictionaryAndBTree.clearBTreeAndDictionaryLRUCache(carbonTable)
     }
     val fileType = FileFactory.getFileType(metadataFilePath)
+    if (carbonTable != null) {
+      val parentRelations = carbonTable.getTableInfo.getParentRelationIdentifiers
+      if (parentRelations != null && !parentRelations.isEmpty) {
+        for (parentRelation: RelationIdentifier <- parentRelations.asScala) {
+          updateParentTableInfo(parentRelation, carbonTable)(sparkSession)
+        }
+      }
+    }
 
     if (FileFactory.isFileExist(metadataFilePath, fileType)) {
       // while drop we should refresh the schema modified time so that if any thing has changed
@@ -468,6 +512,7 @@ class CarbonFileMetastore extends CarbonMetaStore {
       checkSchemasModifiedTimeAndReloadTables(identifier.getStorePath)
 
       removeTableFromMetadata(dbName, tableName)
+
       updateSchemasUpdatedTime(touchSchemaFileSystemTime(identifier.getStorePath))
       CarbonHiveMetadataUtil.invalidateAndDropTable(dbName, tableName, sparkSession)
       // discard cached table info in cachedDataSourceTables

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ee6bd2f8/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonHiveMetaStore.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonHiveMetaStore.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonHiveMetaStore.scala
index c64b7bb..6bd80f3 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonHiveMetaStore.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonHiveMetaStore.scala
@@ -18,14 +18,16 @@ package org.apache.spark.sql.hive
 
 import scala.collection.JavaConverters._
 
-import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.{CarbonEnv, SparkSession}
 import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.execution.command.preaaggregate.PreAggregateUtil
 
 import org.apache.carbondata.core.cache.dictionary.ManageDictionaryAndBTree
 import org.apache.carbondata.core.datamap.DataMapStoreManager
+import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonMetadata, CarbonTableIdentifier}
 import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, RelationIdentifier}
 import org.apache.carbondata.core.util.CarbonUtil
 import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
 import org.apache.carbondata.format
@@ -79,6 +81,12 @@ class CarbonHiveMetaStore extends CarbonFileMetastore {
       ManageDictionaryAndBTree.clearBTreeAndDictionaryLRUCache(carbonTable)
     }
     checkSchemasModifiedTimeAndReloadTables(identifier.getStorePath)
+    val parentRelations = carbonTable.getTableInfo.getParentRelationIdentifiers
+    if (parentRelations != null && !parentRelations.isEmpty) {
+      for (parentRelation: RelationIdentifier <- parentRelations.asScala) {
+        updateParentTableInfo(parentRelation, carbonTable)(sparkSession)
+      }
+    }
     removeTableFromMetadata(dbName, tableName)
     CarbonHiveMetadataUtil.invalidateAndDropTable(dbName, tableName, sparkSession)
     // discard cached table info in cachedDataSourceTables
@@ -107,6 +115,7 @@ class CarbonHiveMetaStore extends CarbonFileMetastore {
       carbonTable.getFactTableName)
   }
 
+
   /**
    * This method will overwrite the existing schema and update it with the given details
    *

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ee6bd2f8/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonSessionState.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonSessionState.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonSessionState.scala
index 9cad7b0..97ea7f8 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonSessionState.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonSessionState.scala
@@ -26,6 +26,7 @@ import org.apache.spark.sql.catalyst.optimizer.Optimizer
 import org.apache.spark.sql.catalyst.parser.ParserInterface
 import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, SubqueryAlias}
 import org.apache.spark.sql.execution.SparkOptimizer
+import org.apache.spark.sql.execution.command.preaaggregate.DropPreAggregateTablePostListener
 import org.apache.spark.sql.execution.datasources._
 import org.apache.spark.sql.execution.strategy.{CarbonLateDecodeStrategy, DDLStrategy, StreamingTableStrategy}
 import org.apache.spark.sql.internal.SQLConf
@@ -34,6 +35,7 @@ import org.apache.spark.sql.parser.CarbonSparkSqlParser
 
 import org.apache.carbondata.core.datamap.DataMapStoreManager
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
+import org.apache.carbondata.events.{DropTablePostEvent, Event, OperationListenerBus}
 
 /**
  * This class will have carbon catalog and refresh the relation from cache if the carbontable in
@@ -124,6 +126,15 @@ class CarbonSessionCatalog(
   }
 }
 
+object CarbonSessionState {
+
+  def init(): Unit = {
+    OperationListenerBus.getInstance()
+      .addListener(classOf[DropTablePostEvent], new DropPreAggregateTablePostListener)
+  }
+
+}
+
 /**
  * Session state implementation to override sql parser and adding strategies
  * @param sparkSession
@@ -140,6 +151,8 @@ class CarbonSessionState(sparkSession: SparkSession) extends HiveSessionState(sp
     )
   experimentalMethods.extraOptimizations = Seq(new CarbonLateDecodeRule)
 
+  CarbonSessionState.init()
+
   override lazy val optimizer: Optimizer = new CarbonOptimizer(catalog, conf, experimentalMethods)
 
   override lazy val analyzer: Analyzer = {


[07/11] carbondata git commit: [DOCS] Removed unused parameters, added SORT_SCOPE, and updated dictionary details

Posted by ra...@apache.org.
[DOCS] Removed unused parameters, added SORT_SCOPE, and updated dictionary details

This closes #1426


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

Branch: refs/heads/pre-aggregate
Commit: 520e50f32f3716b1335df37efb26222d37bc2b20
Parents: 9f6c8e6
Author: sgururajshetty <sg...@gmail.com>
Authored: Sun Oct 22 15:38:01 2017 +0530
Committer: chenliang613 <ch...@huawei.com>
Committed: Sat Nov 11 16:12:09 2017 +0800

----------------------------------------------------------------------
 docs/configuration-parameters.md    |  6 +-----
 docs/ddl-operation-on-carbondata.md | 31 ++++++++++++++++++++++++++++---
 2 files changed, 29 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/520e50f3/docs/configuration-parameters.md
----------------------------------------------------------------------
diff --git a/docs/configuration-parameters.md b/docs/configuration-parameters.md
index e085317..141a60c 100644
--- a/docs/configuration-parameters.md
+++ b/docs/configuration-parameters.md
@@ -48,12 +48,8 @@ This section provides the details of all the configurations required for CarbonD
 
 | Parameter | Default Value | Description | Range |
 |--------------------------------------|---------------|----------------------------------------------------------------------------------------------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
-| carbon.sort.file.buffer.size | 20 | File read buffer size used during sorting. This value is expressed in MB. | Min=1 and Max=100 |
-| carbon.graph.rowset.size | 100000 | Rowset size exchanged between data load graph steps. | Min=500 and Max=1000000 |
 | carbon.number.of.cores.while.loading | 6 | Number of cores to be used while loading data. |  |
 | carbon.sort.size | 500000 | Record count to sort and write intermediate files to temp. |  |
-| carbon.enableXXHash | true | Algorithm for hashmap for hashkey calculation. |  |
-| carbon.number.of.cores.block.sort | 7 | Number of cores to use for block sort while loading data. |  |
 | carbon.max.driver.lru.cache.size | -1 | Max LRU cache size upto which data will be loaded at the driver side. This value is expressed in MB. Default value of -1 means there is no memory limit for caching. Only integer values greater than 0 are accepted. |  |
 | carbon.max.executor.lru.cache.size | -1 | Max LRU cache size upto which data will be loaded at the executor side. This value is expressed in MB. Default value of -1 means there is no memory limit for caching. Only integer values greater than 0 are accepted. If this parameter is not configured, then the carbon.max.driver.lru.cache.size value will be considered. |  |
 | carbon.merge.sort.prefetch | true | Enable prefetch of data during merge sort while reading data from sort temp files in data loading. |  |
@@ -135,7 +131,7 @@ This section provides the details of all the configurations required for CarbonD
 |---------------------------------------|---------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
 | high.cardinality.identify.enable | true | If the parameter is true, the high cardinality columns of the dictionary code are automatically recognized and these columns will not be used as global dictionary encoding. If the parameter is false, all dictionary encoding columns are used as dictionary encoding. The high cardinality column must meet the following requirements: value of cardinality > configured value of high.cardinality. <b> Note: </b> If SINGLE_PASS is used during data load, then this property will be disabled.|
 | high.cardinality.threshold | 1000000  | It is a threshold to identify high cardinality of the columns.If the value of columns' cardinality > the configured value, then the columns are excluded from dictionary encoding. |
-| carbon.cutOffTimestamp | 1970-01-01 05:30:00 | Sets the start date for calculating the timestamp. Java counts the number of milliseconds from start of "1970-01-01 00:00:00". This property is used to customize the start of position. For example "2000-01-01 00:00:00". The date must be in the form "carbon.timestamp.format". NOTE: The CarbonData supports data store up to 68 years from the cut-off time defined. For example, if the cut-off time is 1970-01-01 05:30:00, then the data can be stored up to 2038-01-01 05:30:00. |
+| carbon.cutOffTimestamp | 1970-01-01 05:30:00 | Sets the start date for calculating the timestamp. Java counts the number of milliseconds from start of "1970-01-01 00:00:00". This property is used to customize the start of position. For example "2000-01-01 00:00:00". The date must be in the form "carbon.timestamp.format". |
 | carbon.timegranularity | SECOND | The property used to set the data granularity level DAY, HOUR, MINUTE, or SECOND. |
   
 ##  Spark Configuration

http://git-wip-us.apache.org/repos/asf/carbondata/blob/520e50f3/docs/ddl-operation-on-carbondata.md
----------------------------------------------------------------------
diff --git a/docs/ddl-operation-on-carbondata.md b/docs/ddl-operation-on-carbondata.md
index 55d7063..d1fee46 100644
--- a/docs/ddl-operation-on-carbondata.md
+++ b/docs/ddl-operation-on-carbondata.md
@@ -62,14 +62,14 @@ The following DDL operations are supported in CarbonData :
 
    - **Dictionary Encoding Configuration**
 
-       Dictionary encoding is enabled by default for all String columns, and disabled for non-String columns. You can include and exclude columns for dictionary encoding.
+       Dictionary encoding is turned off for all columns by default. You can include and exclude columns for dictionary encoding.
 
 ```
        TBLPROPERTIES ('DICTIONARY_EXCLUDE'='column1, column2')
        TBLPROPERTIES ('DICTIONARY_INCLUDE'='column1, column2')
 ```
 
-   Here, DICTIONARY_EXCLUDE will exclude dictionary creation. This is applicable for high-cardinality columns and is an optional parameter. DICTIONARY_INCLUDE will generate dictionary for the columns specified in the list.
+   Here, DICTIONARY_INCLUDE will improve the performance for low cardinality dimensions, considerably for string. DICTIONARY_INCLUDE will generate dictionary for the columns specified.
 
 
 
@@ -129,7 +129,7 @@ The following DDL operations are supported in CarbonData :
 
    - **SORT_COLUMNS**
 
-    This table property specifies the order of the sort column.
+      This table property specifies the order of the sort column.
 
 ```
     TBLPROPERTIES('SORT_COLUMNS'='column1, column3')
@@ -140,6 +140,31 @@ The following DDL operations are supported in CarbonData :
    - If this property is not specified, then by default SORT_COLUMNS consist of all dimension (exclude Complex Column).
 
    - If this property is specified but with empty argument, then the table will be loaded without sort. For example, ('SORT_COLUMNS'='')
+   
+   - **SORT_SCOPE**
+      This option specifies the scope of the sort during data load. Following are the types of sort scope.
+     * BATCH_SORT: it will increase the load performance but decreases the query performance if identified blocks > parallelism.
+```
+    OPTIONS ('SORT_SCOPE'='BATCH_SORT')
+```
+      You can also specify the sort size option for sort scope.
+```
+    OPTIONS ('SORT_SCOPE'='BATCH_SORT', 'batch_sort_size_inmb'='7')
+```
+     * GLOBAL_SORT: it increases the query performance, especially point query.
+```
+    OPTIONS ('SORT_SCOPE'= GLOBAL_SORT ')
+```
+	 You can also specify the number of partitions to use when shuffling data for sort. If it is not configured, or configured less than 1, then it uses the number of map tasks as reduce tasks. It is recommended that each reduce task deal with 512MB - 1GB data.
+```
+    OPTIONS( 'SORT_SCOPE'='GLOBAL_SORT', 'GLOBAL_SORT_PARTITIONS'='2')
+```
+   NOTE:
+   - Increasing number of partitions might require increasing spark.driver.maxResultSize as sampling data collected at driver increases with increasing partitions.
+   - Increasing number of partitions might increase the number of Btree.
+     * LOCAL_SORT: it is the default sort scope.
+	 * NO_SORT: it will load the data in unsorted manner.
+	 
 
 ## SHOW TABLE
 


[05/11] carbondata git commit: [CARBONDATA-1694] [BugFix] Resolved bug for Incorrect exception on presto CLI when a column is dropped from carbon

Posted by ra...@apache.org.
[CARBONDATA-1694] [BugFix] Resolved bug for Incorrect exception on presto CLI when a column is dropped from carbon

Steps to reproduce : same as https://issues.apache.org/jira/browse/CARBONDATA-1694

This closes #1486


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

Branch: refs/heads/pre-aggregate
Commit: c6ca6409486d59a79230a7ea64f1e05bb1ce3d16
Parents: ccb6560
Author: anubhav100 <an...@knoldus.in>
Authored: Fri Nov 10 19:47:25 2017 +0530
Committer: chenliang613 <ch...@huawei.com>
Committed: Sat Nov 11 15:24:01 2017 +0800

----------------------------------------------------------------------
 .../presto/impl/CarbonTableReader.java          | 79 +++++++++++++-------
 1 file changed, 52 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/c6ca6409/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonTableReader.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonTableReader.java b/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonTableReader.java
index 0fa7684..d61322d 100755
--- a/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonTableReader.java
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonTableReader.java
@@ -28,6 +28,7 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 
+import org.apache.carbondata.core.datamap.DataMapStoreManager;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
@@ -41,6 +42,8 @@ import org.apache.carbondata.core.reader.ThriftReader;
 import org.apache.carbondata.core.scan.expression.Expression;
 import org.apache.carbondata.core.service.impl.PathFactory;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
+import org.apache.carbondata.events.OperationEventListener;
+import org.apache.carbondata.events.OperationListenerBus;
 import org.apache.carbondata.hadoop.CarbonInputSplit;
 import org.apache.carbondata.hadoop.api.CarbonTableInputFormat;
 
@@ -109,10 +112,11 @@ public class CarbonTableReader {
    * @return
    */
   public CarbonTableCacheModel getCarbonCache(SchemaTableName table) {
+
     if (!cc.containsKey(table) || cc.get(table) == null) {
-      // if this table is not cached, try to read the metadata of the table and cache it.
+// if this table is not cached, try to read the metadata of the table and cache it.
       try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(
-          FileFactory.class.getClassLoader())) {
+              FileFactory.class.getClassLoader())) {
         if (carbonFileList == null) {
           fileType = FileFactory.getFileType(config.getStorePath());
           try {
@@ -125,7 +129,6 @@ public class CarbonTableReader {
       updateSchemaTables(table);
       parseCarbonMetadata(table);
     }
-
     if (cc.containsKey(table)) {
       return cc.get(table);
     } else {
@@ -133,6 +136,12 @@ public class CarbonTableReader {
     }
   }
 
+  private void removeTableFromCache(SchemaTableName table) {
+    DataMapStoreManager.getInstance().clearDataMap(cc.get(table).carbonTable.getAbsoluteTableIdentifier());
+    cc.remove(table);
+    tableList.remove(table);
+  }
+
   /**
    * Return the schema names under a schema store path (this.carbonFileList).
    *
@@ -192,11 +201,11 @@ public class CarbonTableReader {
    */
   private Set<String> updateTableList(String schemaName) {
     List<CarbonFile> schema =
-        Stream.of(carbonFileList.listFiles()).filter(a -> schemaName.equals(a.getName()))
-            .collect(Collectors.toList());
+            Stream.of(carbonFileList.listFiles()).filter(a -> schemaName.equals(a.getName()))
+                    .collect(Collectors.toList());
     if (schema.size() > 0) {
       return Stream.of((schema.get(0)).listFiles()).map(CarbonFile::getName)
-          .collect(Collectors.toSet());
+              .collect(Collectors.toSet());
     } else return ImmutableSet.of();
   }
 
@@ -225,10 +234,24 @@ public class CarbonTableReader {
    * is called, it clears this.tableList and populate the list by reading the files.
    */
   private void updateSchemaTables(SchemaTableName schemaTableName) {
-    // update logic determine later
+// update logic determine later
+    boolean isKeyExists = cc.containsKey(schemaTableName);
+
     if (carbonFileList == null) {
       updateSchemaList();
     }
+    try {
+      if(isKeyExists && !FileFactory.isFileExist(cc.get(schemaTableName).carbonTablePath.getSchemaFilePath(),fileType)){
+        removeTableFromCache(schemaTableName);
+        throw new TableNotFoundException(schemaTableName);
+      }
+    } catch (IOException e) {
+      e.printStackTrace();
+      throw new RuntimeException();
+    }
+    if(isKeyExists && FileFactory.getCarbonFile(cc.get(schemaTableName).carbonTablePath.getPath()).getLastModifiedTime() > cc.get(schemaTableName).tableInfo.getLastUpdatedTime()){
+      removeTableFromCache(schemaTableName);
+    }
     if(!tableList.contains(schemaTableName)) {
       for (CarbonFile cf : carbonFileList.listFiles()) {
         if (!cf.getName().endsWith(".mdt")) {
@@ -240,6 +263,7 @@ public class CarbonTableReader {
     }
   }
 
+
   /**
    * Find the table with the given name and build a CarbonTable instance for it.
    * This method should be called after this.updateSchemaTables().
@@ -274,11 +298,11 @@ public class CarbonTableReader {
       String storePath = config.getStorePath();
       // create table identifier. the table id is randomly generated.
       cache.carbonTableIdentifier =
-          new CarbonTableIdentifier(table.getSchemaName(), table.getTableName(),
-              UUID.randomUUID().toString());
+              new CarbonTableIdentifier(table.getSchemaName(), table.getTableName(),
+                      UUID.randomUUID().toString());
       // get the store path of the table.
       cache.carbonTablePath =
-          PathFactory.getInstance().getCarbonTablePath(storePath, cache.carbonTableIdentifier, null);
+              PathFactory.getInstance().getCarbonTablePath(storePath, cache.carbonTableIdentifier, null);
       // cache the table
       cc.put(table, cache);
 
@@ -292,27 +316,28 @@ public class CarbonTableReader {
         }
       };
       ThriftReader thriftReader =
-          new ThriftReader(cache.carbonTablePath.getSchemaFilePath(), createTBase);
+              new ThriftReader(cache.carbonTablePath.getSchemaFilePath(), createTBase);
       thriftReader.open();
       org.apache.carbondata.format.TableInfo tableInfo =
-          (org.apache.carbondata.format.TableInfo) thriftReader.read();
+              (org.apache.carbondata.format.TableInfo) thriftReader.read();
       thriftReader.close();
 
+
       // Step 3: convert format level TableInfo to code level TableInfo
       SchemaConverter schemaConverter = new ThriftWrapperSchemaConverterImpl();
       // wrapperTableInfo is the code level information of a table in carbondata core, different from the Thrift TableInfo.
       TableInfo wrapperTableInfo = schemaConverter
-          .fromExternalToWrapperTableInfo(tableInfo, table.getSchemaName(), table.getTableName(),
-              storePath);
+              .fromExternalToWrapperTableInfo(tableInfo, table.getSchemaName(), table.getTableName(),
+                      storePath);
       wrapperTableInfo.setMetaDataFilepath(
-          CarbonTablePath.getFolderContainingFile(cache.carbonTablePath.getSchemaFilePath()));
+              CarbonTablePath.getFolderContainingFile(cache.carbonTablePath.getSchemaFilePath()));
 
       // Step 4: Load metadata info into CarbonMetadata
       CarbonMetadata.getInstance().loadTableMetadata(wrapperTableInfo);
 
       cache.tableInfo = wrapperTableInfo;
       cache.carbonTable = CarbonMetadata.getInstance()
-          .getCarbonTable(cache.carbonTableIdentifier.getTableUniqueName());
+              .getCarbonTable(cache.carbonTableIdentifier.getTableUniqueName());
       result = cache.carbonTable;
     } catch (Exception ex) {
       throw new RuntimeException(ex);
@@ -323,7 +348,7 @@ public class CarbonTableReader {
 
 
   public List<CarbonLocalInputSplit> getInputSplits2(CarbonTableCacheModel tableCacheModel,
-      Expression filters)  {
+                                                     Expression filters)  {
     List<CarbonLocalInputSplit> result = new ArrayList<>();
 
     CarbonTable carbonTable = tableCacheModel.carbonTable;
@@ -331,14 +356,14 @@ public class CarbonTableReader {
     Configuration config = new Configuration();
     config.set(CarbonTableInputFormat.INPUT_SEGMENT_NUMBERS, "");
     String carbonTablePath = PathFactory.getInstance()
-        .getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier().getStorePath(),
-            carbonTable.getCarbonTableIdentifier(), null).getPath();
+            .getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier().getStorePath(),
+                    carbonTable.getCarbonTableIdentifier(), null).getPath();
     config.set(CarbonTableInputFormat.INPUT_DIR, carbonTablePath);
 
     try {
       CarbonTableInputFormat.setTableInfo(config, tableInfo);
       CarbonTableInputFormat carbonTableInputFormat =
-          createInputFormat(config, carbonTable.getAbsoluteTableIdentifier(), filters);
+              createInputFormat(config, carbonTable.getAbsoluteTableIdentifier(), filters);
       JobConf jobConf = new JobConf(config);
       Job job = Job.getInstance(jobConf);
       List<InputSplit> splits = carbonTableInputFormat.getSplits(job);
@@ -348,11 +373,11 @@ public class CarbonTableReader {
         for (InputSplit inputSplit : splits) {
           carbonInputSplit = (CarbonInputSplit) inputSplit;
           result.add(new CarbonLocalInputSplit(carbonInputSplit.getSegmentId(),
-              carbonInputSplit.getPath().toString(), carbonInputSplit.getStart(),
-              carbonInputSplit.getLength(), Arrays.asList(carbonInputSplit.getLocations()),
-              carbonInputSplit.getNumberOfBlocklets(), carbonInputSplit.getVersion().number(),
-              carbonInputSplit.getDeleteDeltaFiles(),
-              gson.toJson(carbonInputSplit.getDetailInfo())));
+                  carbonInputSplit.getPath().toString(), carbonInputSplit.getStart(),
+                  carbonInputSplit.getLength(), Arrays.asList(carbonInputSplit.getLocations()),
+                  carbonInputSplit.getNumberOfBlocklets(), carbonInputSplit.getVersion().number(),
+                  carbonInputSplit.getDeleteDeltaFiles(),
+                  gson.toJson(carbonInputSplit.getDetailInfo())));
         }
       }
 
@@ -364,10 +389,10 @@ public class CarbonTableReader {
   }
 
   private CarbonTableInputFormat<Object>  createInputFormat( Configuration conf, AbsoluteTableIdentifier identifier, Expression filterExpression)
-      throws IOException {
+          throws IOException {
     CarbonTableInputFormat format = new CarbonTableInputFormat<Object>();
     CarbonTableInputFormat.setTablePath(conf,
-        identifier.appendWithLocalPrefix(identifier.getTablePath()));
+            identifier.appendWithLocalPrefix(identifier.getTablePath()));
     CarbonTableInputFormat.setFilterPredicates(conf, filterExpression);
 
     return format;


[08/11] carbondata git commit: [CARBONDATA-1701][SEGMENT READING] Threadsafe api revealed for set segment to read

Posted by ra...@apache.org.
[CARBONDATA-1701][SEGMENT READING] Threadsafe api revealed for set segment to read

Example: CarbonSession.threadSet(carbon.input.segments.default.carbon_table_MulTI_THread, 1,2,3)

This closes #1482


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

Branch: refs/heads/pre-aggregate
Commit: 80195da41390cd122e6099483149aa4cf59300fd
Parents: 520e50f
Author: rahulforallp <ra...@knoldus.in>
Authored: Fri Nov 10 18:25:09 2017 +0530
Committer: Venkata Ramana G <ra...@huawei.com>
Committed: Sat Nov 11 15:06:01 2017 +0530

----------------------------------------------------------------------
 .../carbondata/core/util/CarbonSessionInfo.java | 38 +++++++++-
 .../carbondata/core/util/SessionParams.java     | 22 ++++++
 .../sql/CarbonDatasourceHadoopRelation.scala    |  5 +-
 .../scala/org/apache/spark/sql/CarbonEnv.scala  |  7 +-
 .../org/apache/spark/sql/CarbonSession.scala    | 31 +++++++-
 .../execution/command/CarbonHiveCommands.scala  | 37 ++++++----
 .../spark/sql/parser/CarbonSparkSqlParser.scala |  6 +-
 .../TestSegmentReadingForMultiThreading.scala   | 78 ++++++++++++++++++++
 8 files changed, 198 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/80195da4/core/src/main/java/org/apache/carbondata/core/util/CarbonSessionInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/CarbonSessionInfo.java b/core/src/main/java/org/apache/carbondata/core/util/CarbonSessionInfo.java
index fb778bc..a44bde7 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/CarbonSessionInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/CarbonSessionInfo.java
@@ -18,15 +18,19 @@
 package org.apache.carbondata.core.util;
 
 import java.io.Serializable;
+import java.util.Map;
+
+import org.apache.carbondata.core.exception.InvalidConfigurationException;
 
 /**
  * This class maintains carbon session information details
  */
-public class CarbonSessionInfo implements Serializable {
+public class CarbonSessionInfo implements Serializable, Cloneable {
 
   private static final long serialVersionUID = 7738818814501121256L;
   // contains carbon session param details
   private SessionParams sessionParams;
+  private SessionParams threadParams;
 
   public SessionParams getSessionParams() {
     return sessionParams;
@@ -36,4 +40,36 @@ public class CarbonSessionInfo implements Serializable {
     this.sessionParams = sessionParams;
   }
 
+  public SessionParams getThreadParams() {
+    return threadParams;
+  }
+
+  public void setThreadParams(SessionParams threadParams) {
+    this.threadParams = threadParams;
+  }
+
+  public CarbonSessionInfo() {
+    this.sessionParams = new SessionParams();
+    this.threadParams = new SessionParams();
+  }
+
+  public CarbonSessionInfo clone() throws CloneNotSupportedException {
+    super.clone();
+    CarbonSessionInfo newObj = new CarbonSessionInfo();
+    for (Map.Entry<String, String> entry : sessionParams.getAll().entrySet()) {
+      try {
+        newObj.getSessionParams().addProperty(entry.getKey(), entry.getValue(), false);
+      } catch (InvalidConfigurationException ex) {
+        ex.printStackTrace();
+      }
+    }
+    for (Map.Entry<String, String> entry : threadParams.getAll().entrySet()) {
+      try {
+        newObj.getThreadParams().addProperty(entry.getKey(), entry.getValue(), false);
+      } catch (InvalidConfigurationException ex) {
+        ex.printStackTrace();
+      }
+    }
+    return newObj;
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/80195da4/core/src/main/java/org/apache/carbondata/core/util/SessionParams.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/SessionParams.java b/core/src/main/java/org/apache/carbondata/core/util/SessionParams.java
index 1878416..1a91272 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/SessionParams.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/SessionParams.java
@@ -81,6 +81,28 @@ public class SessionParams implements Serializable {
     return this;
   }
 
+  /**
+   * This method will be used to add a new property
+   *
+   * @param key
+   * @return properties value
+   */
+  public SessionParams addProperty(String key, String value, Boolean doAuditing)
+      throws InvalidConfigurationException {
+    boolean isValidConf = validateKeyValue(key, value);
+    if (isValidConf) {
+      if (doAuditing) {
+        LOGGER.audit("The key " + key + " with value " + value + " added in the session param");
+      }
+      sProps.put(key, value);
+    }
+    return this;
+  }
+
+  public Map<String, String> getAll() {
+    return sProps;
+  }
+
   public SessionParams addProps(Map<String, String> addedProps) {
     this.addedProps.putAll(addedProps);
     return this;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/80195da4/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDatasourceHadoopRelation.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDatasourceHadoopRelation.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDatasourceHadoopRelation.scala
index 10336eb..3fb65be 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDatasourceHadoopRelation.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDatasourceHadoopRelation.scala
@@ -35,7 +35,6 @@ import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.scan.expression.Expression
 import org.apache.carbondata.core.scan.expression.logical.AndExpression
-import org.apache.carbondata.core.util.{CarbonSessionInfo, ThreadLocalSessionInfo}
 import org.apache.carbondata.hadoop.CarbonProjection
 import org.apache.carbondata.spark.rdd.CarbonScanRDD
 import org.apache.carbondata.streaming.StreamSinkFactory
@@ -51,9 +50,7 @@ case class CarbonDatasourceHadoopRelation(
   lazy val identifier: AbsoluteTableIdentifier = AbsoluteTableIdentifier.fromTablePath(paths.head)
   lazy val databaseName: String = carbonTable.getDatabaseName
   lazy val tableName: String = carbonTable.getFactTableName
-  lazy val carbonSessionInfo : CarbonSessionInfo =
-    CarbonEnv.getInstance(sparkSession).carbonSessionInfo
-  ThreadLocalSessionInfo.setCarbonSessionInfo(carbonSessionInfo)
+  CarbonSession.updateSessionInfoToCurrentThread(sparkSession)
 
   @transient lazy val carbonRelation: CarbonRelation =
     CarbonEnv.getInstance(sparkSession).carbonMetastore.

http://git-wip-us.apache.org/repos/asf/carbondata/blob/80195da4/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonEnv.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonEnv.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonEnv.scala
index b324b10..a37b55b 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonEnv.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonEnv.scala
@@ -53,8 +53,13 @@ class CarbonEnv {
   def init(sparkSession: SparkSession): Unit = {
     sparkSession.udf.register("getTupleId", () => "")
     if (!initialized) {
+      // update carbon session parameters , preserve thread parameters
+      val currentThreadSesssionInfo = ThreadLocalSessionInfo.getCarbonSessionInfo
       carbonSessionInfo = new CarbonSessionInfo()
-      sessionParams = new SessionParams()
+      sessionParams = carbonSessionInfo.getSessionParams
+      if (currentThreadSesssionInfo != null) {
+        carbonSessionInfo.setThreadParams(currentThreadSesssionInfo.getThreadParams)
+      }
       carbonSessionInfo.setSessionParams(sessionParams)
       ThreadLocalSessionInfo.setCarbonSessionInfo(carbonSessionInfo)
       val config = new CarbonSQLConf(sparkSession)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/80195da4/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala
index 6020eee..33e34bb 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala
@@ -18,17 +18,20 @@ package org.apache.spark.sql
 
 import java.io.File
 
+import scala.collection.JavaConverters._
+
 import org.apache.hadoop.conf.Configuration
 import org.apache.spark.{SparkConf, SparkContext}
 import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd}
 import org.apache.spark.sql.SparkSession.Builder
 import org.apache.spark.sql.execution.streaming.CarbonStreamingQueryListener
 import org.apache.spark.sql.hive.CarbonSessionState
+import org.apache.spark.sql.hive.execution.command.CarbonSetCommand
 import org.apache.spark.sql.internal.{SessionState, SharedState}
 import org.apache.spark.util.Utils
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.core.util.{CarbonProperties, CarbonSessionInfo, ThreadLocalSessionInfo}
 import org.apache.carbondata.spark.util.CommonUtil
 
 /**
@@ -189,4 +192,30 @@ object CarbonSession {
     }
   }
 
+  def threadSet(key: String, value: String): Unit = {
+    var currentThreadSessionInfo = ThreadLocalSessionInfo.getCarbonSessionInfo
+    if (currentThreadSessionInfo == null) {
+      currentThreadSessionInfo = new CarbonSessionInfo()
+    }
+    else {
+      currentThreadSessionInfo = currentThreadSessionInfo.clone()
+    }
+    val threadParams = currentThreadSessionInfo.getThreadParams
+    CarbonSetCommand.validateAndSetValue(threadParams, key, value)
+    ThreadLocalSessionInfo.setCarbonSessionInfo(currentThreadSessionInfo)
+  }
+
+  private[spark] def updateSessionInfoToCurrentThread(sparkSession: SparkSession): Unit = {
+    val carbonSessionInfo = CarbonEnv.getInstance(sparkSession).carbonSessionInfo.clone()
+    val currentThreadSessionInfoOrig = ThreadLocalSessionInfo.getCarbonSessionInfo
+    if (currentThreadSessionInfoOrig != null) {
+      val currentThreadSessionInfo = currentThreadSessionInfoOrig.clone()
+      // copy all the thread parameters to apply to session parameters
+      currentThreadSessionInfo.getThreadParams.getAll.asScala
+        .foreach(entry => carbonSessionInfo.getSessionParams.addProperty(entry._1, entry._2))
+      carbonSessionInfo.setThreadParams(currentThreadSessionInfo.getThreadParams)
+    }
+    // preserve thread parameters across call
+    ThreadLocalSessionInfo.setCarbonSessionInfo(carbonSessionInfo)
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/80195da4/integration/spark2/src/main/scala/org/apache/spark/sql/hive/execution/command/CarbonHiveCommands.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/execution/command/CarbonHiveCommands.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/execution/command/CarbonHiveCommands.scala
index 56560fd..f2c8a0a 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/execution/command/CarbonHiveCommands.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/execution/command/CarbonHiveCommands.scala
@@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.expressions.Attribute
 import org.apache.spark.sql.execution.command._
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
+import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil, SessionParams}
 import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 
 case class CarbonDropDatabaseCommand(command: DropDatabaseCommand)
@@ -59,20 +59,7 @@ case class CarbonSetCommand(command: SetCommand)
     val sessionParms = CarbonEnv.getInstance(sparkSession).carbonSessionInfo.getSessionParams
     command.kv match {
       case Some((key, Some(value))) =>
-        val isCarbonProperty: Boolean = CarbonProperties.getInstance().isCarbonProperty(key)
-        if (isCarbonProperty) {
-          sessionParms.addProperty(key, value)
-        }
-        else if (key.startsWith(CarbonCommonConstants.CARBON_INPUT_SEGMENTS)) {
-          if (key.split("\\.").length == 5) {
-            sessionParms.addProperty(key.toLowerCase(), value)
-          }
-          else {
-            throw new MalformedCarbonCommandException(
-              "property should be in \" carbon.input.segments.<database_name>" +
-              ".<table_name>=<seg_id list> \" format.")
-          }
-        }
+        CarbonSetCommand.validateAndSetValue(sessionParms, key, value)
       case _ =>
 
     }
@@ -80,6 +67,26 @@ case class CarbonSetCommand(command: SetCommand)
   }
 }
 
+object CarbonSetCommand {
+  def validateAndSetValue(sessionParams: SessionParams, key: String, value: String): Unit = {
+
+    val isCarbonProperty: Boolean = CarbonProperties.getInstance().isCarbonProperty(key)
+    if (isCarbonProperty) {
+      sessionParams.addProperty(key, value)
+    }
+    else if (key.startsWith(CarbonCommonConstants.CARBON_INPUT_SEGMENTS)) {
+      if (key.split("\\.").length == 5) {
+        sessionParams.addProperty(key.toLowerCase(), value)
+      }
+      else {
+        throw new MalformedCarbonCommandException(
+          "property should be in \" carbon.input.segments.<database_name>" +
+          ".<table_name>=<seg_id list> \" format.")
+      }
+    }
+  }
+}
+
 case class CarbonResetCommand()
   extends RunnableCommand {
   override val output = ResetCommand.output

http://git-wip-us.apache.org/repos/asf/carbondata/blob/80195da4/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala
index 0a918df..a53e71f 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala
@@ -18,7 +18,7 @@ package org.apache.spark.sql.parser
 
 import scala.collection.mutable
 
-import org.apache.spark.sql.{CarbonEnv, SparkSession}
+import org.apache.spark.sql.{CarbonSession, SparkSession}
 import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.catalyst.parser.{AbstractSqlParser, ParseException, SqlBaseParser}
 import org.apache.spark.sql.catalyst.parser.ParserUtils._
@@ -29,7 +29,6 @@ import org.apache.spark.sql.execution.command.{BucketFields, CarbonCreateTableCo
 import org.apache.spark.sql.internal.{SQLConf, VariableSubstitution}
 import org.apache.spark.sql.types.StructField
 
-import org.apache.carbondata.core.util.{CarbonSessionInfo, ThreadLocalSessionInfo}
 import org.apache.carbondata.spark.CarbonOption
 import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 import org.apache.carbondata.spark.util.CommonUtil
@@ -45,8 +44,7 @@ class CarbonSparkSqlParser(conf: SQLConf, sparkSession: SparkSession) extends Ab
   private val substitutor = new VariableSubstitution(conf)
 
   override def parsePlan(sqlText: String): LogicalPlan = {
-    val carbonSessionInfo: CarbonSessionInfo = CarbonEnv.getInstance(sparkSession).carbonSessionInfo
-    ThreadLocalSessionInfo.setCarbonSessionInfo(carbonSessionInfo)
+    CarbonSession.updateSessionInfoToCurrentThread(sparkSession)
     try {
       super.parsePlan(sqlText)
     } catch {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/80195da4/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/segmentreading/TestSegmentReadingForMultiThreading.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/segmentreading/TestSegmentReadingForMultiThreading.scala b/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/segmentreading/TestSegmentReadingForMultiThreading.scala
new file mode 100644
index 0000000..e24f8b8
--- /dev/null
+++ b/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/segmentreading/TestSegmentReadingForMultiThreading.scala
@@ -0,0 +1,78 @@
+package org.apache.carbondata.spark.testsuite.segmentreading
+
+import java.util.concurrent.TimeUnit
+
+import scala.concurrent.ExecutionContext.Implicits.global
+import scala.concurrent.duration.Duration
+import scala.concurrent.{Await, Future}
+
+import org.apache.spark.sql.{CarbonSession, Row}
+import org.apache.spark.sql.test.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+
+/**
+ * Testcase for set segment in multhread env
+ */
+class TestSegmentReadingForMultiThreading extends QueryTest with BeforeAndAfterAll {
+
+  override def beforeAll: Unit = {
+    sql("DROP TABLE IF EXISTS carbon_table_MulTI_THread")
+    sql(
+      "CREATE TABLE carbon_table_MulTI_THread (empno int, empname String, designation String, doj " +
+      "Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname " +
+      "String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance " +
+      "int,utilization int,salary int) STORED BY 'org.apache.carbondata.format'")
+    sql(
+      s"LOAD DATA LOCAL INPATH '$resourcesPath/data.csv' INTO TABLE carbon_table_MulTI_THread " +
+      "OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')")
+    sql(
+      s"LOAD DATA LOCAL INPATH '$resourcesPath/data1.csv' INTO TABLE carbon_table_MulTI_THread " +
+      "OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')")
+    sql(
+      s"LOAD DATA LOCAL INPATH '$resourcesPath/data.csv' INTO TABLE carbon_table_MulTI_THread " +
+      "OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')")
+    sql(
+      s"LOAD DATA LOCAL INPATH '$resourcesPath/data1.csv' INTO TABLE carbon_table_MulTI_THread " +
+      "OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')")
+  }
+
+  test("test multithreading for segment reading") {
+
+
+    CarbonSession.threadSet("carbon.input.segments.default.carbon_table_MulTI_THread", "1,2,3")
+    val df = sql("select count(empno) from carbon_table_MulTI_THread")
+    checkAnswer(df, Seq(Row(30)))
+
+    val four = Future {
+      CarbonSession.threadSet("carbon.input.segments.default.carbon_table_MulTI_THread", "1,3")
+      val df = sql("select count(empno) from carbon_table_MulTI_THread")
+      checkAnswer(df, Seq(Row(20)))
+    }
+
+    val three = Future {
+      CarbonSession.threadSet("carbon.input.segments.default.carbon_table_MulTI_THread", "0,1,2")
+      val df = sql("select count(empno) from carbon_table_MulTI_THread")
+      checkAnswer(df, Seq(Row(30)))
+    }
+
+
+    val one = Future {
+      CarbonSession.threadSet("carbon.input.segments.default.carbon_table_MulTI_THread", "0,2")
+      val df = sql("select count(empno) from carbon_table_MulTI_THread")
+      checkAnswer(df, Seq(Row(20)))
+    }
+
+    val two = Future {
+      CarbonSession.threadSet("carbon.input.segments.default.carbon_table_MulTI_THread", "1")
+      val df = sql("select count(empno) from carbon_table_MulTI_THread")
+      checkAnswer(df, Seq(Row(10)))
+    }
+    Await.result(Future.sequence(Seq(one, two, three, four)), Duration(300, TimeUnit.SECONDS))
+  }
+
+  override def afterAll: Unit = {
+    sql("DROP TABLE IF EXISTS carbon_table_MulTI_THread")
+
+  }
+}


[09/11] carbondata git commit: [CARBONDATA-1609] Thrift Changes to support Pre-aggregate

Posted by ra...@apache.org.
[CARBONDATA-1609] Thrift Changes to support Pre-aggregate

This closes #1425


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

Branch: refs/heads/pre-aggregate
Commit: 037f630a5ab331a9f45a6413e9052d4672c8d675
Parents: 80195da
Author: kumarvishal <ku...@gmail.com>
Authored: Sat Oct 21 16:31:15 2017 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Sat Nov 11 17:29:56 2017 +0530

----------------------------------------------------------------------
 .../metadata/converter/SchemaConverter.java     |  10 ++
 .../ThriftWrapperSchemaConverterImpl.java       | 147 +++++++++++++++++--
 .../core/metadata/schema/table/CarbonTable.java |  24 +--
 .../metadata/schema/table/DataMapSchema.java    | 121 +++++++++++++++
 .../schema/table/RelationIdentifier.java        |  88 +++++++++++
 .../core/metadata/schema/table/TableInfo.java   |  83 ++++++++++-
 .../core/metadata/schema/table/TableSchema.java |  26 ++++
 .../schema/table/column/ColumnSchema.java       |  48 +++++-
 .../util/AbstractDataFileFooterConverter.java   |  28 ++++
 format/src/main/thrift/schema.thrift            |  31 ++++
 10 files changed, 580 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/037f630a/core/src/main/java/org/apache/carbondata/core/metadata/converter/SchemaConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/converter/SchemaConverter.java b/core/src/main/java/org/apache/carbondata/core/metadata/converter/SchemaConverter.java
index 4faa017..bfbb6f7 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/converter/SchemaConverter.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/converter/SchemaConverter.java
@@ -18,6 +18,7 @@ package org.apache.carbondata.core.metadata.converter;
 
 import org.apache.carbondata.core.metadata.schema.SchemaEvolution;
 import org.apache.carbondata.core.metadata.schema.SchemaEvolutionEntry;
+import org.apache.carbondata.core.metadata.schema.table.DataMapSchema;
 import org.apache.carbondata.core.metadata.schema.table.TableInfo;
 import org.apache.carbondata.core.metadata.schema.table.TableSchema;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
@@ -100,4 +101,13 @@ public interface SchemaConverter {
    */
   TableInfo fromExternalToWrapperTableInfo(org.apache.carbondata.format.TableInfo externalTableInfo,
       String dbName, String tableName, String storePath);
+
+  /**
+   * method to convert thrift datamap schema object to wrapper
+   * data map object
+   * @param thriftchildSchema
+   * @return DataMapSchema
+   */
+  DataMapSchema fromExternalToWrapperDataMapSchema(
+      org.apache.carbondata.format.DataMapSchema thriftchildSchema);
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/037f630a/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java b/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java
index adcac7d..5ffc612 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java
@@ -30,9 +30,12 @@ import org.apache.carbondata.core.metadata.schema.PartitionInfo;
 import org.apache.carbondata.core.metadata.schema.SchemaEvolution;
 import org.apache.carbondata.core.metadata.schema.SchemaEvolutionEntry;
 import org.apache.carbondata.core.metadata.schema.partition.PartitionType;
+import org.apache.carbondata.core.metadata.schema.table.DataMapSchema;
+import org.apache.carbondata.core.metadata.schema.table.RelationIdentifier;
 import org.apache.carbondata.core.metadata.schema.table.TableInfo;
 import org.apache.carbondata.core.metadata.schema.table.TableSchema;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
+import org.apache.carbondata.core.metadata.schema.table.column.ParentColumnTableRelation;
 
 /**
  * Thrift schema to carbon schema converter and vice versa
@@ -40,11 +43,11 @@ import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
 public class ThriftWrapperSchemaConverterImpl implements SchemaConverter {
 
   /* (non-Javadoc)
-   * Converts  from wrapper to thrift schema evolution entry
-   */
+    * Converts  from wrapper to thrift schema evolution entry
+    */
   @Override
   public org.apache.carbondata.format.SchemaEvolutionEntry
-       fromWrapperToExternalSchemaEvolutionEntry(SchemaEvolutionEntry wrapperSchemaEvolutionEntry) {
+      fromWrapperToExternalSchemaEvolutionEntry(SchemaEvolutionEntry wrapperSchemaEvolutionEntry) {
     org.apache.carbondata.format.SchemaEvolutionEntry thriftSchemaEvolutionEntry =
         new org.apache.carbondata.format.SchemaEvolutionEntry(
             wrapperSchemaEvolutionEntry.getTimeStamp());
@@ -159,7 +162,6 @@ public class ThriftWrapperSchemaConverterImpl implements SchemaConverter {
    */
   @Override public org.apache.carbondata.format.ColumnSchema fromWrapperToExternalColumnSchema(
       ColumnSchema wrapperColumnSchema) {
-
     List<org.apache.carbondata.format.Encoding> encoders =
         new ArrayList<org.apache.carbondata.format.Encoding>();
     for (Encoding encoder : wrapperColumnSchema.getEncodingList()) {
@@ -188,13 +190,18 @@ public class ThriftWrapperSchemaConverterImpl implements SchemaConverter {
     thriftColumnSchema.setInvisible(wrapperColumnSchema.isInvisible());
     thriftColumnSchema.setColumnReferenceId(wrapperColumnSchema.getColumnReferenceId());
     thriftColumnSchema.setSchemaOrdinal(wrapperColumnSchema.getSchemaOrdinal());
-
     if (wrapperColumnSchema.isSortColumn()) {
       Map<String, String> properties = new HashMap<String, String>();
       properties.put(CarbonCommonConstants.SORT_COLUMNS, "true");
       thriftColumnSchema.setColumnProperties(properties);
     }
-
+    thriftColumnSchema.setAggregate_function(wrapperColumnSchema.getAggFunction());
+    List<ParentColumnTableRelation> parentColumnTableRelations =
+        wrapperColumnSchema.getParentColumnTableRelations();
+    if (null != parentColumnTableRelations) {
+      thriftColumnSchema.setParentColumnTableRelations(
+          wrapperToThriftRelationList(parentColumnTableRelations));
+    }
     return thriftColumnSchema;
   }
 
@@ -280,11 +287,79 @@ public class ThriftWrapperSchemaConverterImpl implements SchemaConverter {
    */
   @Override public org.apache.carbondata.format.TableInfo fromWrapperToExternalTableInfo(
       TableInfo wrapperTableInfo, String dbName, String tableName) {
-
     org.apache.carbondata.format.TableSchema thriftFactTable =
         fromWrapperToExternalTableSchema(wrapperTableInfo.getFactTable());
-    return new org.apache.carbondata.format.TableInfo(thriftFactTable, new ArrayList<org.apache
-        .carbondata.format.TableSchema>());
+    org.apache.carbondata.format.TableInfo tableInfo =
+        new org.apache.carbondata.format.TableInfo(thriftFactTable,
+            new ArrayList<org.apache.carbondata.format.TableSchema>());
+    List<DataMapSchema> wrapperChildSchemaList = wrapperTableInfo.getDataMapSchemaList();
+    if (null != wrapperChildSchemaList) {
+      List<org.apache.carbondata.format.DataMapSchema> thriftChildSchemas =
+          fromWrapperToExternalChildSchemaList(wrapperChildSchemaList);
+      tableInfo.setDataMapSchemas(thriftChildSchemas);
+    }
+    return tableInfo;
+  }
+
+  private List<org.apache.carbondata.format.RelationIdentifier> fromWrapperToExternalRI(
+      List<RelationIdentifier> relationIdentifiersList) {
+    List<org.apache.carbondata.format.RelationIdentifier> thriftRelationIdentifierList =
+        new ArrayList<>();
+    for (RelationIdentifier relationIdentifier : relationIdentifiersList) {
+      org.apache.carbondata.format.RelationIdentifier thriftRelationIdentifier =
+          new org.apache.carbondata.format.RelationIdentifier();
+      thriftRelationIdentifier.setDatabaseName(relationIdentifier.getDatabaseName());
+      thriftRelationIdentifier.setTableName(relationIdentifier.getTableName());
+      thriftRelationIdentifier.setTableId(relationIdentifier.getTableId());
+      thriftRelationIdentifierList.add(thriftRelationIdentifier);
+    }
+    return thriftRelationIdentifierList;
+  }
+
+  private List<org.apache.carbondata.format.DataMapSchema> fromWrapperToExternalChildSchemaList(
+      List<DataMapSchema> wrapperChildSchemaList) {
+    List<org.apache.carbondata.format.DataMapSchema> thriftChildSchemas = new ArrayList<>();
+    for (DataMapSchema wrapperChildSchema : wrapperChildSchemaList) {
+      org.apache.carbondata.format.DataMapSchema thriftChildSchema =
+          new org.apache.carbondata.format.DataMapSchema();
+      org.apache.carbondata.format.RelationIdentifier relationIdentifier =
+          new org.apache.carbondata.format.RelationIdentifier();
+      relationIdentifier
+          .setDatabaseName(wrapperChildSchema.getRelationIdentifier().getDatabaseName());
+      relationIdentifier.setTableName(wrapperChildSchema.getRelationIdentifier().getTableName());
+      relationIdentifier.setTableId(wrapperChildSchema.getRelationIdentifier().getTableId());
+      thriftChildSchema.setRelationIdentifire(relationIdentifier);
+      thriftChildSchema.setProperties(wrapperChildSchema.getProperties());
+      thriftChildSchema.setClassName(wrapperChildSchema.getClassName());
+      thriftChildSchema.setChildTableSchema(
+          fromWrapperToExternalTableSchema(wrapperChildSchema.getChildSchema()));
+      thriftChildSchemas.add(thriftChildSchema);
+    }
+    return thriftChildSchemas;
+  }
+
+  private List<org.apache.carbondata.format.ParentColumnTableRelation> wrapperToThriftRelationList(
+      List<ParentColumnTableRelation> wrapperColumnRelations) {
+    List<org.apache.carbondata.format.ParentColumnTableRelation> thriftColumnRelationList =
+        new ArrayList<>();
+
+    for (ParentColumnTableRelation wrapperColumnRealtion : wrapperColumnRelations) {
+      org.apache.carbondata.format.ParentColumnTableRelation thriftColumnTableRelation =
+          new org.apache.carbondata.format.ParentColumnTableRelation();
+      thriftColumnTableRelation.setColumnId(wrapperColumnRealtion.getColumnId());
+      thriftColumnTableRelation.setColumnName(wrapperColumnRealtion.getColumnName());
+      org.apache.carbondata.format.RelationIdentifier thriftRelationIdentifier =
+          new org.apache.carbondata.format.RelationIdentifier();
+      thriftRelationIdentifier
+          .setDatabaseName(wrapperColumnRealtion.getRelationIdentifier().getDatabaseName());
+      thriftRelationIdentifier
+          .setTableName(wrapperColumnRealtion.getRelationIdentifier().getTableName());
+      thriftRelationIdentifier
+          .setTableId(wrapperColumnRealtion.getRelationIdentifier().getTableId());
+      thriftColumnTableRelation.setRelationIdentifier(thriftRelationIdentifier);
+      thriftColumnRelationList.add(thriftColumnTableRelation);
+    }
+    return thriftColumnRelationList;
   }
 
   /* (non-Javadoc)
@@ -436,9 +511,17 @@ public class ThriftWrapperSchemaConverterImpl implements SchemaConverter {
         wrapperColumnSchema.setSortColumn(true);
       }
     }
+    wrapperColumnSchema.setAggFunction(externalColumnSchema.getAggregate_function());
+    List<org.apache.carbondata.format.ParentColumnTableRelation> parentColumnTableRelation =
+        externalColumnSchema.getParentColumnTableRelations();
+    if (null != parentColumnTableRelation) {
+      wrapperColumnSchema.setParentColumnTableRelations(
+          fromExtrenalToWrapperParentTableColumnRelations(parentColumnTableRelation));
+    }
     return wrapperColumnSchema;
   }
 
+
   private PartitionType fromExternalToWrapperPartitionType(
       org.apache.carbondata.format.PartitionType externalPartitionType) {
     if (null == externalPartitionType) {
@@ -508,7 +591,7 @@ public class ThriftWrapperSchemaConverterImpl implements SchemaConverter {
       org.apache.carbondata.format.BucketingInfo externalBucketInfo) {
     List<ColumnSchema> listOfColumns = new ArrayList<ColumnSchema>();
     for (org.apache.carbondata.format.ColumnSchema externalColumnSchema :
-          externalBucketInfo.table_columns) {
+        externalBucketInfo.table_columns) {
       listOfColumns.add(fromExternalToWrapperColumnSchema(externalColumnSchema));
     }
     return new BucketingInfo(listOfColumns, externalBucketInfo.number_of_buckets);
@@ -531,7 +614,51 @@ public class ThriftWrapperSchemaConverterImpl implements SchemaConverter {
     wrapperTableInfo.setStorePath(storePath);
     wrapperTableInfo.setFactTable(
         fromExternalToWrapperTableSchema(externalTableInfo.getFact_table(), tableName));
+    if (null != externalTableInfo.getDataMapSchemas()) {
+      wrapperTableInfo.setDataMapSchemaList(
+          fromExternalToWrapperChildSchemaList(externalTableInfo.getDataMapSchemas()));
+    }
     return wrapperTableInfo;
   }
 
+  @Override public DataMapSchema fromExternalToWrapperDataMapSchema(
+      org.apache.carbondata.format.DataMapSchema thriftDataMapSchema) {
+    RelationIdentifier relationIdentifier =
+        new RelationIdentifier(thriftDataMapSchema.getRelationIdentifire().getDatabaseName(),
+            thriftDataMapSchema.getRelationIdentifire().getTableName(),
+            thriftDataMapSchema.getRelationIdentifire().getTableId());
+    DataMapSchema childSchema = new DataMapSchema(thriftDataMapSchema.getClassName());
+    childSchema.setProperties(thriftDataMapSchema.getProperties());
+    childSchema.setChildSchema(
+        fromExternalToWrapperTableSchema(thriftDataMapSchema.getChildTableSchema(),
+            relationIdentifier.getTableName()));
+    childSchema.setRelationIdentifier(relationIdentifier);
+    return childSchema;
+  }
+
+  private List<ParentColumnTableRelation> fromExtrenalToWrapperParentTableColumnRelations(
+      List<org.apache.carbondata.format.ParentColumnTableRelation> thirftParentColumnRelation) {
+    List<ParentColumnTableRelation> parentColumnTableRelationList = new ArrayList<>();
+    for (org.apache.carbondata.format.ParentColumnTableRelation carbonTableRelation :
+        thirftParentColumnRelation) {
+      RelationIdentifier relationIdentifier =
+          new RelationIdentifier(carbonTableRelation.getRelationIdentifier().getDatabaseName(),
+              carbonTableRelation.getRelationIdentifier().getTableName(),
+              carbonTableRelation.getRelationIdentifier().getTableId());
+      ParentColumnTableRelation parentColumnTableRelation =
+          new ParentColumnTableRelation(relationIdentifier, carbonTableRelation.getColumnId(),
+              carbonTableRelation.getColumnName());
+      parentColumnTableRelationList.add(parentColumnTableRelation);
+    }
+    return parentColumnTableRelationList;
+  }
+
+  public List<DataMapSchema> fromExternalToWrapperChildSchemaList(
+      List<org.apache.carbondata.format.DataMapSchema> childSchemaList) {
+    List<DataMapSchema> childSchemas = new ArrayList<>();
+    for (org.apache.carbondata.format.DataMapSchema childSchemaThrift : childSchemaList) {
+      childSchemas.add(fromExternalToWrapperDataMapSchema(childSchemaThrift));
+    }
+    return childSchemas;
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/037f630a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
index d4aaa29..e63f4e3 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
@@ -214,7 +214,7 @@ public class CarbonTable implements Serializable {
         if (columnSchema.getNumberOfChild() > 0) {
           CarbonDimension complexDimension =
               new CarbonDimension(columnSchema, dimensionOrdinal++,
-                    columnSchema.getSchemaOrdinal(), -1, -1, ++complexTypeOrdinal);
+                  columnSchema.getSchemaOrdinal(), -1, -1, ++complexTypeOrdinal);
           complexDimension.initializeChildDimensionsList(columnSchema.getNumberOfChild());
           allDimensions.add(complexDimension);
           dimensionOrdinal =
@@ -228,8 +228,8 @@ public class CarbonTable implements Serializable {
           }
           if (!columnSchema.getEncodingList().contains(Encoding.DICTIONARY)) {
             CarbonDimension dimension =
-                    new CarbonDimension(columnSchema, dimensionOrdinal++,
-                            columnSchema.getSchemaOrdinal(), -1, -1, -1);
+                new CarbonDimension(columnSchema, dimensionOrdinal++,
+                    columnSchema.getSchemaOrdinal(), -1, -1, -1);
             if (!columnSchema.isInvisible() && columnSchema.isSortColumn()) {
               this.numberOfNoDictSortColumns++;
             }
@@ -238,8 +238,8 @@ public class CarbonTable implements Serializable {
           } else if (columnSchema.getEncodingList().contains(Encoding.DICTIONARY)
               && columnSchema.getColumnGroupId() == -1) {
             CarbonDimension dimension =
-                    new CarbonDimension(columnSchema, dimensionOrdinal++,
-                            columnSchema.getSchemaOrdinal(), keyOrdinal++, -1, -1);
+                new CarbonDimension(columnSchema, dimensionOrdinal++,
+                    columnSchema.getSchemaOrdinal(), keyOrdinal++, -1, -1);
             allDimensions.add(dimension);
             primitiveDimensions.add(dimension);
           } else {
@@ -247,15 +247,15 @@ public class CarbonTable implements Serializable {
                 previousColumnGroupId == columnSchema.getColumnGroupId() ? ++columnGroupOrdinal : 0;
             previousColumnGroupId = columnSchema.getColumnGroupId();
             CarbonDimension dimension = new CarbonDimension(columnSchema, dimensionOrdinal++,
-                    columnSchema.getSchemaOrdinal(), keyOrdinal++,
-                    columnGroupOrdinal, -1);
+                columnSchema.getSchemaOrdinal(), keyOrdinal++,
+                columnGroupOrdinal, -1);
             allDimensions.add(dimension);
             primitiveDimensions.add(dimension);
           }
         }
       } else {
         allMeasures.add(new CarbonMeasure(columnSchema, measureOrdinal++,
-                 columnSchema.getSchemaOrdinal()));
+            columnSchema.getSchemaOrdinal()));
       }
     }
     fillVisibleDimensions(tableSchema.getTableName());
@@ -290,14 +290,14 @@ public class CarbonTable implements Serializable {
    */
   private int readAllComplexTypeChildrens(int dimensionOrdinal, int childCount,
       List<ColumnSchema> listOfColumns, CarbonDimension parentDimension,
-                                          List<CarbonDimension> primitiveDimensions) {
+      List<CarbonDimension> primitiveDimensions) {
     for (int i = 0; i < childCount; i++) {
       ColumnSchema columnSchema = listOfColumns.get(dimensionOrdinal);
       if (columnSchema.isDimensionColumn()) {
         if (columnSchema.getNumberOfChild() > 0) {
           CarbonDimension complexDimension =
               new CarbonDimension(columnSchema, dimensionOrdinal++,
-                        columnSchema.getSchemaOrdinal(), -1, -1, -1);
+                  columnSchema.getSchemaOrdinal(), -1, -1, -1);
           complexDimension.initializeChildDimensionsList(columnSchema.getNumberOfChild());
           parentDimension.getListOfChildDimensions().add(complexDimension);
           dimensionOrdinal =
@@ -305,8 +305,8 @@ public class CarbonTable implements Serializable {
                   listOfColumns, complexDimension, primitiveDimensions);
         } else {
           CarbonDimension carbonDimension =
-                  new CarbonDimension(columnSchema, dimensionOrdinal++,
-                          columnSchema.getSchemaOrdinal(), -1, -1, -1);
+              new CarbonDimension(columnSchema, dimensionOrdinal++,
+                  columnSchema.getSchemaOrdinal(), -1, -1, -1);
           parentDimension.getListOfChildDimensions().add(carbonDimension);
           primitiveDimensions.add(carbonDimension);
         }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/037f630a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchema.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchema.java
new file mode 100644
index 0000000..5f92ec8
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchema.java
@@ -0,0 +1,121 @@
+/*
+ * 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.core.metadata.schema.table;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Child schema class to maintain the child table details inside parent table
+ */
+public class DataMapSchema implements Serializable, Writable {
+
+  private String className;
+
+  private RelationIdentifier relationIdentifier;
+  /**
+   * child table schema
+   */
+  private TableSchema childSchema;
+
+  /**
+   * relation properties
+   */
+  private Map<String, String> properties;
+
+  public DataMapSchema(String className) {
+    this.className = className;
+  }
+
+  public String getClassName() {
+    return className;
+  }
+
+  public TableSchema getChildSchema() {
+    return childSchema;
+  }
+
+  public RelationIdentifier getRelationIdentifier() {
+    return relationIdentifier;
+  }
+
+  public Map<String, String> getProperties() {
+    return properties;
+  }
+
+  public void setRelationIdentifier(RelationIdentifier relationIdentifier) {
+    this.relationIdentifier = relationIdentifier;
+  }
+
+  public void setChildSchema(TableSchema childSchema) {
+    this.childSchema = childSchema;
+  }
+
+  public void setProperties(Map<String, String> properties) {
+    this.properties = properties;
+  }
+
+  @Override public void write(DataOutput out) throws IOException {
+    out.writeUTF(className);
+    boolean isRelationIdentifierExists = null != relationIdentifier;
+    out.writeBoolean(isRelationIdentifierExists);
+    if (isRelationIdentifierExists) {
+      this.relationIdentifier.write(out);
+    }
+    boolean isChildSchemaExists = null != this.childSchema;
+    out.writeBoolean(isChildSchemaExists);
+    if (isChildSchemaExists) {
+      this.childSchema.write(out);
+    }
+    if (properties == null) {
+      out.writeShort(0);
+    } else {
+      out.writeShort(properties.size());
+      for (Map.Entry<String, String> entry : properties.entrySet()) {
+        out.writeUTF(entry.getKey());
+        out.writeUTF(entry.getValue());
+      }
+    }
+  }
+
+  @Override public void readFields(DataInput in) throws IOException {
+    this.className = in.readUTF();
+    boolean isRelationIdnentifierExists = in.readBoolean();
+    if (isRelationIdnentifierExists) {
+      this.relationIdentifier = new RelationIdentifier(null, null, null);
+      this.relationIdentifier.readFields(in);
+    }
+    boolean isChildSchemaExists = in.readBoolean();
+    if (isChildSchemaExists) {
+      this.childSchema = new TableSchema();
+      this.childSchema.readFields(in);
+    }
+
+    int mapSize = in.readShort();
+    this.properties = new HashMap<>(mapSize);
+    for (int i = 0; i < mapSize; i++) {
+      String key = in.readUTF();
+      String value = in.readUTF();
+      this.properties.put(key, value);
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/037f630a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/RelationIdentifier.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/RelationIdentifier.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/RelationIdentifier.java
new file mode 100644
index 0000000..9a70b8b
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/RelationIdentifier.java
@@ -0,0 +1,88 @@
+/*
+ * 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.core.metadata.schema.table;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.Serializable;
+
+/**
+ * class to maintain the relation between parent and child
+ */
+public class RelationIdentifier implements Serializable, Writable {
+
+  private String databaseName;
+
+  private String tableName;
+
+  private String tableId;
+
+  public RelationIdentifier(String databaseName, String tableName, String tableId) {
+    this.databaseName = databaseName;
+    this.tableName = tableName;
+    this.tableId = tableId;
+  }
+
+  public String getDatabaseName() {
+    return databaseName;
+  }
+
+  public String getTableName() {
+    return tableName;
+  }
+
+  public String getTableId() {
+    return tableId;
+  }
+
+  @Override public void write(DataOutput out) throws IOException {
+    out.writeUTF(databaseName);
+    out.writeUTF(tableName);
+    out.writeUTF(tableId);
+  }
+
+  @Override public void readFields(DataInput in) throws IOException {
+    this.databaseName = in.readUTF();
+    this.tableName = in.readUTF();
+    this.tableId = in.readUTF();
+  }
+
+  @Override public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+
+    RelationIdentifier that = (RelationIdentifier) o;
+
+    if (databaseName != null ?
+        !databaseName.equals(that.databaseName) :
+        that.databaseName != null) {
+      return false;
+    }
+    if (tableName != null ? !tableName.equals(that.tableName) : that.tableName != null) {
+      return false;
+    }
+    return tableId != null ? tableId.equals(that.tableId) : that.tableId == null;
+  }
+
+  @Override public int hashCode() {
+    int result = databaseName != null ? databaseName.hashCode() : 0;
+    result = 31 * result + (tableName != null ? tableName.hashCode() : 0);
+    result = 31 * result + (tableId != null ? tableId.hashCode() : 0);
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/037f630a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java
index d1a7e5b..3acd6d6 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java
@@ -24,13 +24,15 @@ import java.io.DataOutput;
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.io.Serializable;
-import java.util.Map;
+import java.util.*;
 
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
+import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
+import org.apache.carbondata.core.metadata.schema.table.column.ParentColumnTableRelation;
 
 /**
  * Store the information about the table.
@@ -79,6 +81,14 @@ public class TableInfo implements Serializable, Writable {
   // this idenifier is a lazy field which will be created when it is used first time
   private AbsoluteTableIdentifier identifier;
 
+  private List<DataMapSchema> dataMapSchemaList;
+
+  private List<RelationIdentifier> parentRelationIdentifiers;
+
+  public TableInfo() {
+    dataMapSchemaList = new ArrayList<>();
+  }
+
   /**
    * @return the factTable
    */
@@ -91,6 +101,23 @@ public class TableInfo implements Serializable, Writable {
    */
   public void setFactTable(TableSchema factTable) {
     this.factTable = factTable;
+    updateParentRelationIdentifier();
+  }
+
+  private void updateParentRelationIdentifier() {
+    Set<RelationIdentifier> parentRelationIdentifiers = new HashSet<>();
+    this.parentRelationIdentifiers = new ArrayList<>();
+    List<ColumnSchema> listOfColumns = this.factTable.getListOfColumns();
+    for (ColumnSchema columnSchema : listOfColumns) {
+      List<ParentColumnTableRelation> parentColumnTableRelations =
+          columnSchema.getParentColumnTableRelations();
+      if (null != parentColumnTableRelations) {
+        for (int i = 0; i < parentColumnTableRelations.size(); i++) {
+          parentRelationIdentifiers.add(parentColumnTableRelations.get(i).getRelationIdentifier());
+        }
+      }
+    }
+    this.parentRelationIdentifiers.addAll(parentRelationIdentifiers);
   }
 
   /**
@@ -157,6 +184,14 @@ public class TableInfo implements Serializable, Writable {
     this.storePath = storePath;
   }
 
+  public List<DataMapSchema> getDataMapSchemaList() {
+    return dataMapSchemaList;
+  }
+
+  public void setDataMapSchemaList(List<DataMapSchema> dataMapSchemaList) {
+    this.dataMapSchemaList = dataMapSchemaList;
+  }
+
   /**
    * to generate the hash code
    */
@@ -225,10 +260,27 @@ public class TableInfo implements Serializable, Writable {
     out.writeLong(lastUpdatedTime);
     out.writeUTF(metaDataFilepath);
     out.writeUTF(storePath);
+    boolean isChildSchemaExists =
+        null != dataMapSchemaList && dataMapSchemaList.size() > 0 ? true : false;
+    out.writeBoolean(isChildSchemaExists);
+    if (isChildSchemaExists) {
+      out.writeShort(dataMapSchemaList.size());
+      for (int i = 0; i < dataMapSchemaList.size(); i++) {
+        dataMapSchemaList.get(i).write(out);
+      }
+    }
+    boolean isParentTableRelationIndentifierExists =
+        null != parentRelationIdentifiers && parentRelationIdentifiers.size() > 0;
+    out.writeBoolean(isParentTableRelationIndentifierExists);
+    if (isParentTableRelationIndentifierExists) {
+      out.writeShort(parentRelationIdentifiers.size());
+      for (int i = 0; i < parentRelationIdentifiers.size(); i++) {
+        parentRelationIdentifiers.get(i).write(out);
+      }
+    }
   }
 
-  @Override
-  public void readFields(DataInput in) throws IOException {
+  @Override public void readFields(DataInput in) throws IOException {
     this.databaseName = in.readUTF();
     this.tableUniqueName = in.readUTF();
     this.factTable = new TableSchema();
@@ -236,6 +288,26 @@ public class TableInfo implements Serializable, Writable {
     this.lastUpdatedTime = in.readLong();
     this.metaDataFilepath = in.readUTF();
     this.storePath = in.readUTF();
+    boolean isChildSchemaExists = in.readBoolean();
+    this.dataMapSchemaList = new ArrayList<>();
+    if (isChildSchemaExists) {
+      short numberOfChildTable = in.readShort();
+      for (int i = 0; i < numberOfChildTable; i++) {
+        DataMapSchema childSchema = new DataMapSchema(null);
+        childSchema.readFields(in);
+        dataMapSchemaList.add(childSchema);
+      }
+    }
+    boolean isParentTableRelationIndentifierExists = in.readBoolean();
+    if (isParentTableRelationIndentifierExists) {
+      short parentTableIndentifiersListSize = in.readShort();
+      this.parentRelationIdentifiers = new ArrayList<>();
+      for (int i = 0; i < parentTableIndentifiersListSize; i++) {
+        RelationIdentifier relationIdentifier = new RelationIdentifier(null, null, null);
+        relationIdentifier.readFields(in);
+        this.parentRelationIdentifiers.add(relationIdentifier);
+      }
+    }
   }
 
   public AbsoluteTableIdentifier getOrCreateAbsoluteTableIdentifier() {
@@ -259,4 +331,9 @@ public class TableInfo implements Serializable, Writable {
     tableInfo.readFields(in);
     return tableInfo;
   }
+
+  public List<RelationIdentifier> getParentRelationIdentifiers() {
+    return parentRelationIdentifiers;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/037f630a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableSchema.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableSchema.java
index 7dc41a4..5b013d1 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableSchema.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableSchema.java
@@ -21,6 +21,7 @@ import java.io.DataOutput;
 import java.io.IOException;
 import java.io.Serializable;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
@@ -250,4 +251,29 @@ public class TableSchema implements Serializable, Writable {
     }
   }
 
+  /**
+   * Below method will be used to build child schema object which will be stored in
+   * parent table
+   *
+   * @param className
+   * @param databaseName
+   * @param queryString
+   * @param queryType
+   *
+   * @return datamap schema
+   */
+  public DataMapSchema buildChildSchema(String className, String databaseName, String queryString,
+      String queryType) {
+    RelationIdentifier relationIdentifier =
+        new RelationIdentifier(databaseName, tableName, tableId);
+    Map<String, String> properties = new HashMap<>();
+    properties.put("CHILD_SELECT QUERY", queryString);
+    properties.put("QUERYTYPE", queryType);
+    DataMapSchema dataMapSchema = new DataMapSchema(className);
+    dataMapSchema.setChildSchema(this);
+    dataMapSchema.setProperties(properties);
+    dataMapSchema.setRelationIdentifier(relationIdentifier);
+    return dataMapSchema;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/037f630a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/ColumnSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/ColumnSchema.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/ColumnSchema.java
index cad3dd6..ea7005f 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/ColumnSchema.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/ColumnSchema.java
@@ -122,6 +122,13 @@ public class ColumnSchema implements Serializable, Writable {
   private boolean isSortColumn = false;
 
   /**
+   * aggregate function used in pre aggregate table
+   */
+  private String aggFunction = "";
+
+  private List<ParentColumnTableRelation> parentColumnTableRelations;
+
+  /**
    * @return the columnName
    */
   public String getColumnName() {
@@ -274,6 +281,15 @@ public class ColumnSchema implements Serializable, Writable {
     this.defaultValue = defaultValue;
   }
 
+  public List<ParentColumnTableRelation> getParentColumnTableRelations() {
+    return parentColumnTableRelations;
+  }
+
+  public void setParentColumnTableRelations(
+      List<ParentColumnTableRelation> parentColumnTableRelations) {
+    this.parentColumnTableRelations = parentColumnTableRelations;
+  }
+
   /**
    * hash code method to check get the hashcode based.
    * for generating the hash code only column name and column unique id will considered
@@ -282,7 +298,7 @@ public class ColumnSchema implements Serializable, Writable {
     final int prime = 31;
     int result = 1;
     result = prime * result + ((columnName == null) ? 0 : columnName.hashCode()) +
-      ((dataType == null) ? 0 : dataType.hashCode());
+        ((dataType == null) ? 0 : dataType.hashCode());
     return result;
   }
 
@@ -415,6 +431,14 @@ public class ColumnSchema implements Serializable, Writable {
     isSortColumn = sortColumn;
   }
 
+  public String getAggFunction() {
+    return aggFunction;
+  }
+
+  public void setAggFunction(String aggFunction) {
+    this.aggFunction = aggFunction;
+  }
+
   @Override
   public void write(DataOutput out) throws IOException {
     out.writeShort(dataType.getId());
@@ -451,6 +475,16 @@ public class ColumnSchema implements Serializable, Writable {
     }
     out.writeBoolean(invisible);
     out.writeBoolean(isSortColumn);
+    out.writeUTF(null != aggFunction ? aggFunction : "");
+    boolean isParentTableColumnRelationExists =
+        null != parentColumnTableRelations && parentColumnTableRelations.size() > 0;
+    out.writeBoolean(isParentTableColumnRelationExists);
+    if (isParentTableColumnRelationExists) {
+      out.writeShort(parentColumnTableRelations.size());
+      for (int i = 0; i < parentColumnTableRelations.size(); i++) {
+        parentColumnTableRelations.get(i).write(out);
+      }
+    }
   }
 
   @Override
@@ -486,5 +520,17 @@ public class ColumnSchema implements Serializable, Writable {
     }
     this.invisible = in.readBoolean();
     this.isSortColumn = in.readBoolean();
+    this.aggFunction = in.readUTF();
+    boolean isParentTableColumnRelationExists = in.readBoolean();
+    if (isParentTableColumnRelationExists) {
+      short parentColumnTableRelationSize = in.readShort();
+      this.parentColumnTableRelations = new ArrayList<>(parentColumnTableRelationSize);
+      for (int i = 0; i < parentColumnTableRelationSize; i++) {
+        ParentColumnTableRelation parentColumnTableRelation =
+            new ParentColumnTableRelation(null, null, null);
+        parentColumnTableRelation.readFields(in);
+        parentColumnTableRelations.add(parentColumnTableRelation);
+      }
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/037f630a/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java b/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java
index ea8bcb2..b9ec3f1 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java
@@ -39,7 +39,9 @@ import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.datatype.DecimalType;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
+import org.apache.carbondata.core.metadata.schema.table.RelationIdentifier;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
+import org.apache.carbondata.core.metadata.schema.table.column.ParentColumnTableRelation;
 import org.apache.carbondata.core.reader.CarbonIndexFileReader;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.format.BlockIndex;
@@ -287,9 +289,35 @@ public abstract class AbstractDataFileFooterConverter {
         wrapperColumnSchema.setSortColumn(true);
       }
     }
+    wrapperColumnSchema.setAggFunction(externalColumnSchema.getAggregate_function());
+    List<org.apache.carbondata.format.ParentColumnTableRelation> parentColumnTableRelation =
+        externalColumnSchema.getParentColumnTableRelations();
+    if (null != parentColumnTableRelation) {
+      wrapperColumnSchema.setParentColumnTableRelations(
+          fromThriftToWrapperParentTableColumnRelations(parentColumnTableRelation));
+    }
     return wrapperColumnSchema;
   }
 
+  private List<ParentColumnTableRelation> fromThriftToWrapperParentTableColumnRelations(
+      List<org.apache.carbondata.format.ParentColumnTableRelation> thirftParentColumnRelation) {
+    List<ParentColumnTableRelation> parentColumnTableRelationList = new ArrayList<>();
+    for (org.apache.carbondata.format.ParentColumnTableRelation carbonTableRelation :
+        thirftParentColumnRelation) {
+      RelationIdentifier relationIdentifier =
+          new RelationIdentifier(carbonTableRelation.getRelationIdentifier().getDatabaseName(),
+              carbonTableRelation.getRelationIdentifier().getTableName(),
+              carbonTableRelation.getRelationIdentifier().getTableId());
+      ParentColumnTableRelation parentColumnTableRelation =
+          new ParentColumnTableRelation(relationIdentifier, carbonTableRelation.getColumnId(),
+              carbonTableRelation.getColumnName());
+      parentColumnTableRelationList.add(parentColumnTableRelation);
+    }
+    return parentColumnTableRelationList;
+  }
+
+
+
   /**
    * Below method is convert the thrift encoding to wrapper encoding
    *

http://git-wip-us.apache.org/repos/asf/carbondata/blob/037f630a/format/src/main/thrift/schema.thrift
----------------------------------------------------------------------
diff --git a/format/src/main/thrift/schema.thrift b/format/src/main/thrift/schema.thrift
index 216d91f..4e9bb21 100644
--- a/format/src/main/thrift/schema.thrift
+++ b/format/src/main/thrift/schema.thrift
@@ -116,6 +116,12 @@ struct ColumnSchema{
 	 * It will have column order which user has provided
 	 */	
 	16: optional i32 schemaOrdinal
+
+  /**
+  *  to maintain the column relation with parent table.
+  *  will be usefull in case of pre-aggregate
+  **/
+	17: optional list<ParentColumnTableRelation> parentColumnTableRelations;
 }
 
 /**
@@ -168,7 +174,32 @@ struct TableSchema{
   6: optional PartitionInfo partitionInfo; // Partition information
 }
 
+struct RelationIdentifier {
+   1: optional string databaseName;
+   2: required string tableName;
+   3: required string tableId;
+}
+
+struct ParentColumnTableRelation {
+   1: required RelationIdentifier relationIdentifier;
+   2: required string columnId;
+   3: required string columnName
+}
+
+struct DataMapSchema  {
+    // class name
+    1: required string className;
+    // relation indentifier
+    2: optional RelationIdentifier relationIdentifire;
+    // in case of preaggregate it will be used to maintain the child schema
+    // which will be usefull in case of query and data load
+    3: optional TableSchema childTableSchema;
+    // to maintain properties like select query, query type like groupby, join
+    4: optional map<string, string> properties;
+}
+
 struct TableInfo{
 	1: required TableSchema fact_table;
 	2: required list<TableSchema> aggregate_table_list;
+	3: optional list<DataMapSchema> dataMapSchemas; // childSchema information
 }


[02/11] carbondata git commit: [CARBONDATA-1662] Make ArrayType and StructType contain child DataType

Posted by ra...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/933e30cc/processing/src/main/java/org/apache/carbondata/processing/loading/parser/CarbonParserFactory.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/parser/CarbonParserFactory.java b/processing/src/main/java/org/apache/carbondata/processing/loading/parser/CarbonParserFactory.java
index 608d0a9..c7adf7f 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/parser/CarbonParserFactory.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/parser/CarbonParserFactory.java
@@ -54,7 +54,7 @@ public final class CarbonParserFactory {
   private static GenericParser createParser(CarbonColumn carbonColumn, String[] complexDelimiters,
       String nullFormat, int depth) {
     DataType dataType = carbonColumn.getDataType();
-    if (dataType == DataTypes.ARRAY) {
+    if (DataTypes.isArrayType(dataType)) {
       List<CarbonDimension> listOfChildDimensions =
           ((CarbonDimension) carbonColumn).getListOfChildDimensions();
       // Create array parser with complex delimiter
@@ -63,7 +63,7 @@ public final class CarbonParserFactory {
         arrayParser.addChildren(createParser(dimension, complexDelimiters, nullFormat, depth + 1));
       }
       return arrayParser;
-    } else if (dataType == DataTypes.STRUCT) {
+    } else if (DataTypes.isStructType(dataType)) {
       List<CarbonDimension> dimensions =
           ((CarbonDimension) carbonColumn).getListOfChildDimensions();
       // Create struct parser with complex delimiter
@@ -72,7 +72,7 @@ public final class CarbonParserFactory {
         parser.addChildren(createParser(dimension, complexDelimiters, nullFormat, depth + 1));
       }
       return parser;
-    } else if (dataType == DataTypes.MAP) {
+    } else if (DataTypes.isMapType(dataType)) {
       throw new UnsupportedOperationException("Complex type Map is not supported yet");
     } else {
       return new PrimitiveParserImpl();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/933e30cc/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java
index 761867c..ca40830 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java
@@ -296,10 +296,8 @@ public final class CarbonDataProcessorUtil {
 
   private static String getComplexTypeString(DataField[] dataFields) {
     StringBuilder dimString = new StringBuilder();
-    for (int i = 0; i < dataFields.length; i++) {
-      DataField dataField = dataFields[i];
-      if (dataField.getColumn().getDataType() == DataTypes.ARRAY ||
-          dataField.getColumn().getDataType() == DataTypes.STRUCT) {
+    for (DataField dataField : dataFields) {
+      if (dataField.getColumn().getDataType().isComplexType()) {
         addAllComplexTypeChildren((CarbonDimension) dataField.getColumn(), dimString, "");
         dimString.append(CarbonCommonConstants.SEMICOLON_SPC_CHARACTER);
       }


[03/11] carbondata git commit: [CARBONDATA-1662] Make ArrayType and StructType contain child DataType

Posted by ra...@apache.org.
[CARBONDATA-1662] Make ArrayType and StructType contain child DataType

StructType and ArrayType should be class that have nested children.

This closes #1429


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

Branch: refs/heads/pre-aggregate
Commit: 933e30ccc74d499a6323f328349c9a71ba0c44e3
Parents: a91c6e4
Author: Jacky Li <ja...@qq.com>
Authored: Mon Nov 6 11:13:40 2017 +0800
Committer: ravipesala <ra...@gmail.com>
Committed: Fri Nov 10 19:38:49 2017 +0530

----------------------------------------------------------------------
 .../core/datastore/block/SegmentProperties.java |   3 +-
 .../blockletindex/BlockletDataMap.java          |   3 +-
 .../core/indexstore/row/DataMapRowImpl.java     |   2 +-
 .../ThriftWrapperSchemaConverterImpl.java       |   8 +-
 .../core/metadata/datatype/ArrayType.java       |  12 +-
 .../core/metadata/datatype/BooleanType.java     |   4 +-
 .../core/metadata/datatype/ByteArrayType.java   |   4 +-
 .../core/metadata/datatype/ByteType.java        |   4 +-
 .../core/metadata/datatype/DataTypes.java       | 110 ++++++++++++++-----
 .../core/metadata/datatype/DateType.java        |   4 +-
 .../core/metadata/datatype/DoubleType.java      |   4 +-
 .../core/metadata/datatype/FloatType.java       |   5 +-
 .../core/metadata/datatype/IntType.java         |   4 +-
 .../core/metadata/datatype/LongType.java        |   4 +-
 .../core/metadata/datatype/MapType.java         |  14 +--
 .../core/metadata/datatype/NullType.java        |   4 +-
 .../core/metadata/datatype/ShortIntType.java    |   4 +-
 .../core/metadata/datatype/ShortType.java       |   4 +-
 .../core/metadata/datatype/StringType.java      |   4 +-
 .../core/metadata/datatype/StructField.java     |  42 +++++++
 .../core/metadata/datatype/StructType.java      |  18 +--
 .../core/metadata/datatype/TimestampType.java   |   4 +-
 .../executor/impl/AbstractQueryExecutor.java    |   4 +-
 .../core/scan/executor/util/QueryUtil.java      |   8 +-
 .../expression/RangeExpressionEvaluator.java    |  28 +----
 .../scan/filter/FilterExpressionProcessor.java  |  29 ++---
 .../executer/RowLevelFilterExecuterImpl.java    |   6 +-
 .../resolver/ConditionalFilterResolverImpl.java |   7 +-
 .../util/AbstractDataFileFooterConverter.java   |   4 +-
 .../DictionaryColumnUniqueIdentifierTest.java   |   8 +-
 .../datastore/block/SegmentPropertiesTest.java  |   2 +-
 .../block/SegmentPropertiesTestUtil.java        |   2 +-
 ...ctDictionaryKeyGeneratorFactoryUnitTest.java |   2 +-
 .../ThriftWrapperSchemaConverterImplTest.java   |   6 +-
 .../table/CarbonTableWithComplexTypesTest.java  |   4 +-
 .../carbondata/core/util/CarbonUtilTest.java    |   4 +-
 .../CarbonDictionarySortInfoPreparatorTest.java |   4 +-
 .../streaming/CarbonStreamInputFormat.java      |   8 +-
 .../streaming/CarbonStreamRecordReader.java     |  32 +++---
 .../streaming/CarbonStreamRecordWriter.java     |  21 ++--
 .../hive/CarbonDictionaryDecodeReadSupport.java |   4 +-
 .../carbondata/spark/util/CarbonScalaUtil.scala |  16 ++-
 .../spark/util/DataTypeConverterUtil.scala      |  14 ++-
 .../spark/util/GlobalDictionaryUtil.scala       |  37 +++----
 .../spark/sql/CarbonDictionaryDecoder.scala     |  12 +-
 .../converter/impl/FieldEncoderFactory.java     |   6 +-
 .../loading/parser/CarbonParserFactory.java     |   6 +-
 .../util/CarbonDataProcessorUtil.java           |   6 +-
 48 files changed, 305 insertions(+), 240 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/933e30cc/core/src/main/java/org/apache/carbondata/core/datastore/block/SegmentProperties.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/block/SegmentProperties.java b/core/src/main/java/org/apache/carbondata/core/datastore/block/SegmentProperties.java
index c93b771..5f82e87 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/block/SegmentProperties.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/block/SegmentProperties.java
@@ -378,8 +378,7 @@ public class SegmentProperties {
         }
         // as complex type will be stored at last so once complex type started all the dimension
         // will be added to complex type
-        else if (isComplexDimensionStarted || CarbonUtil.hasDataType(columnSchema.getDataType(),
-            new DataType[] { DataTypes.ARRAY, DataTypes.STRUCT })) {
+        else if (isComplexDimensionStarted || columnSchema.getDataType().isComplexType()) {
           cardinalityIndexForComplexDimensionColumn.add(tableOrdinal);
           carbonDimension =
               new CarbonDimension(columnSchema, dimensonOrdinal++, -1, -1, ++complexTypeOrdinal);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/933e30cc/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java
index d5bd695..7829034 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java
@@ -284,7 +284,8 @@ public class BlockletDataMap implements DataMap, Cacheable {
           mapSchemas[i] = new DataMapSchema.FixedDataMapSchema(DataTypes.BYTE_ARRAY, minMaxLen[i]);
         }
       }
-      DataMapSchema mapSchema = new DataMapSchema.StructDataMapSchema(DataTypes.STRUCT, mapSchemas);
+      DataMapSchema mapSchema = new DataMapSchema.StructDataMapSchema(
+          DataTypes.createDefaultStructType(), mapSchemas);
       indexSchemas.add(mapSchema);
     }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/933e30cc/core/src/main/java/org/apache/carbondata/core/indexstore/row/DataMapRowImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/row/DataMapRowImpl.java b/core/src/main/java/org/apache/carbondata/core/indexstore/row/DataMapRowImpl.java
index bc55e74..032b29e 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/row/DataMapRowImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/row/DataMapRowImpl.java
@@ -99,7 +99,7 @@ public class DataMapRowImpl extends DataMapRow {
   }
 
   @Override public void setRow(DataMapRow row, int ordinal) {
-    assert (schemas[ordinal].getDataType() == DataTypes.STRUCT);
+    assert (DataTypes.isStructType(schemas[ordinal].getDataType()));
     data[ordinal] = row;
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/933e30cc/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java b/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java
index 70a6e63..adcac7d 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java
@@ -145,9 +145,9 @@ public class ThriftWrapperSchemaConverterImpl implements SchemaConverter {
       return org.apache.carbondata.format.DataType.DATE;
     } else if (dataType.getId() == DataTypes.TIMESTAMP.getId()) {
       return org.apache.carbondata.format.DataType.TIMESTAMP;
-    } else if (dataType.getId() == DataTypes.ARRAY.getId()) {
+    } else if (DataTypes.isArrayType(dataType)) {
       return org.apache.carbondata.format.DataType.ARRAY;
-    } else if (dataType.getId() == DataTypes.STRUCT.getId()) {
+    } else if (DataTypes.isStructType(dataType)) {
       return org.apache.carbondata.format.DataType.STRUCT;
     } else {
       return org.apache.carbondata.format.DataType.STRING;
@@ -392,9 +392,9 @@ public class ThriftWrapperSchemaConverterImpl implements SchemaConverter {
       case DATE:
         return DataTypes.DATE;
       case ARRAY:
-        return DataTypes.ARRAY;
+        return DataTypes.createDefaultArrayType();
       case STRUCT:
-        return DataTypes.STRUCT;
+        return DataTypes.createDefaultStructType();
       default:
         return DataTypes.STRING;
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/933e30cc/core/src/main/java/org/apache/carbondata/core/metadata/datatype/ArrayType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/ArrayType.java b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/ArrayType.java
index 241d0f6..c30e21c 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/ArrayType.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/ArrayType.java
@@ -19,15 +19,11 @@ package org.apache.carbondata.core.metadata.datatype;
 
 public class ArrayType extends DataType {
 
-  public static final DataType ARRAY = new ArrayType(DataTypes.ARRAY_TYPE_ID, 9, "ARRAY", -1);
+  private DataType elementType;
 
-  private ArrayType(int id, int precedenceOrder, String name, int sizeInBytes) {
-    super(id, precedenceOrder, name, sizeInBytes);
-  }
-
-  // this function is needed to ensure singleton pattern while supporting java serialization
-  private Object readResolve() {
-    return DataTypes.ARRAY;
+  ArrayType(DataType elementType) {
+    super(DataTypes.ARRAY_TYPE_ID, 9, "ARRAY", -1);
+    this.elementType = elementType;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/carbondata/blob/933e30cc/core/src/main/java/org/apache/carbondata/core/metadata/datatype/BooleanType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/BooleanType.java b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/BooleanType.java
index 36cb84f..0ce0412 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/BooleanType.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/BooleanType.java
@@ -17,9 +17,9 @@
 
 package org.apache.carbondata.core.metadata.datatype;
 
-public class BooleanType extends DataType {
+class BooleanType extends DataType {
 
-  public static final DataType BOOLEAN =
+  static final DataType BOOLEAN =
       new BooleanType(DataTypes.BOOLEAN_TYPE_ID, 1, "BOOLEAN", 1);
 
   private BooleanType(int id, int precedenceOrder, String name, int sizeInBytes) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/933e30cc/core/src/main/java/org/apache/carbondata/core/metadata/datatype/ByteArrayType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/ByteArrayType.java b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/ByteArrayType.java
index 1f2872d..9cb33ed 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/ByteArrayType.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/ByteArrayType.java
@@ -17,9 +17,9 @@
 
 package org.apache.carbondata.core.metadata.datatype;
 
-public class ByteArrayType extends DataType {
+class ByteArrayType extends DataType {
 
-  public static final DataType BYTE_ARRAY =
+  static final DataType BYTE_ARRAY =
       new ByteArrayType(DataTypes.BYTE_ARRAY_TYPE_ID, 13, "BYTE_ARRAY", -1);
 
   private ByteArrayType(int id, int precedenceOrder, String name, int sizeInBytes) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/933e30cc/core/src/main/java/org/apache/carbondata/core/metadata/datatype/ByteType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/ByteType.java b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/ByteType.java
index 11138d2..e1b39a3 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/ByteType.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/ByteType.java
@@ -17,9 +17,9 @@
 
 package org.apache.carbondata.core.metadata.datatype;
 
-public class ByteType extends DataType {
+class ByteType extends DataType {
 
-  public static final DataType BYTE = new ByteType(DataTypes.BYTE_TYPE_ID, 12, "BYTE", 1);
+  static final DataType BYTE = new ByteType(DataTypes.BYTE_TYPE_ID, 12, "BYTE", 1);
 
   private ByteType(int id, int precedenceOrder, String name, int sizeInBytes) {
     super(id, precedenceOrder, name, sizeInBytes);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/933e30cc/core/src/main/java/org/apache/carbondata/core/metadata/datatype/DataTypes.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/DataTypes.java b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/DataTypes.java
index 8686583..e96917f 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/DataTypes.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/DataTypes.java
@@ -17,6 +17,9 @@
 
 package org.apache.carbondata.core.metadata.datatype;
 
+import java.util.ArrayList;
+import java.util.List;
+
 /**
  * Holds all singleton object for all data type used in carbon
  */
@@ -44,28 +47,25 @@ public class DataTypes {
   // Only for internal use for backward compatability. It is only used for V1 version
   public static final DataType LEGACY_LONG = LegacyLongType.LEGACY_LONG;
 
-  public static final DataType ARRAY = ArrayType.ARRAY;
-  public static final DataType STRUCT = StructType.STRUCT;
-  public static final DataType MAP = MapType.MAP;
-
-  public static final int STRING_TYPE_ID = 0;
-  public static final int DATE_TYPE_ID = 1;
-  public static final int TIMESTAMP_TYPE_ID = 2;
-  public static final int BOOLEAN_TYPE_ID = 3;
-  public static final int SHORT_TYPE_ID = 4;
-  public static final int INT_TYPE_ID = 5;
-  public static final int FLOAT_TYPE_ID = 6;
-  public static final int LONG_TYPE_ID = 7;
-  public static final int DOUBLE_TYPE_ID = 8;
-  public static final int NULL_TYPE_ID = 9;
-  public static final int BYTE_TYPE_ID = 10;
-  public static final int BYTE_ARRAY_TYPE_ID = 11;
-  public static final int SHORT_INT_TYPE_ID = 12;
-  public static final int LEGACY_LONG_TYPE_ID = 13;
-  public static final int DECIMAL_TYPE_ID = 20;
-  public static final int ARRAY_TYPE_ID = 21;
-  public static final int STRUCT_TYPE_ID = 22;
-  public static final int MAP_TYPE_ID = 23;
+  // these IDs are used within this package only
+  static final int STRING_TYPE_ID = 0;
+  static final int DATE_TYPE_ID = 1;
+  static final int TIMESTAMP_TYPE_ID = 2;
+  static final int BOOLEAN_TYPE_ID = 3;
+  static final int SHORT_TYPE_ID = 4;
+  static final int INT_TYPE_ID = 5;
+  static final int FLOAT_TYPE_ID = 6;
+  static final int LONG_TYPE_ID = 7;
+  static final int DOUBLE_TYPE_ID = 8;
+  static final int NULL_TYPE_ID = 9;
+  static final int BYTE_TYPE_ID = 10;
+  static final int BYTE_ARRAY_TYPE_ID = 11;
+  static final int SHORT_INT_TYPE_ID = 12;
+  static final int LEGACY_LONG_TYPE_ID = 13;
+  static final int DECIMAL_TYPE_ID = 20;
+  static final int ARRAY_TYPE_ID = 21;
+  static final int STRUCT_TYPE_ID = 22;
+  static final int MAP_TYPE_ID = 23;
 
   /**
    * create a DataType instance from uniqueId of the DataType
@@ -99,12 +99,12 @@ public class DataTypes {
       return NULL;
     } else if (id == DECIMAL_TYPE_ID) {
       return createDefaultDecimalType();
-    } else if (id == ARRAY.getId()) {
-      return ARRAY;
-    } else if (id == STRUCT.getId()) {
-      return STRUCT;
-    } else if (id == MAP.getId()) {
-      return MAP;
+    } else if (id == ARRAY_TYPE_ID) {
+      return createDefaultArrayType();
+    } else if (id == STRUCT_TYPE_ID) {
+      return createDefaultStructType();
+    } else if (id == MAP_TYPE_ID) {
+      return createDefaultMapType();
     } else if (id == BYTE_ARRAY.getId()) {
       return BYTE_ARRAY;
     } else {
@@ -130,4 +130,58 @@ public class DataTypes {
     return dataType.getId() == DECIMAL_TYPE_ID;
   }
 
+  /**
+   * create array type with specified element type
+   */
+  public static ArrayType createArrayType(DataType elementType) {
+    return new ArrayType(elementType);
+  }
+
+  /**
+   * create a array type object with no child
+   */
+  public static ArrayType createDefaultArrayType() {
+    return new ArrayType(STRING);
+  }
+
+  public static boolean isArrayType(DataType dataType) {
+    return dataType.getId() == ARRAY_TYPE_ID;
+  }
+
+  /**
+   * create struct type with specified fields
+   */
+  public static StructType createStructType(List<StructField> fields) {
+    return new StructType(fields);
+  }
+
+  /**
+   * create a struct type object with no field
+   */
+  public static StructType createDefaultStructType() {
+    return new StructType(new ArrayList<StructField>());
+  }
+
+  public static boolean isStructType(DataType dataType) {
+    return dataType.getId() == STRUCT_TYPE_ID;
+  }
+
+  /**
+   * create map type with specified key type and value type
+   */
+  public static MapType createMapType(DataType keyType, DataType valueType) {
+    return new MapType(keyType, valueType);
+  }
+
+  /**
+   * create a map type object with no child
+   */
+  public static MapType createDefaultMapType() {
+    return new MapType(STRING, STRING);
+  }
+
+  public static boolean isMapType(DataType dataType) {
+    return dataType.getId() == MAP_TYPE_ID;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/933e30cc/core/src/main/java/org/apache/carbondata/core/metadata/datatype/DateType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/DateType.java b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/DateType.java
index 2b930c3..efc9799 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/DateType.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/DateType.java
@@ -17,9 +17,9 @@
 
 package org.apache.carbondata.core.metadata.datatype;
 
-public class DateType extends DataType {
+class DateType extends DataType {
 
-  public static final DataType DATE = new DateType(DataTypes.DATE_TYPE_ID, 1, "DATE", -1);
+  static final DataType DATE = new DateType(DataTypes.DATE_TYPE_ID, 1, "DATE", -1);
 
   private DateType(int id, int precedenceOrder, String name, int sizeInBytes) {
     super(id, precedenceOrder, name, sizeInBytes);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/933e30cc/core/src/main/java/org/apache/carbondata/core/metadata/datatype/DoubleType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/DoubleType.java b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/DoubleType.java
index 59a4cc2..5e032a0 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/DoubleType.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/DoubleType.java
@@ -17,9 +17,9 @@
 
 package org.apache.carbondata.core.metadata.datatype;
 
-public class DoubleType extends DataType {
+class DoubleType extends DataType {
 
-  public static final DataType DOUBLE = new DoubleType(DataTypes.DOUBLE_TYPE_ID, 6, "DOUBLE", 8);
+  static final DataType DOUBLE = new DoubleType(DataTypes.DOUBLE_TYPE_ID, 6, "DOUBLE", 8);
 
   private DoubleType(int id, int precedenceOrder, String name, int sizeInBytes) {
     super(id, precedenceOrder, name, sizeInBytes);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/933e30cc/core/src/main/java/org/apache/carbondata/core/metadata/datatype/FloatType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/FloatType.java b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/FloatType.java
index 1c0bfb6..9f600f9 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/FloatType.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/FloatType.java
@@ -17,8 +17,9 @@
 
 package org.apache.carbondata.core.metadata.datatype;
 
-public class FloatType extends DataType {
-  public static final DataType FLOAT = new FloatType(DataTypes.FLOAT_TYPE_ID, 4, "FLOAT", 4);
+class FloatType extends DataType {
+
+  static final DataType FLOAT = new FloatType(DataTypes.FLOAT_TYPE_ID, 4, "FLOAT", 4);
 
   private FloatType(int id, int precedenceOrder, String name, int sizeInBytes) {
     super(id, precedenceOrder, name, sizeInBytes);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/933e30cc/core/src/main/java/org/apache/carbondata/core/metadata/datatype/IntType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/IntType.java b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/IntType.java
index 4324d9e..308a53b 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/IntType.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/IntType.java
@@ -17,9 +17,9 @@
 
 package org.apache.carbondata.core.metadata.datatype;
 
-public class IntType extends DataType {
+class IntType extends DataType {
 
-  public static final DataType INT = new IntType(DataTypes.INT_TYPE_ID, 3, "INT", 4);
+  static final DataType INT = new IntType(DataTypes.INT_TYPE_ID, 3, "INT", 4);
 
   private IntType(int id, int precedenceOrder, String name, int sizeInBytes) {
     super(id, precedenceOrder, name, sizeInBytes);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/933e30cc/core/src/main/java/org/apache/carbondata/core/metadata/datatype/LongType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/LongType.java b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/LongType.java
index 3d05fd9..3bc1a4b 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/LongType.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/LongType.java
@@ -17,9 +17,9 @@
 
 package org.apache.carbondata.core.metadata.datatype;
 
-public class LongType extends DataType {
+class LongType extends DataType {
 
-  public static final DataType LONG = new LongType(DataTypes.LONG_TYPE_ID, 5, "LONG", 8);
+  static final DataType LONG = new LongType(DataTypes.LONG_TYPE_ID, 5, "LONG", 8);
 
   private LongType(int id, int precedenceOrder, String name, int sizeInBytes) {
     super(id, precedenceOrder, name, sizeInBytes);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/933e30cc/core/src/main/java/org/apache/carbondata/core/metadata/datatype/MapType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/MapType.java b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/MapType.java
index a360243..69d49b8 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/MapType.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/MapType.java
@@ -19,15 +19,13 @@ package org.apache.carbondata.core.metadata.datatype;
 
 public class MapType extends DataType {
 
-  public static final DataType MAP = new MapType(DataTypes.MAP_TYPE_ID, 11, "MAP", -1);
+  private DataType keyType;
+  private DataType valueType;
 
-  private MapType(int id, int precedenceOrder, String name, int sizeInBytes) {
-    super(id, precedenceOrder, name, sizeInBytes);
-  }
-
-  // this function is needed to ensure singleton pattern while supporting java serialization
-  private Object readResolve() {
-    return DataTypes.MAP;
+  MapType(DataType keyType, DataType valueType) {
+    super(DataTypes.MAP_TYPE_ID, 11, "MAP", -1);
+    this.keyType = keyType;
+    this.valueType = valueType;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/carbondata/blob/933e30cc/core/src/main/java/org/apache/carbondata/core/metadata/datatype/NullType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/NullType.java b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/NullType.java
index 43ffe41..81248c5 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/NullType.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/NullType.java
@@ -17,9 +17,9 @@
 
 package org.apache.carbondata.core.metadata.datatype;
 
-public class NullType extends DataType {
+class NullType extends DataType {
 
-  public static final DataType NULL = new NullType(DataTypes.NULL_TYPE_ID, 7, "NULL", 1);
+  static final DataType NULL = new NullType(DataTypes.NULL_TYPE_ID, 7, "NULL", 1);
 
   private NullType(int id, int precedenceOrder, String name, int sizeInBytes) {
     super(id, precedenceOrder, name, sizeInBytes);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/933e30cc/core/src/main/java/org/apache/carbondata/core/metadata/datatype/ShortIntType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/ShortIntType.java b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/ShortIntType.java
index 5d57748..24c90d5 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/ShortIntType.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/ShortIntType.java
@@ -17,9 +17,9 @@
 
 package org.apache.carbondata.core.metadata.datatype;
 
-public class ShortIntType extends DataType {
+class ShortIntType extends DataType {
 
-  public static final DataType SHORT_INT =
+  static final DataType SHORT_INT =
       new ShortIntType(DataTypes.SHORT_INT_TYPE_ID, 14, "SHORT_INT", 3);
 
   private ShortIntType(int id, int precedenceOrder, String name, int sizeInBytes) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/933e30cc/core/src/main/java/org/apache/carbondata/core/metadata/datatype/ShortType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/ShortType.java b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/ShortType.java
index 0085227..4bf5563 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/ShortType.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/ShortType.java
@@ -17,9 +17,9 @@
 
 package org.apache.carbondata.core.metadata.datatype;
 
-public class ShortType extends DataType {
+class ShortType extends DataType {
 
-  public static final DataType SHORT = new ShortType(DataTypes.SHORT_TYPE_ID, 2, "SHORT", 2);
+  static final DataType SHORT = new ShortType(DataTypes.SHORT_TYPE_ID, 2, "SHORT", 2);
 
   private ShortType(int id, int precedenceOrder, String name, int sizeInBytes) {
     super(id, precedenceOrder, name, sizeInBytes);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/933e30cc/core/src/main/java/org/apache/carbondata/core/metadata/datatype/StringType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/StringType.java b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/StringType.java
index 5c95480..18c505a 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/StringType.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/StringType.java
@@ -17,8 +17,8 @@
 
 package org.apache.carbondata.core.metadata.datatype;
 
-public class StringType extends DataType {
-  public static final DataType STRING = new StringType(DataTypes.STRING_TYPE_ID, 0, "STRING", -1);
+class StringType extends DataType {
+  static final DataType STRING = new StringType(DataTypes.STRING_TYPE_ID, 0, "STRING", -1);
 
   private StringType(int id, int precedenceOrder, String name, int sizeInBytes) {
     super(id, precedenceOrder, name, sizeInBytes);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/933e30cc/core/src/main/java/org/apache/carbondata/core/metadata/datatype/StructField.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/StructField.java b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/StructField.java
new file mode 100644
index 0000000..efdc8e2
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/StructField.java
@@ -0,0 +1,42 @@
+/*
+ * 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.core.metadata.datatype;
+
+import java.io.Serializable;
+
+public class StructField implements Serializable {
+
+  private static final long serialVersionUID = 3271726L;
+
+  private String fieldName;
+
+  private DataType dataType;
+
+  public StructField(String fieldName, DataType dataType) {
+    this.fieldName = fieldName;
+    this.dataType = dataType;
+  }
+
+  public DataType getDataType() {
+    return dataType;
+  }
+
+  public String getFieldName() {
+    return fieldName;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/933e30cc/core/src/main/java/org/apache/carbondata/core/metadata/datatype/StructType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/StructType.java b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/StructType.java
index 081f952..6417f37 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/StructType.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/StructType.java
@@ -17,21 +17,23 @@
 
 package org.apache.carbondata.core.metadata.datatype;
 
-public class StructType extends DataType {
+import java.util.List;
 
-  public static final DataType STRUCT = new StructType(DataTypes.STRUCT_TYPE_ID, 10, "STRUCT", -1);
+class StructType extends DataType {
 
-  private StructType(int id, int precedenceOrder, String name, int sizeInBytes) {
-    super(id, precedenceOrder, name, sizeInBytes);
-  }
+  private List<StructField> fields;
 
-  // this function is needed to ensure singleton pattern while supporting java serialization
-  private Object readResolve() {
-    return DataTypes.STRUCT;
+  StructType(List<StructField> fields) {
+    super(DataTypes.STRUCT_TYPE_ID, 10, "STRUCT", -1);
+    this.fields = fields;
   }
 
   @Override
   public boolean isComplexType() {
     return true;
   }
+
+  public List<StructField> getFields() {
+    return fields;
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/933e30cc/core/src/main/java/org/apache/carbondata/core/metadata/datatype/TimestampType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/TimestampType.java b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/TimestampType.java
index 287c457..42a6855 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/TimestampType.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/TimestampType.java
@@ -17,8 +17,8 @@
 
 package org.apache.carbondata.core.metadata.datatype;
 
-public class TimestampType extends DataType {
-  public static final TimestampType TIMESTAMP =
+class TimestampType extends DataType {
+  static final TimestampType TIMESTAMP =
       new TimestampType(DataTypes.TIMESTAMP_TYPE_ID, 2, "TIMESTAMP", -1);
 
   private TimestampType(int id, int precedenceOrder, String name, int sizeInBytes) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/933e30cc/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/AbstractQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/AbstractQueryExecutor.java b/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/AbstractQueryExecutor.java
index 56ed575..251a06b 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/AbstractQueryExecutor.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/AbstractQueryExecutor.java
@@ -49,7 +49,6 @@ import org.apache.carbondata.core.keygenerator.KeyGenerator;
 import org.apache.carbondata.core.memory.UnsafeMemoryManager;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.datatype.DataType;
-import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
@@ -478,8 +477,7 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
   private int[] getComplexDimensionParentBlockIndexes(List<QueryDimension> queryDimensions) {
     List<Integer> parentBlockIndexList = new ArrayList<Integer>();
     for (QueryDimension queryDimension : queryDimensions) {
-      if (CarbonUtil.hasDataType(queryDimension.getDimension().getDataType(),
-          new DataType[] { DataTypes.ARRAY, DataTypes.STRUCT, DataTypes.MAP })) {
+      if (queryDimension.getDimension().getDataType().isComplexType()) {
         parentBlockIndexList.add(queryDimension.getDimension().getOrdinal());
       }
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/933e30cc/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java b/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java
index 6aab8a0..b090e59 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java
@@ -763,10 +763,10 @@ public class QueryUtil {
       int[] eachComplexColumnValueSize, Map<String, Dictionary> columnIdToDictionaryMap) {
     int parentBlockIndex = dimensionToBlockIndexMap.get(dimension.getOrdinal());
     GenericQueryType parentQueryType;
-    if (dimension.getDataType() == DataTypes.ARRAY) {
+    if (DataTypes.isArrayType(dimension.getDataType())) {
       parentQueryType =
           new ArrayQueryType(dimension.getColName(), dimension.getColName(), parentBlockIndex);
-    } else if (dimension.getDataType() == DataTypes.STRUCT) {
+    } else if (DataTypes.isStructType(dimension.getDataType())) {
       parentQueryType =
           new StructQueryType(dimension.getColName(), dimension.getColName(),
               dimensionToBlockIndexMap.get(dimension.getOrdinal()));
@@ -784,11 +784,11 @@ public class QueryUtil {
       CarbonDimension dimension, GenericQueryType parentQueryType) {
     for (int i = 0; i < dimension.getNumberOfChild(); i++) {
       DataType dataType = dimension.getListOfChildDimensions().get(i).getDataType();
-      if (dataType == DataTypes.ARRAY) {
+      if (DataTypes.isArrayType(dataType)) {
         parentQueryType.addChildren(
             new ArrayQueryType(dimension.getListOfChildDimensions().get(i).getColName(),
                 dimension.getColName(), ++parentBlockIndex));
-      } else if (dataType == DataTypes.STRUCT) {
+      } else if (DataTypes.isStructType(dataType)) {
         parentQueryType.addChildren(
             new StructQueryType(dimension.getListOfChildDimensions().get(i).getColName(),
                 dimension.getColName(), ++parentBlockIndex));

http://git-wip-us.apache.org/repos/asf/carbondata/blob/933e30cc/core/src/main/java/org/apache/carbondata/core/scan/expression/RangeExpressionEvaluator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/expression/RangeExpressionEvaluator.java b/core/src/main/java/org/apache/carbondata/core/scan/expression/RangeExpressionEvaluator.java
index 9d418ee..e7cc1c3 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/expression/RangeExpressionEvaluator.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/expression/RangeExpressionEvaluator.java
@@ -27,7 +27,6 @@ import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.metadata.datatype.DataType;
-import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.scan.expression.conditional.GreaterThanEqualToExpression;
 import org.apache.carbondata.core.scan.expression.conditional.GreaterThanExpression;
 import org.apache.carbondata.core.scan.expression.conditional.LessThanEqualToExpression;
@@ -259,13 +258,9 @@ public class RangeExpressionEvaluator {
    * @return
    */
   private boolean isLessThanGreaterThanExp(Expression expr) {
-    if ((expr instanceof LessThanEqualToExpression) || (expr instanceof LessThanExpression)
+    return (expr instanceof LessThanEqualToExpression) || (expr instanceof LessThanExpression)
         || (expr instanceof GreaterThanEqualToExpression)
-        || (expr instanceof GreaterThanExpression)) {
-      return true;
-    } else {
-      return false;
-    }
+        || (expr instanceof GreaterThanExpression);
   }
 
   /**
@@ -277,15 +272,8 @@ public class RangeExpressionEvaluator {
   private boolean eligibleForRangeExpConv(Expression expChild) {
     for (Expression exp : expChild.getChildren()) {
       if (exp instanceof ColumnExpression) {
-        if (((ColumnExpression) exp).isDimension() == false) {
-          return false;
-        }
-        if ((((ColumnExpression) exp).getDimension().getDataType() == DataTypes.ARRAY) || (
-            ((ColumnExpression) exp).getDimension().getDataType() == DataTypes.STRUCT)) {
-          return false;
-        } else {
-          return true;
-        }
+        return ((ColumnExpression) exp).isDimension() &&
+            ! (((ColumnExpression) exp).getDimension().getDataType().isComplexType());
       }
     }
     return false;
@@ -384,13 +372,9 @@ public class RangeExpressionEvaluator {
    * @return
    */
   private boolean matchExpType(ExpressionType src, ExpressionType tar) {
-    if ((((src == LESSTHAN) || (src == LESSTHAN_EQUALTO)) && ((tar == GREATERTHAN) || (tar
+    return (((src == LESSTHAN) || (src == LESSTHAN_EQUALTO)) && ((tar == GREATERTHAN) || (tar
         == GREATERTHAN_EQUALTO))) || (((src == GREATERTHAN) || (src == GREATERTHAN_EQUALTO)) && (
-        (tar == LESSTHAN) || (tar == LESSTHAN_EQUALTO)))) {
-      return true;
-    } else {
-      return false;
-    }
+        (tar == LESSTHAN) || (tar == LESSTHAN_EQUALTO)));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/933e30cc/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java
index 68787e3..e77d58e 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java
@@ -30,9 +30,9 @@ import org.apache.carbondata.core.datastore.block.AbstractIndex;
 import org.apache.carbondata.core.datastore.impl.btree.BTreeDataRefNodeFinder;
 import org.apache.carbondata.core.keygenerator.KeyGenException;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.PartitionInfo;
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
 import org.apache.carbondata.core.scan.expression.BinaryExpression;
 import org.apache.carbondata.core.scan.expression.ColumnExpression;
 import org.apache.carbondata.core.scan.expression.Expression;
@@ -402,13 +402,9 @@ public class FilterExpressionProcessor implements FilterProcessor {
         return new TrueConditionalResolverImpl(expression, false, false, tableIdentifier);
       case EQUALS:
         currentCondExpression = (BinaryConditionalExpression) expression;
-        if (currentCondExpression.isSingleColumn()
-            && currentCondExpression.getColumnList().get(0).getCarbonColumn().getDataType()
-            != DataTypes.ARRAY
-            && currentCondExpression.getColumnList().get(0).getCarbonColumn().getDataType()
-            != DataTypes.STRUCT) {
-
-          if (currentCondExpression.getColumnList().get(0).getCarbonColumn().isMeasure()) {
+        CarbonColumn column = currentCondExpression.getColumnList().get(0).getCarbonColumn();
+        if (currentCondExpression.isSingleColumn() && ! column.getDataType().isComplexType()) {
+          if (column.isMeasure()) {
             if (FilterUtil.checkIfExpressionContainsColumn(currentCondExpression.getLeft())
                 && FilterUtil.checkIfExpressionContainsColumn(currentCondExpression.getRight()) || (
                 FilterUtil.checkIfRightExpressionRequireEvaluation(currentCondExpression.getRight())
@@ -463,13 +459,9 @@ public class FilterExpressionProcessor implements FilterProcessor {
             tableIdentifier, false);
       case NOT_EQUALS:
         currentCondExpression = (BinaryConditionalExpression) expression;
-        if (currentCondExpression.isSingleColumn()
-            && currentCondExpression.getColumnList().get(0).getCarbonColumn().getDataType()
-            != DataTypes.ARRAY
-            && currentCondExpression.getColumnList().get(0).getCarbonColumn().getDataType()
-            != DataTypes.STRUCT) {
-
-          if (currentCondExpression.getColumnList().get(0).getCarbonColumn().isMeasure()) {
+        column = currentCondExpression.getColumnList().get(0).getCarbonColumn();
+        if (currentCondExpression.isSingleColumn() && ! column.getDataType().isComplexType()) {
+          if (column.isMeasure()) {
             if (FilterUtil.checkIfExpressionContainsColumn(currentCondExpression.getLeft())
                 && FilterUtil.checkIfExpressionContainsColumn(currentCondExpression.getRight()) || (
                 FilterUtil.checkIfRightExpressionRequireEvaluation(currentCondExpression.getRight())
@@ -522,11 +514,8 @@ public class FilterExpressionProcessor implements FilterProcessor {
       default:
         if (expression instanceof ConditionalExpression) {
           condExpression = (ConditionalExpression) expression;
-          if (condExpression.isSingleColumn()
-              && condExpression.getColumnList().get(0).getCarbonColumn().getDataType()
-              != DataTypes.ARRAY
-              && condExpression.getColumnList().get(0).getCarbonColumn().getDataType()
-              != DataTypes.STRUCT) {
+          column = condExpression.getColumnList().get(0).getCarbonColumn();
+          if (condExpression.isSingleColumn() && ! column.isComplex()) {
             condExpression = (ConditionalExpression) expression;
             if ((condExpression.getColumnList().get(0).getCarbonColumn()
                 .hasEncoding(Encoding.DICTIONARY) && !condExpression.getColumnList().get(0)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/933e30cc/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelFilterExecuterImpl.java
index 777f564..224a69f 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelFilterExecuterImpl.java
@@ -297,8 +297,7 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
             getDimensionDefaultValue(dimColumnEvaluatorInfo);
         continue;
       }
-      if (dimColumnEvaluatorInfo.getDimension().getDataType() != DataTypes.ARRAY
-          && dimColumnEvaluatorInfo.getDimension().getDataType() != DataTypes.STRUCT) {
+      if (!dimColumnEvaluatorInfo.getDimension().getDataType().isComplexType()) {
         if (!dimColumnEvaluatorInfo.isDimensionExistsInCurrentSilce()) {
           record[dimColumnEvaluatorInfo.getRowIndex()] =
               dimColumnEvaluatorInfo.getDimension().getDefaultValue();
@@ -534,8 +533,7 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
   @Override public void readBlocks(BlocksChunkHolder blockChunkHolder) throws IOException {
     for (int i = 0; i < dimColEvaluatorInfoList.size(); i++) {
       DimColumnResolvedFilterInfo dimColumnEvaluatorInfo = dimColEvaluatorInfoList.get(i);
-      if (dimColumnEvaluatorInfo.getDimension().getDataType() != DataTypes.ARRAY
-          && dimColumnEvaluatorInfo.getDimension().getDataType() != DataTypes.STRUCT) {
+      if (!dimColumnEvaluatorInfo.getDimension().getDataType().isComplexType()) {
         if (null == blockChunkHolder.getDimensionRawDataChunk()[dimensionBlocksIndex[i]]) {
           blockChunkHolder.getDimensionRawDataChunk()[dimensionBlocksIndex[i]] =
               blockChunkHolder.getDataBlock()

http://git-wip-us.apache.org/repos/asf/carbondata/blob/933e30cc/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/ConditionalFilterResolverImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/ConditionalFilterResolverImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/ConditionalFilterResolverImpl.java
index bd78e08..d966802 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/ConditionalFilterResolverImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/ConditionalFilterResolverImpl.java
@@ -167,11 +167,8 @@ public class ConditionalFilterResolverImpl implements FilterResolverIntf {
             metadata);
 
       } else if ((null != columnList.get(0).getDimension()) && (
-          columnList.get(0).getDimension().hasEncoding(Encoding.DICTIONARY) && !(
-              columnList.get(0).getDimension().getDataType()
-                  == org.apache.carbondata.core.metadata.datatype.DataTypes.STRUCT
-                  || columnList.get(0).getDimension().getDataType()
-                  == org.apache.carbondata.core.metadata.datatype.DataTypes.ARRAY))) {
+          columnList.get(0).getDimension().hasEncoding(Encoding.DICTIONARY) &&
+              ! columnList.get(0).getDimension().getDataType().isComplexType())) {
         dimColResolvedFilterInfo.setFilterValues(FilterUtil
             .getFilterListForAllValues(absoluteTableIdentifier, exp, columnList.get(0),
                 isIncludeFilter, tableProvider));

http://git-wip-us.apache.org/repos/asf/carbondata/blob/933e30cc/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java b/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java
index aca2fd6..ea8bcb2 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java
@@ -381,9 +381,9 @@ public abstract class AbstractDataFileFooterConverter {
       case TIMESTAMP:
         return DataTypes.TIMESTAMP;
       case ARRAY:
-        return DataTypes.ARRAY;
+        return DataTypes.createDefaultArrayType();
       case STRUCT:
-        return DataTypes.STRUCT;
+        return DataTypes.createDefaultStructType();
       default:
         return DataTypes.STRING;
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/933e30cc/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryColumnUniqueIdentifierTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryColumnUniqueIdentifierTest.java b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryColumnUniqueIdentifierTest.java
index ec9a19a..bd8b6bc 100644
--- a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryColumnUniqueIdentifierTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryColumnUniqueIdentifierTest.java
@@ -49,17 +49,17 @@ public class DictionaryColumnUniqueIdentifierTest {
     ColumnIdentifier columnIdentifier2 = new ColumnIdentifier("1", properties, DataTypes.INT);
     dictionaryColumnUniqueIdentifier1 =
         new DictionaryColumnUniqueIdentifier(carbonTableIdentifier1, columnIdentifier,
-            DataTypes.MAP, null);
+            DataTypes.STRING, null);
     dictionaryColumnUniqueIdentifier2 =
         new DictionaryColumnUniqueIdentifier(carbonTableIdentifier2, columnIdentifier2,
-            DataTypes.MAP, null);
+            DataTypes.STRING, null);
     dictionaryColumnUniqueIdentifier3 =
         new DictionaryColumnUniqueIdentifier(carbonTableIdentifier2, columnIdentifier,
-            DataTypes.MAP, null);
+            DataTypes.STRING, null);
   }
 
   @Test public void testToGetDataType() {
-    assertEquals(dictionaryColumnUniqueIdentifier1.getDataType(), DataTypes.MAP);
+    assertEquals(dictionaryColumnUniqueIdentifier1.getDataType(), DataTypes.STRING);
   }
 
   @Test public void testForEqualsWithDifferentObjectsWithDifferentColumnIdentifier() {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/933e30cc/core/src/test/java/org/apache/carbondata/core/datastore/block/SegmentPropertiesTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/datastore/block/SegmentPropertiesTest.java b/core/src/test/java/org/apache/carbondata/core/datastore/block/SegmentPropertiesTest.java
index 2020dd8..9c43553 100644
--- a/core/src/test/java/org/apache/carbondata/core/datastore/block/SegmentPropertiesTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/datastore/block/SegmentPropertiesTest.java
@@ -296,7 +296,7 @@ public class SegmentPropertiesTest extends TestCase {
     dimColumn.setColumnar(true);
     dimColumn.setColumnName("IMEI5");
     dimColumn.setColumnUniqueId(UUID.randomUUID().toString());
-    dimColumn.setDataType(DataTypes.ARRAY);
+    dimColumn.setDataType(DataTypes.createDefaultArrayType());
     dimColumn.setDimensionColumn(true);
     List<Encoding> encodeList =
         new ArrayList<Encoding>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/933e30cc/core/src/test/java/org/apache/carbondata/core/datastore/block/SegmentPropertiesTestUtil.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/datastore/block/SegmentPropertiesTestUtil.java b/core/src/test/java/org/apache/carbondata/core/datastore/block/SegmentPropertiesTestUtil.java
index 1f9c712..9572627 100644
--- a/core/src/test/java/org/apache/carbondata/core/datastore/block/SegmentPropertiesTestUtil.java
+++ b/core/src/test/java/org/apache/carbondata/core/datastore/block/SegmentPropertiesTestUtil.java
@@ -179,7 +179,7 @@ public class SegmentPropertiesTestUtil {
     dimColumn.setColumnar(true);
     dimColumn.setColumnName("IMEI5");
     dimColumn.setColumnUniqueId(UUID.randomUUID().toString());
-    dimColumn.setDataType(DataTypes.ARRAY);
+    dimColumn.setDataType(DataTypes.createDefaultArrayType());
     dimColumn.setDimensionColumn(true);
     List<Encoding> encodeList =
         new ArrayList<Encoding>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/933e30cc/core/src/test/java/org/apache/carbondata/core/keygenerator/directdictionary/DirectDictionaryKeyGeneratorFactoryUnitTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/keygenerator/directdictionary/DirectDictionaryKeyGeneratorFactoryUnitTest.java b/core/src/test/java/org/apache/carbondata/core/keygenerator/directdictionary/DirectDictionaryKeyGeneratorFactoryUnitTest.java
index c077eeb..2b73b86 100644
--- a/core/src/test/java/org/apache/carbondata/core/keygenerator/directdictionary/DirectDictionaryKeyGeneratorFactoryUnitTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/keygenerator/directdictionary/DirectDictionaryKeyGeneratorFactoryUnitTest.java
@@ -37,7 +37,7 @@ public class DirectDictionaryKeyGeneratorFactoryUnitTest {
 
   @Test public void testGetDirectDictionaryGeneratorReturnNull() throws Exception {
     DirectDictionaryGenerator result =
-        DirectDictionaryKeyGeneratorFactory.getDirectDictionaryGenerator(DataTypes.ARRAY);
+        DirectDictionaryKeyGeneratorFactory.getDirectDictionaryGenerator(DataTypes.STRING);
     Assert.assertNull(result);
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/933e30cc/core/src/test/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImplTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImplTest.java b/core/src/test/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImplTest.java
index 42c0ad6..b8e4eea 100644
--- a/core/src/test/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImplTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImplTest.java
@@ -23,8 +23,6 @@ import java.util.Map;
 
 import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
-import org.apache.carbondata.core.metadata.datatype.DecimalType;
-import org.apache.carbondata.core.metadata.datatype.IntType;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.SchemaEvolution;
 import org.apache.carbondata.core.metadata.schema.SchemaEvolutionEntry;
@@ -1025,7 +1023,7 @@ public class ThriftWrapperSchemaConverterImplTest {
       }
 
       @Mock public DataType getDataType() {
-        return DataTypes.ARRAY;
+        return DataTypes.createDefaultArrayType();
       }
 
       @Mock public String getColumnName() {
@@ -1099,7 +1097,7 @@ public class ThriftWrapperSchemaConverterImplTest {
       }
 
       @Mock public DataType getDataType() {
-        return DataTypes.STRUCT;
+        return DataTypes.createDefaultStructType();
       }
 
       @Mock public String getColumnName() {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/933e30cc/core/src/test/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableWithComplexTypesTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableWithComplexTypesTest.java b/core/src/test/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableWithComplexTypesTest.java
index 4979038..0aed719 100644
--- a/core/src/test/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableWithComplexTypesTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableWithComplexTypesTest.java
@@ -83,7 +83,7 @@ public class CarbonTableWithComplexTypesTest extends TestCase {
     structColumn.setColumnar(true);
     structColumn.setColumnName("mobile");
     structColumn.setColumnUniqueId(UUID.randomUUID().toString());
-    structColumn.setDataType(DataTypes.STRUCT);
+    structColumn.setDataType(DataTypes.createDefaultStructType());
     structColumn.setDimensionColumn(true);
     structColumn.setEncodingList(encodeList);
     structColumn.setNumberOfChild(2);
@@ -103,7 +103,7 @@ public class CarbonTableWithComplexTypesTest extends TestCase {
     arrayColumn.setColumnar(true);
     arrayColumn.setColumnName("mobile.val");
     arrayColumn.setColumnUniqueId(UUID.randomUUID().toString());
-    arrayColumn.setDataType(DataTypes.ARRAY);
+    arrayColumn.setDataType(DataTypes.createDefaultArrayType());
     arrayColumn.setDimensionColumn(true);
     arrayColumn.setEncodingList(encodeList);
     arrayColumn.setNumberOfChild(1);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/933e30cc/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java b/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java
index 56c59e1..f9b5ec8 100644
--- a/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java
@@ -515,7 +515,7 @@ public class CarbonUtilTest {
   }
 
   @Test public void testForHasComplexDataTypes() {
-    assertTrue(DataTypes.ARRAY.isComplexType());
+    assertTrue(DataTypes.createDefaultArrayType().isComplexType());
     assertTrue(!DataTypes.DATE.isComplexType());
   }
 
@@ -581,7 +581,7 @@ public class CarbonUtilTest {
     column1.setDimension(new CarbonDimension(column1Schema, 1, 1, 1, 1));
 
     column2Schema.setColumnName("Column2");
-    column2Schema.setDataType(DataTypes.ARRAY);
+    column2Schema.setDataType(DataTypes.createDefaultArrayType());
     column2.setDimension(new CarbonDimension(column2Schema, 1, 1, 1, 1));
 
     QueryDimension[] queryDimensions = { column1, column2 };

http://git-wip-us.apache.org/repos/asf/carbondata/blob/933e30cc/core/src/test/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortInfoPreparatorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortInfoPreparatorTest.java b/core/src/test/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortInfoPreparatorTest.java
index cff1291..0775a3d 100644
--- a/core/src/test/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortInfoPreparatorTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortInfoPreparatorTest.java
@@ -68,7 +68,7 @@ public class CarbonDictionarySortInfoPreparatorTest {
     };
 
     CarbonDictionarySortInfo carbonDictionarySortInfo = carbonDictionarySortInfoPreparator
-        .getDictionarySortInfo(newDistinctValues, dictionary, DataTypes.ARRAY);
+        .getDictionarySortInfo(newDistinctValues, dictionary, DataTypes.STRING);
     int expectedGetSortIndexValue = 1;
     int expectedGetSortInvertedIndexLength = 3;
     int actualGetSortIndexValue = carbonDictionarySortInfo.getSortIndex().get(0);
@@ -87,7 +87,7 @@ public class CarbonDictionarySortInfoPreparatorTest {
     newDistinctValues.add("xyz");
     Dictionary dictionary = null;
     CarbonDictionarySortInfo carbonDictionarySortInfo = carbonDictionarySortInfoPreparator
-        .getDictionarySortInfo(newDistinctValues, dictionary, DataTypes.ARRAY);
+        .getDictionarySortInfo(newDistinctValues, dictionary, DataTypes.createDefaultArrayType());
     int expectedGetSortIndexValue = 1;
     int expectedGetSortInvertedIndexLength = 2;
     int actualGetSortIndexValue = carbonDictionarySortInfo.getSortIndex().get(0);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/933e30cc/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamInputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamInputFormat.java
index b10bc8b..eb07f7e 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamInputFormat.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamInputFormat.java
@@ -59,10 +59,10 @@ public class CarbonStreamInputFormat extends FileInputFormat<Void, Object> {
     GenericQueryType[] queryTypes = new GenericQueryType[carbonColumns.length];
     for (int i = 0; i < carbonColumns.length; i++) {
       if (carbonColumns[i].isComplex()) {
-        if (carbonColumns[i].getDataType() == DataTypes.ARRAY) {
+        if (DataTypes.isArrayType(carbonColumns[i].getDataType())) {
           queryTypes[i] = new ArrayQueryType(carbonColumns[i].getColName(),
               carbonColumns[i].getColName(), i);
-        } else if (carbonColumns[i].getDataType() == DataTypes.STRUCT) {
+        } else if (DataTypes.isStructType(carbonColumns[i].getDataType())) {
           queryTypes[i] = new StructQueryType(carbonColumns[i].getColName(),
               carbonColumns[i].getColName(), i);
         } else {
@@ -84,11 +84,11 @@ public class CarbonStreamInputFormat extends FileInputFormat<Void, Object> {
       CarbonDimension child = dimension.getListOfChildDimensions().get(i);
       DataType dataType = child.getDataType();
       GenericQueryType queryType = null;
-      if (dataType == DataTypes.ARRAY) {
+      if (DataTypes.isArrayType(dataType)) {
         queryType =
             new ArrayQueryType(child.getColName(), dimension.getColName(), ++parentBlockIndex);
 
-      } else if (dataType == DataTypes.STRUCT) {
+      } else if (DataTypes.isStructType(dataType)) {
         queryType =
             new StructQueryType(child.getColName(), dimension.getColName(), ++parentBlockIndex);
         parentQueryType.addChildren(queryType);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/933e30cc/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordReader.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordReader.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordReader.java
index 1ff0fa7..4e3e6cf 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordReader.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordReader.java
@@ -35,6 +35,7 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
 import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
+import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
@@ -75,12 +76,9 @@ import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
 import org.apache.spark.sql.execution.vectorized.ColumnVector;
 import org.apache.spark.sql.execution.vectorized.ColumnarBatch;
 import org.apache.spark.sql.types.CalendarIntervalType;
-import org.apache.spark.sql.types.DataType;
-import org.apache.spark.sql.types.DateType;
 import org.apache.spark.sql.types.Decimal;
 import org.apache.spark.sql.types.DecimalType;
 import org.apache.spark.sql.types.StructType;
-import org.apache.spark.sql.types.TimestampType;
 import org.apache.spark.unsafe.types.CalendarInterval;
 import org.apache.spark.unsafe.types.UTF8String;
 
@@ -95,7 +93,7 @@ public class CarbonStreamRecordReader extends RecordReader<Void, Object> {
   private CarbonTable carbonTable;
   private CarbonColumn[] storageColumns;
   private boolean[] isRequired;
-  private int[] measureDataTypes;
+  private DataType[] measureDataTypes;
   private int dimensionCount;
   private int measureCount;
 
@@ -171,9 +169,9 @@ public class CarbonStreamRecordReader extends RecordReader<Void, Object> {
             .getDirectDictionaryGenerator(storageColumns[i].getDataType());
       }
     }
-    measureDataTypes = new int[measureCount];
+    measureDataTypes = new DataType[measureCount];
     for (int i = 0; i < measureCount; i++) {
-      measureDataTypes[i] = storageColumns[dimensionCount + i].getDataType().getId();
+      measureDataTypes[i] = storageColumns[dimensionCount + i].getDataType();
     }
 
     // decode data
@@ -518,7 +516,7 @@ public class CarbonStreamRecordReader extends RecordReader<Void, Object> {
       }
     }
     // measure
-    int dataType;
+    DataType dataType;
     for (int msrCount = 0; msrCount < measureCount; msrCount++, colCount++) {
       if (nullBitSet.get(colCount)) {
         if (isFilterRequired[colCount]) {
@@ -529,7 +527,7 @@ public class CarbonStreamRecordReader extends RecordReader<Void, Object> {
         }
       } else {
         dataType = measureDataTypes[msrCount];
-        if (dataType == DataTypes.BOOLEAN_TYPE_ID) {
+        if (dataType == DataTypes.BOOLEAN) {
           if (isRequired[colCount]) {
             boolean v = input.readBoolean();
             if (isFilterRequired[colCount]) {
@@ -541,7 +539,7 @@ public class CarbonStreamRecordReader extends RecordReader<Void, Object> {
           } else {
             input.skipBytes(1);
           }
-        } else if (dataType == DataTypes.SHORT_TYPE_ID) {
+        } else if (dataType == DataTypes.SHORT) {
           if (isRequired[colCount]) {
             short v = input.readShort();
             if (isFilterRequired[colCount]) {
@@ -553,7 +551,7 @@ public class CarbonStreamRecordReader extends RecordReader<Void, Object> {
           } else {
             input.skipBytes(2);
           }
-        } else if (dataType == DataTypes.INT_TYPE_ID) {
+        } else if (dataType == DataTypes.INT) {
           if (isRequired[colCount]) {
             int v = input.readInt();
             if (isFilterRequired[colCount]) {
@@ -565,7 +563,7 @@ public class CarbonStreamRecordReader extends RecordReader<Void, Object> {
           } else {
             input.skipBytes(4);
           }
-        } else if (dataType == DataTypes.LONG_TYPE_ID) {
+        } else if (dataType == DataTypes.LONG) {
           if (isRequired[colCount]) {
             long v = input.readLong();
             if (isFilterRequired[colCount]) {
@@ -577,7 +575,7 @@ public class CarbonStreamRecordReader extends RecordReader<Void, Object> {
           } else {
             input.skipBytes(8);
           }
-        } else if (dataType == DataTypes.DOUBLE_TYPE_ID) {
+        } else if (dataType == DataTypes.DOUBLE) {
           if (isRequired[colCount]) {
             double v = input.readDouble();
             if (isFilterRequired[colCount]) {
@@ -589,7 +587,7 @@ public class CarbonStreamRecordReader extends RecordReader<Void, Object> {
           } else {
             input.skipBytes(8);
           }
-        } else if (dataType == DataTypes.DECIMAL_TYPE_ID) {
+        } else if (DataTypes.isDecimal(dataType)) {
           int len = input.readShort();
           if (isRequired[colCount]) {
             BigDecimal v = DataTypeUtil.byteToBigDecimal(input.readBytes(len));
@@ -611,7 +609,7 @@ public class CarbonStreamRecordReader extends RecordReader<Void, Object> {
     for (int i = 0; i < projection.length; i++) {
       Object value = outputValues[i];
       ColumnVector col = columnarBatch.column(i);
-      DataType t = col.dataType();
+      org.apache.spark.sql.types.DataType t = col.dataType();
       if (null == value) {
         col.putNull(rowId);
       } else {
@@ -632,7 +630,7 @@ public class CarbonStreamRecordReader extends RecordReader<Void, Object> {
         } else if (t == org.apache.spark.sql.types.DataTypes.StringType) {
           UTF8String v = (UTF8String) value;
           col.putByteArray(rowId, v.getBytes());
-        } else if (t instanceof DecimalType) {
+        } else if (t instanceof org.apache.spark.sql.types.DecimalType) {
           DecimalType dt = (DecimalType)t;
           Decimal d = (Decimal) value;
           if (dt.precision() <= Decimal.MAX_INT_DIGITS()) {
@@ -648,9 +646,9 @@ public class CarbonStreamRecordReader extends RecordReader<Void, Object> {
           CalendarInterval c = (CalendarInterval) value;
           col.getChildColumn(0).putInt(rowId, c.months);
           col.getChildColumn(1).putLong(rowId, c.microseconds);
-        } else if (t instanceof DateType) {
+        } else if (t instanceof org.apache.spark.sql.types.DateType) {
           col.putInt(rowId, (int) value);
-        } else if (t instanceof TimestampType) {
+        } else if (t instanceof org.apache.spark.sql.types.TimestampType) {
           col.putLong(rowId, (long) value);
         }
       }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/933e30cc/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordWriter.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordWriter.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordWriter.java
index 8d7a2e3..3c4b55c 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordWriter.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordWriter.java
@@ -30,6 +30,7 @@ import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.datastore.row.CarbonRow;
+import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
@@ -83,7 +84,7 @@ public class CarbonStreamRecordWriter extends RecordWriter<Void, Object> {
   private boolean[] isNoDictionaryDimensionColumn;
   private int dimensionWithComplexCount;
   private int measureCount;
-  private int[] measureDataTypes;
+  private DataType[] measureDataTypes;
   private StreamBlockletWriter output = null;
 
   // data write
@@ -129,10 +130,10 @@ public class CarbonStreamRecordWriter extends RecordWriter<Void, Object> {
     dimensionWithComplexCount = configuration.getDimensionCount();
     measureCount = configuration.getMeasureCount();
     dataFields = configuration.getDataFields();
-    measureDataTypes = new int[measureCount];
+    measureDataTypes = new DataType[measureCount];
     for (int i = 0; i < measureCount; i++) {
       measureDataTypes[i] =
-          dataFields[dimensionWithComplexCount + i].getColumn().getDataType().getId();
+          dataFields[dimensionWithComplexCount + i].getColumn().getDataType();
     }
 
     // initialize parser and converter
@@ -211,22 +212,22 @@ public class CarbonStreamRecordWriter extends RecordWriter<Void, Object> {
       }
     }
     // measure
-    int dataType;
+    DataType dataType;
     for (int msrCount = 0; msrCount < measureCount; msrCount++) {
       columnValue = currentRow.getObject(dimCount + msrCount);
       if (null != columnValue) {
         dataType = measureDataTypes[msrCount];
-        if (dataType == DataTypes.BOOLEAN_TYPE_ID) {
+        if (dataType == DataTypes.BOOLEAN) {
           output.writeBoolean((boolean) columnValue);
-        } else if (dataType == DataTypes.SHORT_TYPE_ID) {
+        } else if (dataType == DataTypes.SHORT) {
           output.writeShort((short) columnValue);
-        } else if (dataType == DataTypes.INT_TYPE_ID) {
+        } else if (dataType == DataTypes.INT) {
           output.writeInt((int) columnValue);
-        } else if (dataType == DataTypes.LONG_TYPE_ID) {
+        } else if (dataType == DataTypes.LONG) {
           output.writeLong((long) columnValue);
-        } else if (dataType == DataTypes.DOUBLE_TYPE_ID) {
+        } else if (dataType == DataTypes.DOUBLE) {
           output.writeDouble((double) columnValue);
-        } else if (dataType == DataTypes.DECIMAL_TYPE_ID) {
+        } else if (DataTypes.isDecimal(dataType)) {
           BigDecimal val = (BigDecimal) columnValue;
           byte[] bigDecimalInBytes = DataTypeUtil.bigDecimalToByte(val);
           output.writeShort(bigDecimalInBytes.length);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/933e30cc/integration/hive/src/main/java/org/apache/carbondata/hive/CarbonDictionaryDecodeReadSupport.java
----------------------------------------------------------------------
diff --git a/integration/hive/src/main/java/org/apache/carbondata/hive/CarbonDictionaryDecodeReadSupport.java b/integration/hive/src/main/java/org/apache/carbondata/hive/CarbonDictionaryDecodeReadSupport.java
index 05ad93a..2a19271 100644
--- a/integration/hive/src/main/java/org/apache/carbondata/hive/CarbonDictionaryDecodeReadSupport.java
+++ b/integration/hive/src/main/java/org/apache/carbondata/hive/CarbonDictionaryDecodeReadSupport.java
@@ -137,9 +137,9 @@ public class CarbonDictionaryDecodeReadSupport<T> implements CarbonReadSupport<T
    */
   private Writable createWritableObject(Object obj, CarbonColumn carbonColumn) throws IOException {
     DataType dataType = carbonColumn.getDataType();
-    if (dataType == DataTypes.STRUCT) {
+    if (DataTypes.isStructType(dataType)) {
       return createStruct(obj, carbonColumn);
-    } else if (dataType == DataTypes.ARRAY) {
+    } else if (DataTypes.isArrayType(dataType)) {
       return createArray(obj, carbonColumn);
     } else {
       return createWritablePrimitive(obj, carbonColumn);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/933e30cc/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala
index 2043ecf..7abb737 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala
@@ -19,13 +19,14 @@ package org.apache.carbondata.spark.util
 
 import java.nio.charset.Charset
 import java.text.SimpleDateFormat
+import java.util
 
 import org.apache.spark.sql._
 import org.apache.spark.sql.execution.command.DataTypeInfo
 import org.apache.spark.sql.types._
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.metadata.datatype.{DataType => CarbonDataType, DataTypes => CarbonDataTypes, DecimalType => CarbonDecimalType}
+import org.apache.carbondata.core.metadata.datatype.{DataType => CarbonDataType, DataTypes => CarbonDataTypes, DecimalType => CarbonDecimalType, StructField => CarbonStructField}
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn
 
 object CarbonScalaUtil {
@@ -40,8 +41,17 @@ object CarbonScalaUtil {
       case DateType => CarbonDataTypes.DATE
       case BooleanType => CarbonDataTypes.BOOLEAN
       case TimestampType => CarbonDataTypes.TIMESTAMP
-      case ArrayType(_, _) => CarbonDataTypes.ARRAY
-      case StructType(_) => CarbonDataTypes.STRUCT
+      case ArrayType(elementType, _) =>
+        CarbonDataTypes.createArrayType(CarbonScalaUtil.convertSparkToCarbonDataType(elementType))
+      case StructType(fields) =>
+        val carbonFields = new util.ArrayList[CarbonStructField]
+        fields.map { field =>
+          carbonFields.add(
+            new CarbonStructField(
+              field.name,
+              CarbonScalaUtil.convertSparkToCarbonDataType(field.dataType)))
+        }
+        CarbonDataTypes.createStructType(carbonFields)
       case NullType => CarbonDataTypes.NULL
       case decimal: DecimalType =>
         CarbonDataTypes.createDecimalType(decimal.precision, decimal.scale)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/933e30cc/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataTypeConverterUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataTypeConverterUtil.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataTypeConverterUtil.scala
index 0460e30..6cf7298 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataTypeConverterUtil.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataTypeConverterUtil.scala
@@ -41,8 +41,8 @@ object DataTypeConverterUtil {
       case FIXED_DECIMAL(_, _) => DataTypes.createDefaultDecimalType
       case "timestamp" => DataTypes.TIMESTAMP
       case "date" => DataTypes.DATE
-      case "array" => DataTypes.ARRAY
-      case "struct" => DataTypes.STRUCT
+      case "array" => DataTypes.createDefaultArrayType
+      case "struct" => DataTypes.createDefaultStructType
       case _ => convertToCarbonTypeForSpark2(dataType)
     }
   }
@@ -66,9 +66,9 @@ object DataTypeConverterUtil {
       case "datetype" => DataTypes.DATE
       case others =>
         if (others != null && others.startsWith("arraytype")) {
-          DataTypes.ARRAY
+          DataTypes.createDefaultArrayType()
         } else if (others != null && others.startsWith("structtype")) {
-          DataTypes.STRUCT
+          DataTypes.createDefaultStructType()
         } else if (others != null && others.startsWith("char")) {
           DataTypes.STRING
         } else if (others != null && others.startsWith("varchar")) {
@@ -82,6 +82,10 @@ object DataTypeConverterUtil {
   def convertToString(dataType: DataType): String = {
     if (DataTypes.isDecimal(dataType)) {
       "decimal"
+    } else if (DataTypes.isArrayType(dataType)) {
+      "array"
+    } else if (DataTypes.isStructType(dataType)) {
+      "struct"
     } else {
       dataType match {
         case DataTypes.BOOLEAN => "boolean"
@@ -93,8 +97,6 @@ object DataTypeConverterUtil {
         case DataTypes.FLOAT => "double"
         case DataTypes.TIMESTAMP => "timestamp"
         case DataTypes.DATE => "date"
-        case DataTypes.ARRAY => "array"
-        case DataTypes.STRUCT => "struct"
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/933e30cc/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 ddc4763..840e8ae 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
@@ -259,17 +259,16 @@ object GlobalDictionaryUtil {
       case None =>
         None
       case Some(dim) =>
-        dim.getDataType match {
-          case DataTypes.ARRAY =>
-            val arrDim = ArrayParser(dim, format)
-            generateParserForChildrenDimension(dim, format, mapColumnValuesWithId, arrDim)
-            Some(arrDim)
-          case DataTypes.STRUCT =>
-            val stuDim = StructParser(dim, format)
-            generateParserForChildrenDimension(dim, format, mapColumnValuesWithId, stuDim)
-            Some(stuDim)
-          case _ =>
-            Some(PrimitiveParser(dim, mapColumnValuesWithId.get(dim.getColumnId)))
+        if (DataTypes.isArrayType(dim.getDataType)) {
+          val arrDim = ArrayParser(dim, format)
+          generateParserForChildrenDimension(dim, format, mapColumnValuesWithId, arrDim)
+          Some(arrDim)
+        } else if (DataTypes.isStructType(dim.getDataType)) {
+          val stuDim = StructParser(dim, format)
+          generateParserForChildrenDimension(dim, format, mapColumnValuesWithId, stuDim)
+          Some(stuDim)
+        } else {
+          Some(PrimitiveParser(dim, mapColumnValuesWithId.get(dim.getColumnId)))
         }
     }
   }
@@ -477,14 +476,14 @@ object GlobalDictionaryUtil {
       val children = preDictDimension.getListOfChildDimensions.asScala.toArray
       // for Array, user set ArrayFiled: path, while ArrayField has a child Array.val
       val currentColName = {
-        preDictDimension.getDataType match {
-          case DataTypes.ARRAY =>
-            if (children(0).isComplex) {
-              "val." + colName.substring(middleDimName.length + 1)
-            } else {
-              "val"
-            }
-          case _ => colName.substring(middleDimName.length + 1)
+        if (DataTypes.isArrayType(preDictDimension.getDataType)) {
+          if (children(0).isComplex) {
+            "val." + colName.substring(middleDimName.length + 1)
+          } else {
+            "val"
+          }
+        } else {
+          colName.substring(middleDimName.length + 1)
         }
       }
       setPredefineDict(carbonLoadModel, children, table, currentColName,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/933e30cc/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
index 91c07de..18f76d1 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
@@ -392,6 +392,12 @@ object CarbonDictionaryDecoder {
       } else {
         DecimalType(precision, scale)
       }
+    } else if (CarbonDataTypes.isArrayType(carbonDimension.getDataType)) {
+      CarbonMetastoreTypes
+        .toDataType(s"array<${ relation.getArrayChildren(carbonDimension.getColName) }>")
+    } else if (CarbonDataTypes.isStructType(carbonDimension.getDataType)) {
+      CarbonMetastoreTypes
+        .toDataType(s"struct<${ relation.getStructChildren(carbonDimension.getColName) }>")
     } else {
       carbonDimension.getDataType match {
         case CarbonDataTypes.STRING => StringType
@@ -402,12 +408,6 @@ object CarbonDictionaryDecoder {
         case CarbonDataTypes.BOOLEAN => BooleanType
         case CarbonDataTypes.TIMESTAMP => TimestampType
         case CarbonDataTypes.DATE => DateType
-        case CarbonDataTypes.STRUCT =>
-          CarbonMetastoreTypes
-            .toDataType(s"struct<${ relation.getStructChildren(carbonDimension.getColName) }>")
-        case CarbonDataTypes.ARRAY =>
-          CarbonMetastoreTypes
-            .toDataType(s"array<${ relation.getArrayChildren(carbonDimension.getColName) }>")
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/933e30cc/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/FieldEncoderFactory.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/FieldEncoderFactory.java b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/FieldEncoderFactory.java
index 193d3e6..5dbd177 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/FieldEncoderFactory.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/FieldEncoderFactory.java
@@ -111,7 +111,7 @@ public class FieldEncoderFactory {
       CarbonTableIdentifier carbonTableIdentifier, DictionaryClient client, Boolean useOnePass,
       String storePath, Map<Object, Integer> localCache) {
     DataType dataType = carbonColumn.getDataType();
-    if (dataType == DataTypes.ARRAY) {
+    if (DataTypes.isArrayType(dataType)) {
       List<CarbonDimension> listOfChildDimensions =
           ((CarbonDimension) carbonColumn).getListOfChildDimensions();
       // Create array parser with complex delimiter
@@ -123,7 +123,7 @@ public class FieldEncoderFactory {
                 client, useOnePass, storePath, localCache));
       }
       return arrayDataType;
-    } else if (dataType == DataTypes.STRUCT) {
+    } else if (DataTypes.isStructType(dataType)) {
       List<CarbonDimension> dimensions =
           ((CarbonDimension) carbonColumn).getListOfChildDimensions();
       // Create struct parser with complex delimiter
@@ -135,7 +135,7 @@ public class FieldEncoderFactory {
                 client, useOnePass, storePath, localCache));
       }
       return structDataType;
-    } else if (dataType == DataTypes.MAP) {
+    } else if (DataTypes.isMapType(dataType)) {
       throw new UnsupportedOperationException("Complex type Map is not supported yet");
     } else {
       return new PrimitiveDataType(carbonColumn.getColName(), parentName,