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/13 22:11:27 UTC

[01/49] carbondata git commit: [CARBONDATA-1659] Remove spark 1.x info [Forced Update!]

Repository: carbondata
Updated Branches:
  refs/heads/fgdatamap 3c28a1369 -> 7f6172212 (forced update)


[CARBONDATA-1659] Remove spark 1.x info

This closes #1456


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

Branch: refs/heads/fgdatamap
Commit: 9955bed24447034f04f291dbdb2e1446e51ad8f1
Parents: 8e6b0a7
Author: Zhang Zhichao <44...@qq.com>
Authored: Tue Oct 31 14:09:30 2017 +0800
Committer: chenliang613 <ch...@huawei.com>
Committed: Tue Oct 31 23:08:25 2017 +0530

----------------------------------------------------------------------
 integration/spark-common-cluster-test/pom.xml | 22 ----------------------
 1 file changed, 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/9955bed2/integration/spark-common-cluster-test/pom.xml
----------------------------------------------------------------------
diff --git a/integration/spark-common-cluster-test/pom.xml b/integration/spark-common-cluster-test/pom.xml
index e529035..3723bc7 100644
--- a/integration/spark-common-cluster-test/pom.xml
+++ b/integration/spark-common-cluster-test/pom.xml
@@ -154,28 +154,6 @@
   </build>
   <profiles>
     <profile>
-      <id>spark-1.5</id>
-      <dependencies>
-        <dependency>
-          <groupId>org.apache.carbondata</groupId>
-          <artifactId>carbondata-spark</artifactId>
-          <version>${project.version}</version>
-          <scope>test</scope>
-        </dependency>
-      </dependencies>
-    </profile>
-    <profile>
-      <id>spark-1.6</id>
-      <dependencies>
-        <dependency>
-          <groupId>org.apache.carbondata</groupId>
-          <artifactId>carbondata-spark</artifactId>
-          <version>${project.version}</version>
-          <scope>test</scope>
-        </dependency>
-      </dependencies>
-    </profile>
-    <profile>
       <id>spark-2.1</id>
       <activation>
         <activeByDefault>true</activeByDefault>


[04/49] carbondata git commit: [CARBONDATA-1617] Merging carbonindex files within segment

Posted by ra...@apache.org.
[CARBONDATA-1617] Merging carbonindex files within segment

Merge the carbonindex files after data load, so that we can reduce the IO calls to namenode and improves the read performance for first query

This closes #1436


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

Branch: refs/heads/fgdatamap
Commit: 0586146a8bd953db63e1d99608ba8a77a9f5a899
Parents: b491609
Author: ravipesala <ra...@gmail.com>
Authored: Wed Oct 25 11:13:22 2017 +0530
Committer: Jacky Li <ja...@qq.com>
Committed: Wed Nov 1 21:14:34 2017 +0530

----------------------------------------------------------------------
 .../core/constants/CarbonCommonConstants.java   |   8 +
 .../carbondata/core/datamap/TableDataMap.java   |   6 +-
 .../carbondata/core/datamap/dev/DataMap.java    |   2 +-
 .../core/datamap/dev/DataMapFactory.java        |   4 +-
 .../core/datamap/dev/DataMapModel.java          |  34 ++++
 .../indexstore/BlockletDataMapIndexStore.java   |  64 +++++--
 .../blockletindex/BlockletDataMap.java          |  13 +-
 .../blockletindex/BlockletDataMapFactory.java   |  47 ++---
 .../blockletindex/BlockletDataMapModel.java     |  33 ++++
 .../blockletindex/SegmentIndexFileStore.java    | 189 +++++++++++++++++++
 .../core/reader/CarbonIndexFileReader.java      |  10 +
 .../carbondata/core/reader/ThriftReader.java    |   9 +
 .../util/AbstractDataFileFooterConverter.java   |   8 +-
 .../apache/carbondata/core/util/CarbonUtil.java |   4 +-
 .../core/util/path/CarbonTablePath.java         |  49 +++--
 .../core/writer/CarbonIndexFileMergeWriter.java | 116 ++++++++++++
 .../main/thrift/carbondata_index_merge.thrift   |  32 ++++
 .../hadoop/api/CarbonTableInputFormat.java      |  14 +-
 .../sdv/generated/MergeIndexTestCase.scala      | 122 ++++++++++++
 .../cluster/sdv/suite/SDVSuites.scala           |   3 +-
 .../CarbonIndexFileMergeTestCase.scala          | 143 ++++++++++++++
 ...ompactionSupportGlobalSortFunctionTest.scala |   6 +-
 ...mpactionSupportGlobalSortParameterTest.scala |   8 +-
 .../dataload/TestBatchSortDataLoad.scala        |   8 +-
 .../dataload/TestGlobalSortDataLoad.scala       |   7 +-
 .../testsuite/datamap/DataMapWriterSuite.scala  |   2 +-
 .../spark/rdd/CarbonMergeFilesRDD.scala         |  82 ++++++++
 .../apache/carbondata/spark/rdd/Compactor.scala |   6 +-
 .../carbondata/spark/util/CommonUtil.scala      |  20 +-
 .../spark/sql/catalyst/CarbonDDLSqlParser.scala |   3 +-
 .../spark/rdd/CarbonDataRDDFactory.scala        |  23 ++-
 .../sql/execution/strategy/DDLStrategy.scala    |   3 +-
 .../processing/merger/CarbonDataMergerUtil.java |  37 ++--
 .../processing/merger/CompactionType.java       |   1 +
 34 files changed, 996 insertions(+), 120 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/0586146a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
index 4ee3001..3fed18f 100644
--- a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
+++ b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
@@ -1376,6 +1376,14 @@ public final class CarbonCommonConstants {
 
   public static final String BITSET_PIPE_LINE_DEFAULT = "true";
 
+  /**
+   * It is internal configuration and used only for test purpose.
+   * It will merge the carbon index files with in the segment to single segment.
+   */
+  public static final String CARBON_MERGE_INDEX_IN_SEGMENT = "carbon.merge.index.in.segment";
+
+  public static final String CARBON_MERGE_INDEX_IN_SEGMENT_DEFAULT = "true";
+
   private CarbonCommonConstants() {
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/0586146a/core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java b/core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java
index 3e5e9e4..1cf1def 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java
@@ -118,7 +118,11 @@ public final class TableDataMap implements EventListener {
   public List<ExtendedBlocklet> prune(DataMapDistributable distributable,
       FilterResolverIntf filterExp) throws IOException {
     List<ExtendedBlocklet> detailedBlocklets = new ArrayList<>();
-    List<Blocklet> blocklets = dataMapFactory.getDataMap(distributable).prune(filterExp);
+    List<Blocklet> blocklets = new ArrayList<>();
+    List<DataMap> dataMaps = dataMapFactory.getDataMaps(distributable);
+    for (DataMap dataMap : dataMaps) {
+      blocklets.addAll(dataMap.prune(filterExp));
+    }
     for (Blocklet blocklet: blocklets) {
       ExtendedBlocklet detailedBlocklet =
           blockletDetailsFetcher.getExtendedBlocklet(blocklet, distributable.getSegmentId());

http://git-wip-us.apache.org/repos/asf/carbondata/blob/0586146a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java
index f6ea885..ada23ad 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java
@@ -31,7 +31,7 @@ public interface DataMap {
   /**
    * It is called to load the data map to memory or to initialize it.
    */
-  void init(String filePath) throws MemoryException, IOException;
+  void init(DataMapModel dataMapModel) throws MemoryException, IOException;
 
   /**
    * Prune the datamap with filter expression. It returns the list of

http://git-wip-us.apache.org/repos/asf/carbondata/blob/0586146a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java
index 62cf813..cf0519b 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java
@@ -45,9 +45,9 @@ public interface DataMapFactory {
   List<DataMap> getDataMaps(String segmentId) throws IOException;
 
   /**
-   * Get datamap for distributable object.
+   * Get datamaps for distributable object.
    */
-  DataMap getDataMap(DataMapDistributable distributable);
+  List<DataMap> getDataMaps(DataMapDistributable distributable) throws IOException;
 
   /**
    * Get all distributable objects of a segmentid

http://git-wip-us.apache.org/repos/asf/carbondata/blob/0586146a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapModel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapModel.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapModel.java
new file mode 100644
index 0000000..76bbeee
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapModel.java
@@ -0,0 +1,34 @@
+/*
+ * 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.datamap.dev;
+
+/**
+ * Information required to build datamap
+ */
+public class DataMapModel {
+
+  private String filePath;
+
+  public DataMapModel(String filePath) {
+    this.filePath = filePath;
+  }
+
+  public String getFilePath() {
+    return filePath;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/0586146a/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDataMapIndexStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDataMapIndexStore.java b/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDataMapIndexStore.java
index 9d4af7b..1ea4806 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDataMapIndexStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDataMapIndexStore.java
@@ -18,6 +18,7 @@ package org.apache.carbondata.core.indexstore;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
@@ -27,7 +28,10 @@ import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.cache.Cache;
 import org.apache.carbondata.core.cache.CarbonLRUCache;
 import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataMap;
+import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataMapModel;
+import org.apache.carbondata.core.indexstore.blockletindex.SegmentIndexFileStore;
 import org.apache.carbondata.core.memory.MemoryException;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
 
 /**
  * Class to handle loading, unloading,clearing,storing of the table
@@ -66,13 +70,19 @@ public class BlockletDataMapIndexStore
     segmentLockMap = new ConcurrentHashMap<String, Object>();
   }
 
-  @Override public BlockletDataMap get(TableBlockIndexUniqueIdentifier tableSegmentUniqueIdentifier)
+  @Override
+  public BlockletDataMap get(TableBlockIndexUniqueIdentifier identifier)
       throws IOException {
-    String lruCacheKey = tableSegmentUniqueIdentifier.getUniqueTableSegmentIdentifier();
+    String lruCacheKey = identifier.getUniqueTableSegmentIdentifier();
     BlockletDataMap dataMap = (BlockletDataMap) lruCache.get(lruCacheKey);
     if (dataMap == null) {
       try {
-        dataMap = loadAndGetDataMap(tableSegmentUniqueIdentifier);
+        SegmentIndexFileStore indexFileStore = new SegmentIndexFileStore();
+        indexFileStore.readAllIIndexOfSegment(
+            CarbonTablePath.getSegmentPath(
+                identifier.getAbsoluteTableIdentifier().getTablePath(),
+                identifier.getSegmentId()));
+        dataMap = loadAndGetDataMap(identifier, indexFileStore);
       } catch (MemoryException e) {
         LOGGER.error("memory exception when loading datamap: " + e.getMessage());
         throw new RuntimeException(e.getMessage(), e);
@@ -81,18 +91,41 @@ public class BlockletDataMapIndexStore
     return dataMap;
   }
 
-  @Override public List<BlockletDataMap> getAll(
+  @Override
+  public List<BlockletDataMap> getAll(
       List<TableBlockIndexUniqueIdentifier> tableSegmentUniqueIdentifiers) throws IOException {
     List<BlockletDataMap> blockletDataMaps = new ArrayList<>(tableSegmentUniqueIdentifiers.size());
+    List<TableBlockIndexUniqueIdentifier> missedIdentifiers = new ArrayList<>();
+    // Get the datamaps for each indexfile from cache.
     try {
       for (TableBlockIndexUniqueIdentifier identifier : tableSegmentUniqueIdentifiers) {
-        blockletDataMaps.add(get(identifier));
+        BlockletDataMap ifPresent = getIfPresent(identifier);
+        if (ifPresent != null) {
+          blockletDataMaps.add(ifPresent);
+        } else {
+          missedIdentifiers.add(identifier);
+        }
+      }
+      if (missedIdentifiers.size() > 0) {
+        Map<String, SegmentIndexFileStore> segmentIndexFileStoreMap = new HashMap<>();
+        for (TableBlockIndexUniqueIdentifier identifier: missedIdentifiers) {
+          SegmentIndexFileStore indexFileStore =
+              segmentIndexFileStoreMap.get(identifier.getSegmentId());
+          if (indexFileStore == null) {
+            String segmentPath = CarbonTablePath
+                .getSegmentPath(identifier.getAbsoluteTableIdentifier().getTablePath(),
+                    identifier.getSegmentId());
+            indexFileStore = new SegmentIndexFileStore();
+            indexFileStore.readAllIIndexOfSegment(segmentPath);
+            segmentIndexFileStoreMap.put(identifier.getSegmentId(), indexFileStore);
+          }
+          blockletDataMaps.add(loadAndGetDataMap(identifier, indexFileStore));
+        }
       }
     } catch (Throwable e) {
       for (BlockletDataMap dataMap : blockletDataMaps) {
         dataMap.clear();
       }
-      e.printStackTrace();
       throw new IOException("Problem in loading segment blocks.", e);
     }
     return blockletDataMaps;
@@ -104,7 +137,8 @@ public class BlockletDataMapIndexStore
    * @param tableSegmentUniqueIdentifier
    * @return
    */
-  @Override public BlockletDataMap getIfPresent(
+  @Override
+  public BlockletDataMap getIfPresent(
       TableBlockIndexUniqueIdentifier tableSegmentUniqueIdentifier) {
     BlockletDataMap dataMap = (BlockletDataMap) lruCache
         .get(tableSegmentUniqueIdentifier.getUniqueTableSegmentIdentifier());
@@ -116,7 +150,8 @@ public class BlockletDataMapIndexStore
    *
    * @param tableSegmentUniqueIdentifier
    */
-  @Override public void invalidate(TableBlockIndexUniqueIdentifier tableSegmentUniqueIdentifier) {
+  @Override
+  public void invalidate(TableBlockIndexUniqueIdentifier tableSegmentUniqueIdentifier) {
     lruCache.remove(tableSegmentUniqueIdentifier.getUniqueTableSegmentIdentifier());
   }
 
@@ -130,10 +165,11 @@ public class BlockletDataMapIndexStore
    * @throws IOException
    */
   private BlockletDataMap loadAndGetDataMap(
-      TableBlockIndexUniqueIdentifier tableSegmentUniqueIdentifier)
+      TableBlockIndexUniqueIdentifier identifier,
+      SegmentIndexFileStore indexFileStore)
       throws IOException, MemoryException {
     String uniqueTableSegmentIdentifier =
-        tableSegmentUniqueIdentifier.getUniqueTableSegmentIdentifier();
+        identifier.getUniqueTableSegmentIdentifier();
     Object lock = segmentLockMap.get(uniqueTableSegmentIdentifier);
     if (lock == null) {
       lock = addAndGetSegmentLock(uniqueTableSegmentIdentifier);
@@ -141,8 +177,9 @@ public class BlockletDataMapIndexStore
     BlockletDataMap dataMap = null;
     synchronized (lock) {
       dataMap = new BlockletDataMap();
-      dataMap.init(tableSegmentUniqueIdentifier.getFilePath());
-      lruCache.put(tableSegmentUniqueIdentifier.getUniqueTableSegmentIdentifier(), dataMap,
+      dataMap.init(new BlockletDataMapModel(identifier.getFilePath(),
+          indexFileStore.getFileData(identifier.getCarbonIndexFileName())));
+      lruCache.put(identifier.getUniqueTableSegmentIdentifier(), dataMap,
           dataMap.getMemorySize());
     }
     return dataMap;
@@ -170,7 +207,8 @@ public class BlockletDataMapIndexStore
    *
    * @param tableSegmentUniqueIdentifiers
    */
-  @Override public void clearAccessCount(
+  @Override
+  public void clearAccessCount(
       List<TableBlockIndexUniqueIdentifier> tableSegmentUniqueIdentifiers) {
     for (TableBlockIndexUniqueIdentifier segmentUniqueIdentifier : tableSegmentUniqueIdentifiers) {
       BlockletDataMap cacheable =

http://git-wip-us.apache.org/repos/asf/carbondata/blob/0586146a/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 0d7bb71..3e083cc 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
@@ -35,6 +35,7 @@ import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.cache.Cacheable;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datamap.dev.DataMap;
+import org.apache.carbondata.core.datamap.dev.DataMapModel;
 import org.apache.carbondata.core.datastore.IndexKey;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.datastore.block.TableBlockInfo;
@@ -96,10 +97,13 @@ public class BlockletDataMap implements DataMap, Cacheable {
   private int[] columnCardinality;
 
   @Override
-  public void init(String filePath) throws IOException, MemoryException {
+  public void init(DataMapModel dataMapModel) throws IOException, MemoryException {
     long startTime = System.currentTimeMillis();
+    assert (dataMapModel instanceof BlockletDataMapModel);
+    BlockletDataMapModel blockletDataMapInfo = (BlockletDataMapModel) dataMapModel;
     DataFileFooterConverter fileFooterConverter = new DataFileFooterConverter();
-    List<DataFileFooter> indexInfo = fileFooterConverter.getIndexInfo(filePath);
+    List<DataFileFooter> indexInfo = fileFooterConverter
+        .getIndexInfo(blockletDataMapInfo.getFilePath(), blockletDataMapInfo.getFileData());
     for (DataFileFooter fileFooter : indexInfo) {
       List<ColumnSchema> columnInTable = fileFooter.getColumnInTable();
       if (segmentProperties == null) {
@@ -119,8 +123,9 @@ public class BlockletDataMap implements DataMap, Cacheable {
     if (unsafeMemoryDMStore != null) {
       unsafeMemoryDMStore.finishWriting();
     }
-    LOGGER.info("Time taken to load blocklet datamap from file : " + filePath + "is " +
-        (System.currentTimeMillis() - startTime));
+    LOGGER.info(
+        "Time taken to load blocklet datamap from file : " + dataMapModel.getFilePath() + "is " + (
+            System.currentTimeMillis() - startTime));
   }
 
   private void loadToUnsafe(DataFileFooter fileFooter, SegmentProperties segmentProperties,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/0586146a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java
index 5edc5b7..a1bbba5 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java
@@ -31,7 +31,6 @@ import org.apache.carbondata.core.datamap.dev.DataMap;
 import org.apache.carbondata.core.datamap.dev.DataMapFactory;
 import org.apache.carbondata.core.datamap.dev.DataMapWriter;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
-import org.apache.carbondata.core.datastore.filesystem.CarbonFileFilter;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.events.ChangeEvent;
 import org.apache.carbondata.core.indexstore.Blocklet;
@@ -78,15 +77,16 @@ public class BlockletDataMapFactory implements DataMapFactory, BlockletDetailsFe
   }
 
   private List<TableBlockIndexUniqueIdentifier> getTableBlockIndexUniqueIdentifiers(
-      String segmentId) {
+      String segmentId) throws IOException {
     List<TableBlockIndexUniqueIdentifier> tableBlockIndexUniqueIdentifiers =
         segmentMap.get(segmentId);
     if (tableBlockIndexUniqueIdentifiers == null) {
       tableBlockIndexUniqueIdentifiers = new ArrayList<>();
-      CarbonFile[] listFiles = getCarbonIndexFiles(segmentId);
-      for (int i = 0; i < listFiles.length; i++) {
+      String path = CarbonTablePath.getSegmentPath(identifier.getTablePath(), segmentId);
+      List<String> indexFiles = new SegmentIndexFileStore().getIndexFilesFromSegment(path);
+      for (int i = 0; i < indexFiles.size(); i++) {
         tableBlockIndexUniqueIdentifiers.add(
-            new TableBlockIndexUniqueIdentifier(identifier, segmentId, listFiles[i].getName()));
+            new TableBlockIndexUniqueIdentifier(identifier, segmentId, indexFiles.get(i)));
       }
       segmentMap.put(segmentId, tableBlockIndexUniqueIdentifiers);
     }
@@ -141,19 +141,10 @@ public class BlockletDataMapFactory implements DataMapFactory, BlockletDetailsFe
     throw new IOException("Blocklet with blockid " + blocklet.getPath() + " not found ");
   }
 
-  private CarbonFile[] getCarbonIndexFiles(String segmentId) {
-    String path = identifier.getTablePath() + "/Fact/Part0/Segment_" + segmentId;
-    CarbonFile carbonFile = FileFactory.getCarbonFile(path);
-    return carbonFile.listFiles(new CarbonFileFilter() {
-      @Override public boolean accept(CarbonFile file) {
-        return file.getName().endsWith(".carbonindex");
-      }
-    });
-  }
 
   @Override
   public List<DataMapDistributable> toDistributable(String segmentId) {
-    CarbonFile[] carbonIndexFiles = getCarbonIndexFiles(segmentId);
+    CarbonFile[] carbonIndexFiles = SegmentIndexFileStore.getCarbonIndexFiles(segmentId);
     List<DataMapDistributable> distributables = new ArrayList<>();
     for (int i = 0; i < carbonIndexFiles.length; i++) {
       Path path = new Path(carbonIndexFiles[i].getPath());
@@ -195,18 +186,30 @@ public class BlockletDataMapFactory implements DataMapFactory, BlockletDetailsFe
   }
 
   @Override
-  public DataMap getDataMap(DataMapDistributable distributable) {
+  public List<DataMap> getDataMaps(DataMapDistributable distributable) throws IOException {
     BlockletDataMapDistributable mapDistributable = (BlockletDataMapDistributable) distributable;
-    TableBlockIndexUniqueIdentifier uniqueIdentifier =
-        new TableBlockIndexUniqueIdentifier(identifier, distributable.getSegmentId(),
-            mapDistributable.getFilePath());
-    DataMap dataMap;
+    List<TableBlockIndexUniqueIdentifier> identifiers = new ArrayList<>();
+    if (mapDistributable.getFilePath().endsWith(CarbonTablePath.INDEX_FILE_EXT)) {
+      identifiers.add(new TableBlockIndexUniqueIdentifier(identifier, distributable.getSegmentId(),
+          mapDistributable.getFilePath()));
+    } else if (mapDistributable.getFilePath().endsWith(CarbonTablePath.MERGE_INDEX_FILE_EXT)) {
+      SegmentIndexFileStore fileStore = new SegmentIndexFileStore();
+      List<String> indexFiles = fileStore.getIndexFilesFromMergeFile(
+          CarbonTablePath.getSegmentPath(identifier.getTablePath(), mapDistributable.getSegmentId())
+              + "/" + mapDistributable.getFilePath());
+      for (String indexFile : indexFiles) {
+        identifiers.add(
+            new TableBlockIndexUniqueIdentifier(identifier, distributable.getSegmentId(),
+                indexFile));
+      }
+    }
+    List<DataMap> dataMaps;
     try {
-      dataMap = cache.get(uniqueIdentifier);
+      dataMaps = cache.getAll(identifiers);
     } catch (IOException e) {
       throw new RuntimeException(e);
     }
-    return dataMap;
+    return dataMaps;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/carbondata/blob/0586146a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapModel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapModel.java b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapModel.java
new file mode 100644
index 0000000..7ffa32d
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapModel.java
@@ -0,0 +1,33 @@
+/*
+ * 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.indexstore.blockletindex;
+
+import org.apache.carbondata.core.datamap.dev.DataMapModel;
+
+public class BlockletDataMapModel extends DataMapModel {
+
+  private byte[] fileData;
+
+  public BlockletDataMapModel(String filePath, byte[] fileData) {
+    super(filePath);
+    this.fileData = fileData;
+  }
+
+  public byte[] getFileData() {
+    return fileData;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/0586146a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/SegmentIndexFileStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/SegmentIndexFileStore.java b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/SegmentIndexFileStore.java
new file mode 100644
index 0000000..22d3d43
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/SegmentIndexFileStore.java
@@ -0,0 +1,189 @@
+/*
+ * 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.indexstore.blockletindex;
+
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.*;
+
+import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFileFilter;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.reader.ThriftReader;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
+import org.apache.carbondata.format.MergedBlockIndex;
+import org.apache.carbondata.format.MergedBlockIndexHeader;
+
+import org.apache.thrift.TBase;
+
+/**
+ * This class manages reading of index files with in the segment. The files it read can be
+ * carbonindex or carbonindexmerge files.
+ */
+public class SegmentIndexFileStore {
+
+  /**
+   * Stores the indexfile name and related binary file data in it.
+   */
+  private Map<String, byte[]> carbonIndexMap;
+
+  public SegmentIndexFileStore() throws IOException {
+    carbonIndexMap = new HashMap<>();
+  }
+
+  /**
+   * Read all index files and keep the cache in it.
+   *
+   * @param segmentPath
+   * @throws IOException
+   */
+  public void readAllIIndexOfSegment(String segmentPath) throws IOException {
+    CarbonFile[] carbonIndexFiles = getCarbonIndexFiles(segmentPath);
+    for (int i = 0; i < carbonIndexFiles.length; i++) {
+      if (carbonIndexFiles[i].getName().endsWith(CarbonTablePath.MERGE_INDEX_FILE_EXT)) {
+        readMergeFile(carbonIndexFiles[i].getCanonicalPath());
+      } else if (carbonIndexFiles[i].getName().endsWith(CarbonTablePath.INDEX_FILE_EXT)) {
+        readIndexFile(carbonIndexFiles[i]);
+      }
+    }
+  }
+
+  /**
+   * Read all index file names of the segment
+   *
+   * @param segmentPath
+   * @return
+   * @throws IOException
+   */
+  public List<String> getIndexFilesFromSegment(String segmentPath) throws IOException {
+    CarbonFile[] carbonIndexFiles = getCarbonIndexFiles(segmentPath);
+    Set<String> indexFiles = new HashSet<>();
+    for (int i = 0; i < carbonIndexFiles.length; i++) {
+      if (carbonIndexFiles[i].getName().endsWith(CarbonTablePath.MERGE_INDEX_FILE_EXT)) {
+        indexFiles.addAll(getIndexFilesFromMergeFile(carbonIndexFiles[i].getCanonicalPath()));
+      } else if (carbonIndexFiles[i].getName().endsWith(CarbonTablePath.INDEX_FILE_EXT)) {
+        indexFiles.add(carbonIndexFiles[i].getName());
+      }
+    }
+    return new ArrayList<>(indexFiles);
+  }
+
+  /**
+   * List all the index files inside merge file.
+   * @param mergeFile
+   * @return
+   * @throws IOException
+   */
+  public List<String> getIndexFilesFromMergeFile(String mergeFile) throws IOException {
+    List<String> indexFiles = new ArrayList<>();
+    ThriftReader thriftReader = new ThriftReader(mergeFile);
+    thriftReader.open();
+    MergedBlockIndexHeader indexHeader = readMergeBlockIndexHeader(thriftReader);
+    List<String> file_names = indexHeader.getFile_names();
+    indexFiles.addAll(file_names);
+    thriftReader.close();
+    return indexFiles;
+  }
+
+  /**
+   * Read carbonindexmerge file and update the map
+   *
+   * @param mergeFilePath
+   * @throws IOException
+   */
+  private void readMergeFile(String mergeFilePath) throws IOException {
+    ThriftReader thriftReader = new ThriftReader(mergeFilePath);
+    thriftReader.open();
+    MergedBlockIndexHeader indexHeader = readMergeBlockIndexHeader(thriftReader);
+    MergedBlockIndex mergedBlockIndex = readMergeBlockIndex(thriftReader);
+    List<String> file_names = indexHeader.getFile_names();
+    List<ByteBuffer> fileData = mergedBlockIndex.getFileData();
+    assert (file_names.size() == fileData.size());
+    for (int i = 0; i < file_names.size(); i++) {
+      carbonIndexMap.put(file_names.get(i), fileData.get(i).array());
+    }
+    thriftReader.close();
+  }
+
+  /**
+   * Read carbonindex file and convert to stream and add to map
+   *
+   * @param indexFile
+   * @throws IOException
+   */
+  private void readIndexFile(CarbonFile indexFile) throws IOException {
+    String indexFilePath = indexFile.getCanonicalPath();
+    DataInputStream dataInputStream =
+        FileFactory.getDataInputStream(indexFilePath, FileFactory.getFileType(indexFilePath));
+    byte[] bytes = new byte[(int) indexFile.getSize()];
+    dataInputStream.readFully(bytes);
+    carbonIndexMap.put(indexFile.getName(), bytes);
+  }
+
+  private MergedBlockIndexHeader readMergeBlockIndexHeader(ThriftReader thriftReader)
+      throws IOException {
+    return (MergedBlockIndexHeader) thriftReader.read(new ThriftReader.TBaseCreator() {
+      @Override public TBase create() {
+        return new MergedBlockIndexHeader();
+      }
+    });
+  }
+
+  private MergedBlockIndex readMergeBlockIndex(ThriftReader thriftReader) throws IOException {
+    return (MergedBlockIndex) thriftReader.read(new ThriftReader.TBaseCreator() {
+      @Override public TBase create() {
+        return new MergedBlockIndex();
+      }
+    });
+  }
+
+  /**
+   * Get the carbonindex file content
+   *
+   * @param fileName
+   * @return
+   */
+  public byte[] getFileData(String fileName) {
+    return carbonIndexMap.get(fileName);
+  }
+
+  /**
+   * List all the index files of the segment.
+   *
+   * @param segmentPath
+   * @return
+   */
+  public static CarbonFile[] getCarbonIndexFiles(String segmentPath) {
+    CarbonFile carbonFile = FileFactory.getCarbonFile(segmentPath);
+    return carbonFile.listFiles(new CarbonFileFilter() {
+      @Override public boolean accept(CarbonFile file) {
+        return file.getName().endsWith(CarbonTablePath.INDEX_FILE_EXT) || file.getName()
+            .endsWith(CarbonTablePath.MERGE_INDEX_FILE_EXT);
+      }
+    });
+  }
+
+  /**
+   * Return the map that contain index file name and content of the file.
+   *
+   * @return
+   */
+  public Map<String, byte[]> getCarbonIndexMap() {
+    return carbonIndexMap;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/0586146a/core/src/main/java/org/apache/carbondata/core/reader/CarbonIndexFileReader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/reader/CarbonIndexFileReader.java b/core/src/main/java/org/apache/carbondata/core/reader/CarbonIndexFileReader.java
index dc46242..4617a12 100644
--- a/core/src/main/java/org/apache/carbondata/core/reader/CarbonIndexFileReader.java
+++ b/core/src/main/java/org/apache/carbondata/core/reader/CarbonIndexFileReader.java
@@ -80,6 +80,16 @@ public class CarbonIndexFileReader {
   }
 
   /**
+   * Open the thrift reader
+   *
+   * @param fileData
+   * @throws IOException
+   */
+  public void openThriftReader(byte[] fileData) throws IOException {
+    thriftReader = new ThriftReader(fileData);
+  }
+
+  /**
    * check if any more object is present
    *
    * @return true if any more object can be read

http://git-wip-us.apache.org/repos/asf/carbondata/blob/0586146a/core/src/main/java/org/apache/carbondata/core/reader/ThriftReader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/reader/ThriftReader.java b/core/src/main/java/org/apache/carbondata/core/reader/ThriftReader.java
index 3e2aab4..221a285 100644
--- a/core/src/main/java/org/apache/carbondata/core/reader/ThriftReader.java
+++ b/core/src/main/java/org/apache/carbondata/core/reader/ThriftReader.java
@@ -17,6 +17,7 @@
 
 package org.apache.carbondata.core.reader;
 
+import java.io.ByteArrayInputStream;
 import java.io.DataInputStream;
 import java.io.IOException;
 
@@ -70,6 +71,14 @@ public class ThriftReader {
   }
 
   /**
+   * Constructor.
+   */
+  public ThriftReader(byte[] fileData) {
+    dataInputStream = new DataInputStream(new ByteArrayInputStream(fileData));
+    binaryIn = new TCompactProtocol(new TIOStreamTransport(dataInputStream));
+  }
+
+  /**
    * Opens the fileName for reading.
    */
   public void open() throws IOException {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/0586146a/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 15f85d6..cba9931 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
@@ -126,13 +126,17 @@ public abstract class AbstractDataFileFooterConverter {
    * @return list of index info
    * @throws IOException problem while reading the index file
    */
-  public List<DataFileFooter> getIndexInfo(String filePath) throws IOException {
+  public List<DataFileFooter> getIndexInfo(String filePath, byte[] fileData) throws IOException {
     CarbonIndexFileReader indexReader = new CarbonIndexFileReader();
     List<DataFileFooter> dataFileFooters = new ArrayList<DataFileFooter>();
     String parentPath = filePath.substring(0, filePath.lastIndexOf("/"));
     try {
       // open the reader
-      indexReader.openThriftReader(filePath);
+      if (fileData != null) {
+        indexReader.openThriftReader(fileData);
+      } else {
+        indexReader.openThriftReader(filePath);
+      }
       // get the index header
       org.apache.carbondata.format.IndexHeader readIndexHeader = indexReader.readIndexHeader();
       List<ColumnSchema> columnSchemaList = new ArrayList<ColumnSchema>();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/0586146a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
index ef3e71d..77789c5 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
@@ -389,7 +389,9 @@ public final class CarbonUtil {
 
   public static void deleteFiles(File[] intermediateFiles) throws IOException {
     for (int i = 0; i < intermediateFiles.length; i++) {
-      if (!intermediateFiles[i].delete()) {
+      // ignore deleting for index file since it is inside merged file.
+      if (!intermediateFiles[i].delete() && !intermediateFiles[i].getName()
+          .endsWith(CarbonTablePath.INDEX_FILE_EXT)) {
         throw new IOException("Problem while deleting intermediate file");
       }
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/0586146a/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java b/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java
index 7be9c76..02a000a 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java
@@ -33,26 +33,28 @@ import org.apache.hadoop.fs.Path;
  */
 public class CarbonTablePath extends Path {
 
-  protected static final String METADATA_DIR = "Metadata";
-  protected static final String DICTIONARY_EXT = ".dict";
-  protected static final String DICTIONARY_META_EXT = ".dictmeta";
-  protected static final String SORT_INDEX_EXT = ".sortindex";
-  protected static final String SCHEMA_FILE = "schema";
-  protected static final String TABLE_STATUS_FILE = "tablestatus";
-  protected static final String TABLE_UPDATE_STATUS_FILE = "tableupdatestatus";
-  protected static final String FACT_DIR = "Fact";
-  protected static final String SEGMENT_PREFIX = "Segment_";
-  protected static final String PARTITION_PREFIX = "Part";
-  protected static final String CARBON_DATA_EXT = ".carbondata";
-  protected static final String CARBON_DELTE_DELTA_EXT = ".deletedelta";
-  protected static final String CARBON_UPDATE_DELTA_EXT = ".updatedelta";
-  protected static final String DATA_PART_PREFIX = "part-";
-  protected static final String BATCH_PREFIX = "_batchno";
-  protected static final String INDEX_FILE_EXT = ".carbonindex";
-  protected static final String DELETE_DELTA_FILE_EXT = ".deletedelta";
-
-  protected String tablePath;
-  protected CarbonTableIdentifier carbonTableIdentifier;
+  private static final String METADATA_DIR = "Metadata";
+  private static final String DICTIONARY_EXT = ".dict";
+  private static final String DICTIONARY_META_EXT = ".dictmeta";
+  private static final String SORT_INDEX_EXT = ".sortindex";
+  private static final String SCHEMA_FILE = "schema";
+  private static final String TABLE_STATUS_FILE = "tablestatus";
+  private static final String TABLE_UPDATE_STATUS_FILE = "tableupdatestatus";
+  private static final String FACT_DIR = "Fact";
+  private static final String SEGMENT_PREFIX = "Segment_";
+  private static final String PARTITION_PREFIX = "Part";
+  private static final String CARBON_DATA_EXT = ".carbondata";
+  private static final String CARBON_DELTE_DELTA_EXT = ".deletedelta";
+  private static final String CARBON_UPDATE_DELTA_EXT = ".updatedelta";
+  private static final String DATA_PART_PREFIX = "part-";
+  private static final String BATCH_PREFIX = "_batchno";
+  private static final String DELETE_DELTA_FILE_EXT = ".deletedelta";
+
+  public static final String INDEX_FILE_EXT = ".carbonindex";
+  public static final String MERGE_INDEX_FILE_EXT = ".carbonindexmerge";
+
+  private String tablePath;
+  private CarbonTableIdentifier carbonTableIdentifier;
 
   /**
    * structure CarbonTablePath object to manage table paths
@@ -746,4 +748,11 @@ public class CarbonTablePath extends Path {
     return DataFileUtil.getTaskNo(actualBlockName) + "-" + DataFileUtil.getBucketNo(actualBlockName)
         + "-" + DataFileUtil.getTimeStampFromFileName(actualBlockName) + INDEX_FILE_EXT;
   }
+
+  /**
+   * Get the segment path from table path and segmentid
+   */
+  public static String getSegmentPath(String tablePath, String segmentId) {
+    return tablePath + "/Fact/Part0/Segment_" + segmentId;
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/0586146a/core/src/main/java/org/apache/carbondata/core/writer/CarbonIndexFileMergeWriter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/writer/CarbonIndexFileMergeWriter.java b/core/src/main/java/org/apache/carbondata/core/writer/CarbonIndexFileMergeWriter.java
new file mode 100644
index 0000000..8d9bddc
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/writer/CarbonIndexFileMergeWriter.java
@@ -0,0 +1,116 @@
+/*
+ * 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.writer;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFileFilter;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.indexstore.blockletindex.SegmentIndexFileStore;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
+import org.apache.carbondata.format.MergedBlockIndex;
+import org.apache.carbondata.format.MergedBlockIndexHeader;
+
+public class CarbonIndexFileMergeWriter {
+
+  /**
+   * thrift writer object
+   */
+  private ThriftWriter thriftWriter;
+
+  /**
+   * Merge all the carbonindex files of segment to a  merged file
+   * @param segmentPath
+   * @throws IOException
+   */
+  public void mergeCarbonIndexFilesOfSegment(String segmentPath) throws IOException {
+    CarbonFile carbonFile = FileFactory.getCarbonFile(segmentPath);
+    CarbonFile[] indexFiles = carbonFile.listFiles(new CarbonFileFilter() {
+      @Override public boolean accept(CarbonFile file) {
+        return file.getName().endsWith(CarbonTablePath.INDEX_FILE_EXT);
+      }
+    });
+    if (indexFiles.length > 0) {
+      SegmentIndexFileStore fileStore = new SegmentIndexFileStore();
+      fileStore.readAllIIndexOfSegment(segmentPath);
+      openThriftWriter(segmentPath + "/" +
+          System.currentTimeMillis() + CarbonTablePath.MERGE_INDEX_FILE_EXT);
+      Map<String, byte[]> indexMap = fileStore.getCarbonIndexMap();
+      MergedBlockIndexHeader indexHeader = new MergedBlockIndexHeader();
+      MergedBlockIndex mergedBlockIndex = new MergedBlockIndex();
+      List<String> fileNames = new ArrayList<>(indexMap.size());
+      List<ByteBuffer> data = new ArrayList<>(indexMap.size());
+      for (Map.Entry<String, byte[]> entry : indexMap.entrySet()) {
+        fileNames.add(entry.getKey());
+        data.add(ByteBuffer.wrap(entry.getValue()));
+      }
+      indexHeader.setFile_names(fileNames);
+      mergedBlockIndex.setFileData(data);
+      writeMergedBlockIndexHeader(indexHeader);
+      writeMergedBlockIndex(mergedBlockIndex);
+      close();
+      for (CarbonFile indexFile : indexFiles) {
+        indexFile.delete();
+      }
+    }
+
+  }
+
+  /**
+   * It writes thrift object to file
+   *
+   * @throws IOException
+   */
+  private void writeMergedBlockIndexHeader(MergedBlockIndexHeader indexObject) throws IOException {
+    thriftWriter.write(indexObject);
+  }
+
+  /**
+   * It writes thrift object to file
+   *
+   * @throws IOException
+   */
+  private void writeMergedBlockIndex(MergedBlockIndex indexObject) throws IOException {
+    thriftWriter.write(indexObject);
+  }
+
+  /**
+   * Below method will be used to open the thrift writer
+   *
+   * @param filePath file path where data need to be written
+   * @throws IOException throws io exception in case of any failure
+   */
+  private void openThriftWriter(String filePath) throws IOException {
+    // create thrift writer instance
+    thriftWriter = new ThriftWriter(filePath, false);
+    // open the file stream
+    thriftWriter.open();
+  }
+
+  /**
+   * Below method will be used to close the thrift object
+   */
+  private void close() throws IOException {
+    thriftWriter.close();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/0586146a/format/src/main/thrift/carbondata_index_merge.thrift
----------------------------------------------------------------------
diff --git a/format/src/main/thrift/carbondata_index_merge.thrift b/format/src/main/thrift/carbondata_index_merge.thrift
new file mode 100644
index 0000000..468d90d
--- /dev/null
+++ b/format/src/main/thrift/carbondata_index_merge.thrift
@@ -0,0 +1,32 @@
+/*
+ * 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.
+ */
+
+/**
+ * File format description for CarbonData merged index file.
+ */
+namespace java org.apache.carbondata.format
+
+struct MergedBlockIndexHeader{
+  1: required list<string> file_names; // list of carbon index file names
+}
+
+/**
+ * Merged carbon index file information
+ */
+struct MergedBlockIndex {
+  1: required list<binary> fileData;	// content of the indexed files
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/0586146a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
index 314ffd5..9fbeb8a 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
@@ -19,6 +19,7 @@ package org.apache.carbondata.hadoop.api;
 
 import java.io.ByteArrayInputStream;
 import java.io.DataInputStream;
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.lang.reflect.Constructor;
 import java.util.ArrayList;
@@ -576,7 +577,10 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
         // for partition table, the task id of carbaondata file name is the partition id.
         // if this partition is not required, here will skip it.
         if (matchedPartitions == null || matchedPartitions.get(partitionIndex)) {
-          resultFilterredBlocks.add(convertToCarbonInputSplit(blocklet));
+          CarbonInputSplit inputSplit = convertToCarbonInputSplit(blocklet);
+          if (inputSplit != null) {
+            resultFilterredBlocks.add(inputSplit);
+          }
         }
       }
     }
@@ -588,7 +592,13 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
 
   private CarbonInputSplit convertToCarbonInputSplit(ExtendedBlocklet blocklet)
       throws IOException {
-    blocklet.updateLocations();
+    try {
+      blocklet.updateLocations();
+    } catch (FileNotFoundException e) {
+      // In case of clean files there is a chance of carbondata file is deleted but index file
+      // exist inside merged file. So just return null.
+      return null;
+    }
     org.apache.carbondata.hadoop.CarbonInputSplit split =
         org.apache.carbondata.hadoop.CarbonInputSplit.from(blocklet.getSegmentId(),
             new FileSplit(new Path(blocklet.getPath()), 0, blocklet.getLength(),

http://git-wip-us.apache.org/repos/asf/carbondata/blob/0586146a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/MergeIndexTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/MergeIndexTestCase.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/MergeIndexTestCase.scala
new file mode 100644
index 0000000..01146ee
--- /dev/null
+++ b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/MergeIndexTestCase.scala
@@ -0,0 +1,122 @@
+
+/*
+ * 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.cluster.sdv.generated
+
+import org.apache.spark.sql.CarbonEnv
+import org.apache.spark.sql.common.util._
+import org.scalatest.BeforeAndAfterAll
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.filesystem.{CarbonFile, CarbonFileFilter}
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
+import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+/**
+ * Test Class for AlterTableTestCase to verify all scenerios
+ */
+
+class MergeIndexTestCase extends QueryTest with BeforeAndAfterAll {
+
+
+  override protected def afterAll(): Unit = {
+    sql("DROP TABLE IF EXISTS nonindexmerge")
+    sql("DROP TABLE IF EXISTS indexmerge")
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_MERGE_INDEX_IN_SEGMENT, "true")
+  }
+
+  test("Verify correctness of index merge sdv") {
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_MERGE_INDEX_IN_SEGMENT, "false")
+    sql(s"""drop table if exists carbon_automation_nonmerge""").collect
+
+    sql(s"""create table carbon_automation_nonmerge (imei string,deviceInformationId int,MAC string,deviceColor string,device_backColor string,modelId string,marketName string,AMSize string,ROMSize string,CUPAudit string,CPIClocked string,series string,productionDate timestamp,bomCode string,internalModels string, deliveryTime string, channelsId string, channelsName string , deliveryAreaId string, deliveryCountry string, deliveryProvince string, deliveryCity string,deliveryDistrict string, deliveryStreet string, oxSingleNumber string, ActiveCheckTime string, ActiveAreaId string, ActiveCountry string, ActiveProvince string, Activecity string, ActiveDistrict string, ActiveStreet string, ActiveOperatorId string, Active_releaseId string, Active_EMUIVersion string, Active_operaSysVersion string, Active_BacVerNumber string, Active_BacFlashVer string, Active_webUIVersion string, Active_webUITypeCarrVer string,Active_webTypeDataVerNumber string, Active_operatorsVersion string, Active_phoneP
 ADPartitionedVersions string, Latest_YEAR int, Latest_MONTH int, Latest_DAY int, Latest_HOUR string, Latest_areaId string, Latest_country string, Latest_province string, Latest_city string, Latest_district string, Latest_street string, Latest_releaseId string, Latest_EMUIVersion string, Latest_operaSysVersion string, Latest_BacVerNumber string, Latest_BacFlashVer string, Latest_webUIVersion string, Latest_webUITypeCarrVer string, Latest_webTypeDataVerNumber string, Latest_operatorsVersion string, Latest_phonePADPartitionedVersions string, Latest_operatorId string, gamePointDescription string,gamePointId double,contractNumber double) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES ('DICTIONARY_INCLUDE'='deviceInformationId,Latest_YEAR,Latest_MONTH,Latest_DAY')""").collect
+
+    sql(s"""LOAD DATA INPATH '$resourcesPath/Data/VmaLL100' INTO TABLE carbon_automation_nonmerge OPTIONS('DELIMITER'=',','QUOTECHAR'='"','FILEHEADER'='imei,deviceInformationId,MAC,deviceColor,device_backColor,modelId,marketName,AMSize,ROMSize,CUPAudit,CPIClocked,series,productionDate,bomCode,internalModels,deliveryTime,channelsId,channelsName,deliveryAreaId,deliveryCountry,deliveryProvince,deliveryCity,deliveryDistrict,deliveryStreet,oxSingleNumber,contractNumber,ActiveCheckTime,ActiveAreaId,ActiveCountry,ActiveProvince,Activecity,ActiveDistrict,ActiveStreet,ActiveOperatorId,Active_releaseId,Active_EMUIVersion,Active_operaSysVersion,Active_BacVerNumber,Active_BacFlashVer,Active_webUIVersion,Active_webUITypeCarrVer,Active_webTypeDataVerNumber,Active_operatorsVersion,Active_phonePADPartitionedVersions,Latest_YEAR,Latest_MONTH,Latest_DAY,Latest_HOUR,Latest_areaId,Latest_country,Latest_province,Latest_city,Latest_district,Latest_street,Latest_releaseId,Latest_EMUIVersion,Latest_operaSy
 sVersion,Latest_BacVerNumber,Latest_BacFlashVer,Latest_webUIVersion,Latest_webUITypeCarrVer,Latest_webTypeDataVerNumber,Latest_operatorsVersion,Latest_phonePADPartitionedVersions,Latest_operatorId,gamePointId,gamePointDescription')""").collect
+    assert(getIndexFileCount("default", "carbon_automation_nonmerge", "0") == 2)
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_MERGE_INDEX_IN_SEGMENT, "true")
+    sql("DROP TABLE IF EXISTS carbon_automation_merge")
+    sql(s"""create table carbon_automation_merge (imei string,deviceInformationId int,MAC string,deviceColor string,device_backColor string,modelId string,marketName string,AMSize string,ROMSize string,CUPAudit string,CPIClocked string,series string,productionDate timestamp,bomCode string,internalModels string, deliveryTime string, channelsId string, channelsName string , deliveryAreaId string, deliveryCountry string, deliveryProvince string, deliveryCity string,deliveryDistrict string, deliveryStreet string, oxSingleNumber string, ActiveCheckTime string, ActiveAreaId string, ActiveCountry string, ActiveProvince string, Activecity string, ActiveDistrict string, ActiveStreet string, ActiveOperatorId string, Active_releaseId string, Active_EMUIVersion string, Active_operaSysVersion string, Active_BacVerNumber string, Active_BacFlashVer string, Active_webUIVersion string, Active_webUITypeCarrVer string,Active_webTypeDataVerNumber string, Active_operatorsVersion string, Active_phonePADP
 artitionedVersions string, Latest_YEAR int, Latest_MONTH int, Latest_DAY int, Latest_HOUR string, Latest_areaId string, Latest_country string, Latest_province string, Latest_city string, Latest_district string, Latest_street string, Latest_releaseId string, Latest_EMUIVersion string, Latest_operaSysVersion string, Latest_BacVerNumber string, Latest_BacFlashVer string, Latest_webUIVersion string, Latest_webUITypeCarrVer string, Latest_webTypeDataVerNumber string, Latest_operatorsVersion string, Latest_phonePADPartitionedVersions string, Latest_operatorId string, gamePointDescription string,gamePointId double,contractNumber double) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES ('DICTIONARY_INCLUDE'='deviceInformationId,Latest_YEAR,Latest_MONTH,Latest_DAY')""").collect
+
+    sql(s"""LOAD DATA INPATH '$resourcesPath/Data/VmaLL100' INTO TABLE carbon_automation_merge OPTIONS('DELIMITER'=',','QUOTECHAR'='"','FILEHEADER'='imei,deviceInformationId,MAC,deviceColor,device_backColor,modelId,marketName,AMSize,ROMSize,CUPAudit,CPIClocked,series,productionDate,bomCode,internalModels,deliveryTime,channelsId,channelsName,deliveryAreaId,deliveryCountry,deliveryProvince,deliveryCity,deliveryDistrict,deliveryStreet,oxSingleNumber,contractNumber,ActiveCheckTime,ActiveAreaId,ActiveCountry,ActiveProvince,Activecity,ActiveDistrict,ActiveStreet,ActiveOperatorId,Active_releaseId,Active_EMUIVersion,Active_operaSysVersion,Active_BacVerNumber,Active_BacFlashVer,Active_webUIVersion,Active_webUITypeCarrVer,Active_webTypeDataVerNumber,Active_operatorsVersion,Active_phonePADPartitionedVersions,Latest_YEAR,Latest_MONTH,Latest_DAY,Latest_HOUR,Latest_areaId,Latest_country,Latest_province,Latest_city,Latest_district,Latest_street,Latest_releaseId,Latest_EMUIVersion,Latest_operaSysVe
 rsion,Latest_BacVerNumber,Latest_BacFlashVer,Latest_webUIVersion,Latest_webUITypeCarrVer,Latest_webTypeDataVerNumber,Latest_operatorsVersion,Latest_phonePADPartitionedVersions,Latest_operatorId,gamePointId,gamePointDescription')""").collect
+
+    assert(getIndexFileCount("default", "carbon_automation_merge", "0") == 0)
+    checkAnswer(sql("""Select count(*) from carbon_automation_nonmerge"""),
+      sql("""Select count(*) from carbon_automation_merge"""))
+  }
+
+  test("Verify command of index merge  sdv") {
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_MERGE_INDEX_IN_SEGMENT, "false")
+    sql(s"""drop table if exists carbon_automation_nonmerge""").collect
+
+    sql(s"""create table carbon_automation_nonmerge (imei string,deviceInformationId int,MAC string,deviceColor string,device_backColor string,modelId string,marketName string,AMSize string,ROMSize string,CUPAudit string,CPIClocked string,series string,productionDate timestamp,bomCode string,internalModels string, deliveryTime string, channelsId string, channelsName string , deliveryAreaId string, deliveryCountry string, deliveryProvince string, deliveryCity string,deliveryDistrict string, deliveryStreet string, oxSingleNumber string, ActiveCheckTime string, ActiveAreaId string, ActiveCountry string, ActiveProvince string, Activecity string, ActiveDistrict string, ActiveStreet string, ActiveOperatorId string, Active_releaseId string, Active_EMUIVersion string, Active_operaSysVersion string, Active_BacVerNumber string, Active_BacFlashVer string, Active_webUIVersion string, Active_webUITypeCarrVer string,Active_webTypeDataVerNumber string, Active_operatorsVersion string, Active_phoneP
 ADPartitionedVersions string, Latest_YEAR int, Latest_MONTH int, Latest_DAY int, Latest_HOUR string, Latest_areaId string, Latest_country string, Latest_province string, Latest_city string, Latest_district string, Latest_street string, Latest_releaseId string, Latest_EMUIVersion string, Latest_operaSysVersion string, Latest_BacVerNumber string, Latest_BacFlashVer string, Latest_webUIVersion string, Latest_webUITypeCarrVer string, Latest_webTypeDataVerNumber string, Latest_operatorsVersion string, Latest_phonePADPartitionedVersions string, Latest_operatorId string, gamePointDescription string,gamePointId double,contractNumber double) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES ('DICTIONARY_INCLUDE'='deviceInformationId,Latest_YEAR,Latest_MONTH,Latest_DAY')""").collect
+
+    sql(s"""LOAD DATA INPATH '$resourcesPath/Data/VmaLL100' INTO TABLE carbon_automation_nonmerge OPTIONS('DELIMITER'=',','QUOTECHAR'='"','FILEHEADER'='imei,deviceInformationId,MAC,deviceColor,device_backColor,modelId,marketName,AMSize,ROMSize,CUPAudit,CPIClocked,series,productionDate,bomCode,internalModels,deliveryTime,channelsId,channelsName,deliveryAreaId,deliveryCountry,deliveryProvince,deliveryCity,deliveryDistrict,deliveryStreet,oxSingleNumber,contractNumber,ActiveCheckTime,ActiveAreaId,ActiveCountry,ActiveProvince,Activecity,ActiveDistrict,ActiveStreet,ActiveOperatorId,Active_releaseId,Active_EMUIVersion,Active_operaSysVersion,Active_BacVerNumber,Active_BacFlashVer,Active_webUIVersion,Active_webUITypeCarrVer,Active_webTypeDataVerNumber,Active_operatorsVersion,Active_phonePADPartitionedVersions,Latest_YEAR,Latest_MONTH,Latest_DAY,Latest_HOUR,Latest_areaId,Latest_country,Latest_province,Latest_city,Latest_district,Latest_street,Latest_releaseId,Latest_EMUIVersion,Latest_operaSy
 sVersion,Latest_BacVerNumber,Latest_BacFlashVer,Latest_webUIVersion,Latest_webUITypeCarrVer,Latest_webTypeDataVerNumber,Latest_operatorsVersion,Latest_phonePADPartitionedVersions,Latest_operatorId,gamePointId,gamePointDescription')""").collect
+    sql(s"""LOAD DATA INPATH '$resourcesPath/Data/VmaLL100' INTO TABLE carbon_automation_nonmerge OPTIONS('DELIMITER'=',','QUOTECHAR'='"','FILEHEADER'='imei,deviceInformationId,MAC,deviceColor,device_backColor,modelId,marketName,AMSize,ROMSize,CUPAudit,CPIClocked,series,productionDate,bomCode,internalModels,deliveryTime,channelsId,channelsName,deliveryAreaId,deliveryCountry,deliveryProvince,deliveryCity,deliveryDistrict,deliveryStreet,oxSingleNumber,contractNumber,ActiveCheckTime,ActiveAreaId,ActiveCountry,ActiveProvince,Activecity,ActiveDistrict,ActiveStreet,ActiveOperatorId,Active_releaseId,Active_EMUIVersion,Active_operaSysVersion,Active_BacVerNumber,Active_BacFlashVer,Active_webUIVersion,Active_webUITypeCarrVer,Active_webTypeDataVerNumber,Active_operatorsVersion,Active_phonePADPartitionedVersions,Latest_YEAR,Latest_MONTH,Latest_DAY,Latest_HOUR,Latest_areaId,Latest_country,Latest_province,Latest_city,Latest_district,Latest_street,Latest_releaseId,Latest_EMUIVersion,Latest_operaSy
 sVersion,Latest_BacVerNumber,Latest_BacFlashVer,Latest_webUIVersion,Latest_webUITypeCarrVer,Latest_webTypeDataVerNumber,Latest_operatorsVersion,Latest_phonePADPartitionedVersions,Latest_operatorId,gamePointId,gamePointDescription')""").collect
+    val rows = sql("""Select count(*) from carbon_automation_nonmerge""").collect()
+    assert(getIndexFileCount("default", "carbon_automation_nonmerge", "0") == 2)
+    assert(getIndexFileCount("default", "carbon_automation_nonmerge", "1") == 2)
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_MERGE_INDEX_IN_SEGMENT, "true")
+    sql("ALTER TABLE carbon_automation_nonmerge COMPACT 'SEGMENT_INDEX_COMPACTION'").collect()
+    assert(getIndexFileCount("default", "carbon_automation_nonmerge", "0") == 0)
+    assert(getIndexFileCount("default", "carbon_automation_nonmerge", "1") == 0)
+    checkAnswer(sql("""Select count(*) from carbon_automation_nonmerge"""), rows)
+  }
+
+  test("Verify index index merge with compaction  sdv") {
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_MERGE_INDEX_IN_SEGMENT, "false")
+    sql(s"""drop table if exists carbon_automation_nonmerge""").collect
+
+    sql(s"""create table carbon_automation_nonmerge (imei string,deviceInformationId int,MAC string,deviceColor string,device_backColor string,modelId string,marketName string,AMSize string,ROMSize string,CUPAudit string,CPIClocked string,series string,productionDate timestamp,bomCode string,internalModels string, deliveryTime string, channelsId string, channelsName string , deliveryAreaId string, deliveryCountry string, deliveryProvince string, deliveryCity string,deliveryDistrict string, deliveryStreet string, oxSingleNumber string, ActiveCheckTime string, ActiveAreaId string, ActiveCountry string, ActiveProvince string, Activecity string, ActiveDistrict string, ActiveStreet string, ActiveOperatorId string, Active_releaseId string, Active_EMUIVersion string, Active_operaSysVersion string, Active_BacVerNumber string, Active_BacFlashVer string, Active_webUIVersion string, Active_webUITypeCarrVer string,Active_webTypeDataVerNumber string, Active_operatorsVersion string, Active_phoneP
 ADPartitionedVersions string, Latest_YEAR int, Latest_MONTH int, Latest_DAY int, Latest_HOUR string, Latest_areaId string, Latest_country string, Latest_province string, Latest_city string, Latest_district string, Latest_street string, Latest_releaseId string, Latest_EMUIVersion string, Latest_operaSysVersion string, Latest_BacVerNumber string, Latest_BacFlashVer string, Latest_webUIVersion string, Latest_webUITypeCarrVer string, Latest_webTypeDataVerNumber string, Latest_operatorsVersion string, Latest_phonePADPartitionedVersions string, Latest_operatorId string, gamePointDescription string,gamePointId double,contractNumber double) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES ('DICTIONARY_INCLUDE'='deviceInformationId,Latest_YEAR,Latest_MONTH,Latest_DAY')""").collect
+
+    sql(s"""LOAD DATA INPATH '$resourcesPath/Data/VmaLL100' INTO TABLE carbon_automation_nonmerge OPTIONS('DELIMITER'=',','QUOTECHAR'='"','FILEHEADER'='imei,deviceInformationId,MAC,deviceColor,device_backColor,modelId,marketName,AMSize,ROMSize,CUPAudit,CPIClocked,series,productionDate,bomCode,internalModels,deliveryTime,channelsId,channelsName,deliveryAreaId,deliveryCountry,deliveryProvince,deliveryCity,deliveryDistrict,deliveryStreet,oxSingleNumber,contractNumber,ActiveCheckTime,ActiveAreaId,ActiveCountry,ActiveProvince,Activecity,ActiveDistrict,ActiveStreet,ActiveOperatorId,Active_releaseId,Active_EMUIVersion,Active_operaSysVersion,Active_BacVerNumber,Active_BacFlashVer,Active_webUIVersion,Active_webUITypeCarrVer,Active_webTypeDataVerNumber,Active_operatorsVersion,Active_phonePADPartitionedVersions,Latest_YEAR,Latest_MONTH,Latest_DAY,Latest_HOUR,Latest_areaId,Latest_country,Latest_province,Latest_city,Latest_district,Latest_street,Latest_releaseId,Latest_EMUIVersion,Latest_operaSy
 sVersion,Latest_BacVerNumber,Latest_BacFlashVer,Latest_webUIVersion,Latest_webUITypeCarrVer,Latest_webTypeDataVerNumber,Latest_operatorsVersion,Latest_phonePADPartitionedVersions,Latest_operatorId,gamePointId,gamePointDescription')""").collect
+    sql(s"""LOAD DATA INPATH '$resourcesPath/Data/VmaLL100' INTO TABLE carbon_automation_nonmerge OPTIONS('DELIMITER'=',','QUOTECHAR'='"','FILEHEADER'='imei,deviceInformationId,MAC,deviceColor,device_backColor,modelId,marketName,AMSize,ROMSize,CUPAudit,CPIClocked,series,productionDate,bomCode,internalModels,deliveryTime,channelsId,channelsName,deliveryAreaId,deliveryCountry,deliveryProvince,deliveryCity,deliveryDistrict,deliveryStreet,oxSingleNumber,contractNumber,ActiveCheckTime,ActiveAreaId,ActiveCountry,ActiveProvince,Activecity,ActiveDistrict,ActiveStreet,ActiveOperatorId,Active_releaseId,Active_EMUIVersion,Active_operaSysVersion,Active_BacVerNumber,Active_BacFlashVer,Active_webUIVersion,Active_webUITypeCarrVer,Active_webTypeDataVerNumber,Active_operatorsVersion,Active_phonePADPartitionedVersions,Latest_YEAR,Latest_MONTH,Latest_DAY,Latest_HOUR,Latest_areaId,Latest_country,Latest_province,Latest_city,Latest_district,Latest_street,Latest_releaseId,Latest_EMUIVersion,Latest_operaSy
 sVersion,Latest_BacVerNumber,Latest_BacFlashVer,Latest_webUIVersion,Latest_webUITypeCarrVer,Latest_webTypeDataVerNumber,Latest_operatorsVersion,Latest_phonePADPartitionedVersions,Latest_operatorId,gamePointId,gamePointDescription')""").collect
+    sql(s"""LOAD DATA INPATH '$resourcesPath/Data/VmaLL100' INTO TABLE carbon_automation_nonmerge OPTIONS('DELIMITER'=',','QUOTECHAR'='"','FILEHEADER'='imei,deviceInformationId,MAC,deviceColor,device_backColor,modelId,marketName,AMSize,ROMSize,CUPAudit,CPIClocked,series,productionDate,bomCode,internalModels,deliveryTime,channelsId,channelsName,deliveryAreaId,deliveryCountry,deliveryProvince,deliveryCity,deliveryDistrict,deliveryStreet,oxSingleNumber,contractNumber,ActiveCheckTime,ActiveAreaId,ActiveCountry,ActiveProvince,Activecity,ActiveDistrict,ActiveStreet,ActiveOperatorId,Active_releaseId,Active_EMUIVersion,Active_operaSysVersion,Active_BacVerNumber,Active_BacFlashVer,Active_webUIVersion,Active_webUITypeCarrVer,Active_webTypeDataVerNumber,Active_operatorsVersion,Active_phonePADPartitionedVersions,Latest_YEAR,Latest_MONTH,Latest_DAY,Latest_HOUR,Latest_areaId,Latest_country,Latest_province,Latest_city,Latest_district,Latest_street,Latest_releaseId,Latest_EMUIVersion,Latest_operaSy
 sVersion,Latest_BacVerNumber,Latest_BacFlashVer,Latest_webUIVersion,Latest_webUITypeCarrVer,Latest_webTypeDataVerNumber,Latest_operatorsVersion,Latest_phonePADPartitionedVersions,Latest_operatorId,gamePointId,gamePointDescription')""").collect
+    val rows = sql("""Select count(*) from carbon_automation_nonmerge""").collect()
+    assert(getIndexFileCount("default", "carbon_automation_nonmerge", "0") == 2)
+    assert(getIndexFileCount("default", "carbon_automation_nonmerge", "1") == 2)
+    assert(getIndexFileCount("default", "carbon_automation_nonmerge", "1") == 2)
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_MERGE_INDEX_IN_SEGMENT, "true")
+    sql("ALTER TABLE carbon_automation_nonmerge COMPACT 'minor'").collect()
+    assert(getIndexFileCount("default", "carbon_automation_nonmerge", "0.1") == 0)
+    checkAnswer(sql("""Select count(*) from carbon_automation_nonmerge"""), rows)
+  }
+
+  private def getIndexFileCount(dbName: String, tableName: String, segment: String): Int = {
+    val identifier = AbsoluteTableIdentifier.from(CarbonEnv.getInstance(sqlContext.sparkSession).storePath, dbName, tableName)
+    val path = CarbonTablePath
+      .getSegmentPath(identifier.getTablePath, segment)
+    val carbonFiles = FileFactory.getCarbonFile(path).listFiles(new CarbonFileFilter {
+      override def accept(file: CarbonFile): Boolean = file.getName.endsWith(CarbonTablePath
+        .INDEX_FILE_EXT)
+    })
+    if (carbonFiles != null) {
+      carbonFiles.length
+    } else {
+      0
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/0586146a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/suite/SDVSuites.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/suite/SDVSuites.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/suite/SDVSuites.scala
index 049a460..2e00b0c 100644
--- a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/suite/SDVSuites.scala
+++ b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/suite/SDVSuites.scala
@@ -78,7 +78,8 @@ class SDVSuites1 extends Suites with BeforeAndAfterAll {
                    new PartitionTestCase ::
                    new QueriesBasicTestCase ::
                    new BatchSortLoad3TestCase ::
-                   new GlobalSortTestCase :: Nil
+                   new GlobalSortTestCase ::
+                   new MergeIndexTestCase :: Nil
 
   override val nestedSuites = suites.toIndexedSeq
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/0586146a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CarbonIndexFileMergeTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CarbonIndexFileMergeTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CarbonIndexFileMergeTestCase.scala
new file mode 100644
index 0000000..110557c
--- /dev/null
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CarbonIndexFileMergeTestCase.scala
@@ -0,0 +1,143 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.spark.testsuite.datacompaction
+
+import org.apache.spark.sql.test.util.QueryTest
+import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach}
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.filesystem.{CarbonFile, CarbonFileFilter}
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.metadata.CarbonMetadata
+import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+class CarbonIndexFileMergeTestCase
+  extends QueryTest with BeforeAndAfterEach with BeforeAndAfterAll {
+  val file2 = resourcesPath + "/compaction/fil2.csv"
+
+  override protected def beforeAll(): Unit = {
+    val n = 150000
+    CompactionSupportGlobalSortBigFileTest.createFile(file2, n * 4, n)
+  }
+
+  override protected def afterAll(): Unit = {
+    CompactionSupportGlobalSortBigFileTest.deleteFile(file2)
+    sql("DROP TABLE IF EXISTS nonindexmerge")
+    sql("DROP TABLE IF EXISTS indexmerge")
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_MERGE_INDEX_IN_SEGMENT, "true")
+  }
+
+  test("Verify correctness of index merge") {
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_MERGE_INDEX_IN_SEGMENT, "false")
+    sql("DROP TABLE IF EXISTS nonindexmerge")
+    sql(
+      """
+        | CREATE TABLE nonindexmerge(id INT, name STRING, city STRING, age INT)
+        | STORED BY 'org.apache.carbondata.format'
+        | TBLPROPERTIES('SORT_COLUMNS'='city,name', 'SORT_SCOPE'='GLOBAL_SORT')
+      """.stripMargin)
+    sql(s"LOAD DATA LOCAL INPATH '$file2' INTO TABLE nonindexmerge OPTIONS('header'='false', " +
+        s"'GLOBAL_SORT_PARTITIONS'='100')")
+    assert(getIndexFileCount("default_nonindexmerge", "0") == 100)
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_MERGE_INDEX_IN_SEGMENT, "true")
+    sql("DROP TABLE IF EXISTS indexmerge")
+    sql(
+      """
+        | CREATE TABLE indexmerge(id INT, name STRING, city STRING, age INT)
+        | STORED BY 'org.apache.carbondata.format'
+        |  TBLPROPERTIES('SORT_COLUMNS'='city,name', 'SORT_SCOPE'='GLOBAL_SORT')
+      """.stripMargin)
+    sql(s"LOAD DATA LOCAL INPATH '$file2' INTO TABLE indexmerge OPTIONS('header'='false', " +
+        s"'GLOBAL_SORT_PARTITIONS'='100')")
+    assert(getIndexFileCount("default_indexmerge", "0") == 0)
+    checkAnswer(sql("""Select count(*) from nonindexmerge"""),
+      sql("""Select count(*) from indexmerge"""))
+  }
+
+  test("Verify command of index merge") {
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_MERGE_INDEX_IN_SEGMENT, "false")
+    sql("DROP TABLE IF EXISTS nonindexmerge")
+    sql(
+      """
+        | CREATE TABLE nonindexmerge(id INT, name STRING, city STRING, age INT)
+        | STORED BY 'org.apache.carbondata.format'
+        | TBLPROPERTIES('SORT_COLUMNS'='city,name', 'SORT_SCOPE'='GLOBAL_SORT')
+      """.stripMargin)
+    sql(s"LOAD DATA LOCAL INPATH '$file2' INTO TABLE nonindexmerge OPTIONS('header'='false', " +
+        s"'GLOBAL_SORT_PARTITIONS'='100')")
+    sql(s"LOAD DATA LOCAL INPATH '$file2' INTO TABLE nonindexmerge OPTIONS('header'='false', " +
+        s"'GLOBAL_SORT_PARTITIONS'='100')")
+    val rows = sql("""Select count(*) from nonindexmerge""").collect()
+    assert(getIndexFileCount("default_nonindexmerge", "0") == 100)
+    assert(getIndexFileCount("default_nonindexmerge", "1") == 100)
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_MERGE_INDEX_IN_SEGMENT, "true")
+    sql("ALTER TABLE nonindexmerge COMPACT 'SEGMENT_INDEX_COMPACTION'").collect()
+    assert(getIndexFileCount("default_nonindexmerge", "0") == 0)
+    assert(getIndexFileCount("default_nonindexmerge", "1") == 0)
+    checkAnswer(sql("""Select count(*) from nonindexmerge"""), rows)
+  }
+
+  test("Verify index index merge with compaction") {
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_MERGE_INDEX_IN_SEGMENT, "false")
+    sql("DROP TABLE IF EXISTS nonindexmerge")
+    sql(
+      """
+        | CREATE TABLE nonindexmerge(id INT, name STRING, city STRING, age INT)
+        | STORED BY 'org.apache.carbondata.format'
+        | TBLPROPERTIES('SORT_COLUMNS'='city,name', 'SORT_SCOPE'='GLOBAL_SORT')
+      """.stripMargin)
+    sql(s"LOAD DATA LOCAL INPATH '$file2' INTO TABLE nonindexmerge OPTIONS('header'='false', " +
+        s"'GLOBAL_SORT_PARTITIONS'='100')")
+    sql(s"LOAD DATA LOCAL INPATH '$file2' INTO TABLE nonindexmerge OPTIONS('header'='false', " +
+        s"'GLOBAL_SORT_PARTITIONS'='100')")
+    sql(s"LOAD DATA LOCAL INPATH '$file2' INTO TABLE nonindexmerge OPTIONS('header'='false', " +
+        s"'GLOBAL_SORT_PARTITIONS'='100')")
+    val rows = sql("""Select count(*) from nonindexmerge""").collect()
+    assert(getIndexFileCount("default_nonindexmerge", "0") == 100)
+    assert(getIndexFileCount("default_nonindexmerge", "1") == 100)
+    assert(getIndexFileCount("default_nonindexmerge", "1") == 100)
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_MERGE_INDEX_IN_SEGMENT, "true")
+    sql("ALTER TABLE nonindexmerge COMPACT 'minor'").collect()
+    assert(getIndexFileCount("default_nonindexmerge", "0.1") == 0)
+    checkAnswer(sql("""Select count(*) from nonindexmerge"""), rows)
+  }
+
+  private def getIndexFileCount(tableName: String, segment: String): Int = {
+    val table = CarbonMetadata.getInstance().getCarbonTable(tableName)
+    val path = CarbonTablePath
+      .getSegmentPath(table.getAbsoluteTableIdentifier.getTablePath, segment)
+    val carbonFiles = FileFactory.getCarbonFile(path).listFiles(new CarbonFileFilter {
+      override def accept(file: CarbonFile): Boolean = file.getName.endsWith(CarbonTablePath
+        .INDEX_FILE_EXT)
+    })
+    if (carbonFiles != null) {
+      carbonFiles.length
+    } else {
+      0
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/0586146a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortFunctionTest.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortFunctionTest.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortFunctionTest.scala
index 6f8648d..c7b39ad 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortFunctionTest.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortFunctionTest.scala
@@ -24,6 +24,7 @@ import org.apache.spark.sql.test.util.QueryTest
 import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach}
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.indexstore.blockletindex.SegmentIndexFileStore
 import org.apache.carbondata.core.util.CarbonProperties
 
 class CompactionSupportGlobalSortFunctionTest extends QueryTest with BeforeAndAfterEach with BeforeAndAfterAll {
@@ -527,9 +528,6 @@ class CompactionSupportGlobalSortFunctionTest extends QueryTest with BeforeAndAf
 
   private def getIndexFileCount(tableName: String, segmentNo: String = "0"): Int = {
     val store = storeLocation + "/default/" + tableName + "/Fact/Part0/Segment_" + segmentNo
-    val list = new File(store).list(new FilenameFilter {
-      override def accept(dir: File, name: String) = name.endsWith(".carbonindex")
-    })
-    list.size
+    new SegmentIndexFileStore().getIndexFilesFromSegment(store).size()
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/0586146a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortParameterTest.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortParameterTest.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortParameterTest.scala
index 1511b51..78c835a 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortParameterTest.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortParameterTest.scala
@@ -25,6 +25,9 @@ import org.apache.spark.sql.Row
 import org.apache.spark.sql.test.util.QueryTest
 import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach}
 
+import org.apache.carbondata.core.indexstore.blockletindex.SegmentIndexFileStore
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
 class CompactionSupportGlobalSortParameterTest extends QueryTest with BeforeAndAfterEach with BeforeAndAfterAll {
   val filePath: String = s"$resourcesPath/globalsort"
   val file1: String = resourcesPath + "/globalsort/sample1.csv"
@@ -526,9 +529,6 @@ class CompactionSupportGlobalSortParameterTest extends QueryTest with BeforeAndA
 
   private def getIndexFileCount(tableName: String, segmentNo: String = "0"): Int = {
     val store = storeLocation + "/default/" + tableName + "/Fact/Part0/Segment_" + segmentNo
-    val list = new File(store).list(new FilenameFilter {
-      override def accept(dir: File, name: String) = name.endsWith(".carbonindex")
-    })
-    list.size
+    new SegmentIndexFileStore().getIndexFilesFromSegment(store).size()
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/0586146a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestBatchSortDataLoad.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestBatchSortDataLoad.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestBatchSortDataLoad.scala
index 4c6c56d..cb3da40 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestBatchSortDataLoad.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestBatchSortDataLoad.scala
@@ -21,10 +21,13 @@ import java.io.{BufferedWriter, File, FileWriter, FilenameFilter}
 
 import org.apache.spark.sql.Row
 import org.scalatest.BeforeAndAfterAll
+
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.util.CarbonProperties
 import org.apache.spark.sql.test.util.QueryTest
 
+import org.apache.carbondata.core.indexstore.blockletindex.SegmentIndexFileStore
+
 class TestBatchSortDataLoad extends QueryTest with BeforeAndAfterAll {
   var filePath: String = _
 
@@ -184,10 +187,7 @@ class TestBatchSortDataLoad extends QueryTest with BeforeAndAfterAll {
 
   def getIndexfileCount(tableName: String, segmentNo: String = "0"): Int = {
     val store  = storeLocation +"/default/"+ tableName + "/Fact/Part0/Segment_"+segmentNo
-    val list = new File(store).list(new FilenameFilter {
-      override def accept(dir: File, name: String) = name.endsWith(".carbonindex")
-    })
-    list.size
+    new SegmentIndexFileStore().getIndexFilesFromSegment(store).size()
   }
 
   override def afterAll {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/0586146a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestGlobalSortDataLoad.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestGlobalSortDataLoad.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestGlobalSortDataLoad.scala
index 35a56ea..5d0c055 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestGlobalSortDataLoad.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestGlobalSortDataLoad.scala
@@ -27,6 +27,8 @@ import org.apache.spark.sql.test.TestQueryExecutor.projectPath
 import org.apache.spark.sql.test.util.QueryTest
 import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach}
 
+import org.apache.carbondata.core.indexstore.blockletindex.SegmentIndexFileStore
+
 class TestGlobalSortDataLoad extends QueryTest with BeforeAndAfterEach with BeforeAndAfterAll {
   var filePath: String = s"$resourcesPath/globalsort"
 
@@ -325,9 +327,6 @@ class TestGlobalSortDataLoad extends QueryTest with BeforeAndAfterEach with Befo
 
   private def getIndexFileCount(tableName: String, segmentNo: String = "0"): Int = {
     val store  = storeLocation + "/default/" + tableName + "/Fact/Part0/Segment_" + segmentNo
-    val list = new File(store).list(new FilenameFilter {
-      override def accept(dir: File, name: String) = name.endsWith(".carbonindex")
-    })
-    list.size
+    new SegmentIndexFileStore().getIndexFilesFromSegment(store).size()
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/0586146a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala
index ba6ad31..ff900ce 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala
@@ -45,7 +45,7 @@ class C2DataMapFactory() extends DataMapFactory {
 
   override def clear(): Unit = {}
 
-  override def getDataMap(distributable: DataMapDistributable): DataMap = ???
+  override def getDataMaps(distributable: DataMapDistributable): java.util.List[DataMap] = ???
 
   override def getDataMaps(segmentId: String): util.List[DataMap] = ???
 


[24/49] carbondata git commit: [CARBONDATA-1572][Streaming] Support streaming ingest and query

Posted by ra...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/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
new file mode 100644
index 0000000..b10bc8b
--- /dev/null
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamInputFormat.java
@@ -0,0 +1,114 @@
+/*
+ * 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.hadoop.streaming;
+
+import java.io.IOException;
+
+import org.apache.carbondata.core.cache.Cache;
+import org.apache.carbondata.core.cache.dictionary.Dictionary;
+import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
+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.CarbonColumn;
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
+import org.apache.carbondata.core.scan.complextypes.ArrayQueryType;
+import org.apache.carbondata.core.scan.complextypes.PrimitiveQueryType;
+import org.apache.carbondata.core.scan.complextypes.StructQueryType;
+import org.apache.carbondata.core.scan.filter.GenericQueryType;
+import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.core.util.path.CarbonStorePath;
+
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+
+/**
+ * Stream input format
+ */
+public class CarbonStreamInputFormat extends FileInputFormat<Void, Object> {
+
+  public static final String READ_BUFFER_SIZE = "carbon.stream.read.buffer.size";
+  public static final String READ_BUFFER_SIZE_DEFAULT = "65536";
+
+  @Override public RecordReader<Void, Object> createRecordReader(InputSplit split,
+      TaskAttemptContext context) throws IOException, InterruptedException {
+    return new CarbonStreamRecordReader();
+  }
+
+  public static GenericQueryType[] getComplexDimensions(CarbonTable carbontable,
+      CarbonColumn[] carbonColumns, Cache<DictionaryColumnUniqueIdentifier, Dictionary> cache)
+      throws IOException {
+    GenericQueryType[] queryTypes = new GenericQueryType[carbonColumns.length];
+    for (int i = 0; i < carbonColumns.length; i++) {
+      if (carbonColumns[i].isComplex()) {
+        if (carbonColumns[i].getDataType() == DataTypes.ARRAY) {
+          queryTypes[i] = new ArrayQueryType(carbonColumns[i].getColName(),
+              carbonColumns[i].getColName(), i);
+        } else if (carbonColumns[i].getDataType() == DataTypes.STRUCT) {
+          queryTypes[i] = new StructQueryType(carbonColumns[i].getColName(),
+              carbonColumns[i].getColName(), i);
+        } else {
+          throw new UnsupportedOperationException(
+              carbonColumns[i].getDataType().getName() + " is not supported");
+        }
+
+        fillChildren(carbontable, queryTypes[i], (CarbonDimension) carbonColumns[i], i, cache);
+      }
+    }
+
+    return queryTypes;
+  }
+
+  private static void fillChildren(CarbonTable carbontable, GenericQueryType parentQueryType,
+      CarbonDimension dimension, int parentBlockIndex,
+      Cache<DictionaryColumnUniqueIdentifier, Dictionary> cache) throws IOException {
+    for (int i = 0; i < dimension.getNumberOfChild(); i++) {
+      CarbonDimension child = dimension.getListOfChildDimensions().get(i);
+      DataType dataType = child.getDataType();
+      GenericQueryType queryType = null;
+      if (dataType == DataTypes.ARRAY) {
+        queryType =
+            new ArrayQueryType(child.getColName(), dimension.getColName(), ++parentBlockIndex);
+
+      } else if (dataType == DataTypes.STRUCT) {
+        queryType =
+            new StructQueryType(child.getColName(), dimension.getColName(), ++parentBlockIndex);
+        parentQueryType.addChildren(queryType);
+      } else {
+        boolean isDirectDictionary =
+            CarbonUtil.hasEncoding(child.getEncoder(), Encoding.DIRECT_DICTIONARY);
+        DictionaryColumnUniqueIdentifier dictionarIdentifier =
+            new DictionaryColumnUniqueIdentifier(carbontable.getCarbonTableIdentifier(),
+                child.getColumnIdentifier(), child.getDataType(),
+                CarbonStorePath.getCarbonTablePath(carbontable.getAbsoluteTableIdentifier()));
+
+        queryType =
+            new PrimitiveQueryType(child.getColName(), dimension.getColName(), ++parentBlockIndex,
+                child.getDataType(), 4, cache.get(dictionarIdentifier),
+                isDirectDictionary);
+      }
+      parentQueryType.addChildren(queryType);
+      if (child.getNumberOfChild() > 0) {
+        fillChildren(carbontable, queryType, child, parentBlockIndex, cache);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamOutputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamOutputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamOutputFormat.java
new file mode 100644
index 0000000..1c21504
--- /dev/null
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamOutputFormat.java
@@ -0,0 +1,75 @@
+/*
+ * 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.hadoop.streaming;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.hadoop.util.ObjectSerializationUtil;
+import org.apache.carbondata.processing.loading.model.CarbonLoadModel;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+
+/**
+ * Stream output format
+ */
+public class CarbonStreamOutputFormat extends FileOutputFormat<Void, Object> {
+
+  static final byte[] CARBON_SYNC_MARKER =
+      "@carbondata_sync".getBytes(Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
+
+  public static final String CARBON_ENCODER_ROW_BUFFER_SIZE = "carbon.stream.row.buffer.size";
+
+  public static final int CARBON_ENCODER_ROW_BUFFER_SIZE_DEFAULT = 1024;
+
+  public static final String CARBON_STREAM_BLOCKLET_ROW_NUMS = "carbon.stream.blocklet.row.nums";
+
+  public static final int CARBON_STREAM_BLOCKLET_ROW_NUMS_DEFAULT = 32000;
+
+  public static final String CARBON_STREAM_CACHE_SIZE = "carbon.stream.cache.size";
+
+  public static final int CARBON_STREAM_CACHE_SIZE_DEFAULT = 32 * 1024 * 1024;
+
+  private static final String LOAD_Model = "mapreduce.output.carbon.load.model";
+
+  @Override public RecordWriter<Void, Object> getRecordWriter(TaskAttemptContext job)
+      throws IOException, InterruptedException {
+    return new CarbonStreamRecordWriter(job);
+  }
+
+  public static void setCarbonLoadModel(Configuration hadoopConf, CarbonLoadModel carbonLoadModel)
+      throws IOException {
+    if (carbonLoadModel != null) {
+      hadoopConf.set(LOAD_Model, ObjectSerializationUtil.convertObjectToString(carbonLoadModel));
+    }
+  }
+
+  public static CarbonLoadModel getCarbonLoadModel(Configuration hadoopConf) throws IOException {
+    String value = hadoopConf.get(LOAD_Model);
+    if (value == null) {
+      return null;
+    } else {
+      return (CarbonLoadModel) ObjectSerializationUtil.convertStringToObject(value);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/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
new file mode 100644
index 0000000..1ff0fa7
--- /dev/null
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordReader.java
@@ -0,0 +1,676 @@
+/*
+ * 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.hadoop.streaming;
+
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.carbondata.core.cache.Cache;
+import org.apache.carbondata.core.cache.CacheProvider;
+import org.apache.carbondata.core.cache.CacheType;
+import org.apache.carbondata.core.cache.dictionary.Dictionary;
+import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
+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.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.CarbonColumn;
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
+import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
+import org.apache.carbondata.core.reader.CarbonHeaderReader;
+import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
+import org.apache.carbondata.core.scan.filter.FilterUtil;
+import org.apache.carbondata.core.scan.filter.GenericQueryType;
+import org.apache.carbondata.core.scan.filter.executer.FilterExecuter;
+import org.apache.carbondata.core.scan.filter.intf.RowImpl;
+import org.apache.carbondata.core.scan.filter.intf.RowIntf;
+import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
+import org.apache.carbondata.core.scan.model.QueryModel;
+import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.core.util.DataTypeUtil;
+import org.apache.carbondata.format.BlockletHeader;
+import org.apache.carbondata.format.FileHeader;
+import org.apache.carbondata.hadoop.CarbonInputSplit;
+import org.apache.carbondata.hadoop.CarbonMultiBlockSplit;
+import org.apache.carbondata.hadoop.api.CarbonTableInputFormat;
+import org.apache.carbondata.hadoop.util.CarbonTypeUtil;
+import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.input.FileSplit;
+import org.apache.spark.memory.MemoryMode;
+import org.apache.spark.sql.catalyst.InternalRow;
+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;
+
+/**
+ * Stream record reader
+ */
+public class CarbonStreamRecordReader extends RecordReader<Void, Object> {
+  // vector reader
+  private boolean isVectorReader;
+
+  // metadata
+  private CarbonTable carbonTable;
+  private CarbonColumn[] storageColumns;
+  private boolean[] isRequired;
+  private int[] measureDataTypes;
+  private int dimensionCount;
+  private int measureCount;
+
+  // input
+  private FileSplit fileSplit;
+  private Configuration hadoopConf;
+  private StreamBlockletReader input;
+  private boolean isFirstRow = true;
+  private QueryModel model;
+
+  // decode data
+  private BitSet allNonNull;
+  private boolean[] isNoDictColumn;
+  private DirectDictionaryGenerator[] directDictionaryGenerators;
+  private CacheProvider cacheProvider;
+  private Cache<DictionaryColumnUniqueIdentifier, Dictionary> cache;
+  private GenericQueryType[] queryTypes;
+
+  // vectorized reader
+  private StructType outputSchema;
+  private ColumnarBatch columnarBatch;
+  private boolean isFinished = false;
+
+  // filter
+  private FilterExecuter filter;
+  private boolean[] isFilterRequired;
+  private Object[] filterValues;
+  private RowIntf filterRow;
+  private int[] filterMap;
+
+  // output
+  private CarbonColumn[] projection;
+  private boolean[] isProjectionRequired;
+  private int[] projectionMap;
+  private Object[] outputValues;
+  private InternalRow outputRow;
+
+  // empty project, null filter
+  private boolean skipScanData;
+
+  @Override public void initialize(InputSplit split, TaskAttemptContext context)
+      throws IOException, InterruptedException {
+    // input
+    if (split instanceof CarbonInputSplit) {
+      fileSplit = (CarbonInputSplit) split;
+    } else if (split instanceof CarbonMultiBlockSplit) {
+      fileSplit = ((CarbonMultiBlockSplit) split).getAllSplits().get(0);
+    } else {
+      fileSplit = (FileSplit) split;
+    }
+
+    // metadata
+    hadoopConf = context.getConfiguration();
+    if (model == null) {
+      CarbonTableInputFormat format = new CarbonTableInputFormat<Object>();
+      model = format.getQueryModel(split, context);
+    }
+    carbonTable = model.getTable();
+    List<CarbonDimension> dimensions =
+        carbonTable.getDimensionByTableName(carbonTable.getFactTableName());
+    dimensionCount = dimensions.size();
+    List<CarbonMeasure> measures =
+        carbonTable.getMeasureByTableName(carbonTable.getFactTableName());
+    measureCount = measures.size();
+    List<CarbonColumn> carbonColumnList =
+        carbonTable.getStreamStorageOrderColumn(carbonTable.getFactTableName());
+    storageColumns = carbonColumnList.toArray(new CarbonColumn[carbonColumnList.size()]);
+    isNoDictColumn = CarbonDataProcessorUtil.getNoDictionaryMapping(storageColumns);
+    directDictionaryGenerators = new DirectDictionaryGenerator[storageColumns.length];
+    for (int i = 0; i < storageColumns.length; i++) {
+      if (storageColumns[i].hasEncoding(Encoding.DIRECT_DICTIONARY)) {
+        directDictionaryGenerators[i] = DirectDictionaryKeyGeneratorFactory
+            .getDirectDictionaryGenerator(storageColumns[i].getDataType());
+      }
+    }
+    measureDataTypes = new int[measureCount];
+    for (int i = 0; i < measureCount; i++) {
+      measureDataTypes[i] = storageColumns[dimensionCount + i].getDataType().getId();
+    }
+
+    // decode data
+    allNonNull = new BitSet(storageColumns.length);
+    projection = model.getProjectionColumns();
+
+    isRequired = new boolean[storageColumns.length];
+    boolean[] isFiltlerDimensions = model.getIsFilterDimensions();
+    boolean[] isFiltlerMeasures = model.getIsFilterMeasures();
+    isFilterRequired = new boolean[storageColumns.length];
+    filterMap = new int[storageColumns.length];
+    for (int i = 0; i < storageColumns.length; i++) {
+      if (storageColumns[i].isDimension()) {
+        if (isFiltlerDimensions[storageColumns[i].getOrdinal()]) {
+          isRequired[i] = true;
+          isFilterRequired[i] = true;
+          filterMap[i] = storageColumns[i].getOrdinal();
+        }
+      } else {
+        if (isFiltlerMeasures[storageColumns[i].getOrdinal()]) {
+          isRequired[i] = true;
+          isFilterRequired[i] = true;
+          filterMap[i] = carbonTable.getDimensionOrdinalMax() + storageColumns[i].getOrdinal();
+        }
+      }
+    }
+
+    isProjectionRequired = new boolean[storageColumns.length];
+    projectionMap = new int[storageColumns.length];
+    for (int i = 0; i < storageColumns.length; i++) {
+      for (int j = 0; j < projection.length; j++) {
+        if (storageColumns[i].getColName().equals(projection[j].getColName())) {
+          isRequired[i] = true;
+          isProjectionRequired[i] = true;
+          projectionMap[i] = j;
+          break;
+        }
+      }
+    }
+
+    // initialize filter
+    if (null != model.getFilterExpressionResolverTree()) {
+      initializeFilter();
+    } else if (projection.length == 0) {
+      skipScanData = true;
+    }
+
+  }
+
+  private void initializeFilter() {
+
+    List<ColumnSchema> wrapperColumnSchemaList = CarbonUtil
+        .getColumnSchemaList(carbonTable.getDimensionByTableName(carbonTable.getFactTableName()),
+            carbonTable.getMeasureByTableName(carbonTable.getFactTableName()));
+    int[] dimLensWithComplex = new int[wrapperColumnSchemaList.size()];
+    for (int i = 0; i < dimLensWithComplex.length; i++) {
+      dimLensWithComplex[i] = Integer.MAX_VALUE;
+    }
+
+    int[] dictionaryColumnCardinality =
+        CarbonUtil.getFormattedCardinality(dimLensWithComplex, wrapperColumnSchemaList);
+    SegmentProperties segmentProperties =
+        new SegmentProperties(wrapperColumnSchemaList, dictionaryColumnCardinality);
+    Map<Integer, GenericQueryType> complexDimensionInfoMap = new HashMap<>();
+
+    FilterResolverIntf resolverIntf = model.getFilterExpressionResolverTree();
+    filter = FilterUtil.getFilterExecuterTree(resolverIntf, segmentProperties,
+        complexDimensionInfoMap);
+    // for row filter, we need update column index
+    FilterUtil.updateIndexOfColumnExpression(resolverIntf.getFilterExpression(),
+        carbonTable.getDimensionOrdinalMax());
+
+  }
+
+  public void setQueryModel(QueryModel model) {
+    this.model = model;
+  }
+
+  private byte[] getSyncMarker(String filePath) throws IOException {
+    CarbonHeaderReader headerReader = new CarbonHeaderReader(filePath);
+    FileHeader header = headerReader.readHeader();
+    return header.getSync_marker();
+  }
+
+  private void initializeAtFirstRow() throws IOException {
+    filterValues = new Object[carbonTable.getDimensionOrdinalMax() + measureCount];
+    filterRow = new RowImpl();
+    filterRow.setValues(filterValues);
+
+    outputValues = new Object[projection.length];
+    outputRow = new GenericInternalRow(outputValues);
+
+    Path file = fileSplit.getPath();
+
+    byte[] syncMarker = getSyncMarker(file.toUri().getPath());
+
+    FileSystem fs = file.getFileSystem(hadoopConf);
+
+    int bufferSize = Integer.parseInt(hadoopConf.get(CarbonStreamInputFormat.READ_BUFFER_SIZE,
+        CarbonStreamInputFormat.READ_BUFFER_SIZE_DEFAULT));
+
+    FSDataInputStream fileIn = fs.open(file, bufferSize);
+    fileIn.seek(fileSplit.getStart());
+    input = new StreamBlockletReader(syncMarker, fileIn, fileSplit.getLength(),
+        fileSplit.getStart() == 0);
+
+    cacheProvider = CacheProvider.getInstance();
+    cache = cacheProvider.createCache(CacheType.FORWARD_DICTIONARY, carbonTable.getStorePath());
+    queryTypes = CarbonStreamInputFormat.getComplexDimensions(carbonTable, storageColumns, cache);
+
+    outputSchema = new StructType(CarbonTypeUtil.convertCarbonSchemaToSparkSchema(projection));
+  }
+
+  @Override public boolean nextKeyValue() throws IOException, InterruptedException {
+    if (isFirstRow) {
+      isFirstRow = false;
+      initializeAtFirstRow();
+    }
+    if (isFinished) {
+      return false;
+    }
+
+    if (isVectorReader) {
+      return nextColumnarBatch();
+    }
+
+    return nextRow();
+  }
+
+  /**
+   * for vector reader, check next columnar batch
+   */
+  private boolean nextColumnarBatch() throws IOException {
+    boolean hasNext;
+    boolean scanMore = false;
+    do {
+      // move to the next blocklet
+      hasNext = input.nextBlocklet();
+      if (hasNext) {
+        // read blocklet header
+        BlockletHeader header = input.readBlockletHeader();
+        if (isScanRequired(header)) {
+          scanMore = !scanBlockletAndFillVector(header);
+        } else {
+          input.skipBlockletData(true);
+          scanMore = true;
+        }
+      } else {
+        isFinished = true;
+        scanMore = false;
+      }
+    } while (scanMore);
+    return hasNext;
+  }
+
+  /**
+   * check next Row
+   */
+  private boolean nextRow() throws IOException {
+    // read row one by one
+    try {
+      boolean hasNext;
+      boolean scanMore = false;
+      do {
+        hasNext = input.hasNext();
+        if (hasNext) {
+          if (skipScanData) {
+            input.nextRow();
+            scanMore = false;
+          } else {
+            readRowFromStream();
+            if (null != filter) {
+              scanMore = !filter.applyFilter(filterRow, carbonTable.getDimensionOrdinalMax());
+            } else {
+              scanMore = false;
+            }
+          }
+        } else {
+          if (input.nextBlocklet()) {
+            BlockletHeader header = input.readBlockletHeader();
+            if (isScanRequired(header)) {
+              if (skipScanData) {
+                input.skipBlockletData(false);
+              } else {
+                input.readBlockletData(header);
+              }
+            } else {
+              input.skipBlockletData(true);
+            }
+            scanMore = true;
+          } else {
+            isFinished = true;
+            scanMore = false;
+          }
+        }
+      } while (scanMore);
+      return hasNext;
+    } catch (FilterUnsupportedException e) {
+      throw new IOException("Failed to filter row in detail reader", e);
+    }
+  }
+
+  @Override public Void getCurrentKey() throws IOException, InterruptedException {
+    return null;
+  }
+
+  @Override public Object getCurrentValue() throws IOException, InterruptedException {
+    if (isVectorReader) {
+      return columnarBatch;
+    }
+    return outputRow;
+  }
+
+  private boolean isScanRequired(BlockletHeader header) {
+    // TODO require to implement min-max index
+    if (null == filter) {
+      return true;
+    }
+    return true;
+  }
+
+  private boolean scanBlockletAndFillVector(BlockletHeader header) throws IOException {
+    // if filter is null and output projection is empty, use the row number of blocklet header
+    if (skipScanData) {
+      int rowNums = header.getBlocklet_info().getNum_rows();
+      columnarBatch = ColumnarBatch.allocate(outputSchema, MemoryMode.OFF_HEAP, rowNums);
+      columnarBatch.setNumRows(rowNums);
+      input.skipBlockletData(true);
+      return rowNums > 0;
+    }
+
+    input.readBlockletData(header);
+    columnarBatch = ColumnarBatch.allocate(outputSchema, MemoryMode.OFF_HEAP, input.getRowNums());
+    int rowNum = 0;
+    if (null == filter) {
+      while (input.hasNext()) {
+        readRowFromStream();
+        putRowToColumnBatch(rowNum++);
+      }
+    } else {
+      try {
+        while (input.hasNext()) {
+          readRowFromStream();
+          if (filter.applyFilter(filterRow, carbonTable.getDimensionOrdinalMax())) {
+            putRowToColumnBatch(rowNum++);
+          }
+        }
+      } catch (FilterUnsupportedException e) {
+        throw new IOException("Failed to filter row in vector reader", e);
+      }
+    }
+    columnarBatch.setNumRows(rowNum);
+    return rowNum > 0;
+  }
+
+  private void readRowFromStream() {
+    input.nextRow();
+    short nullLen = input.readShort();
+    BitSet nullBitSet = allNonNull;
+    if (nullLen > 0) {
+      nullBitSet = BitSet.valueOf(input.readBytes(nullLen));
+    }
+    int colCount = 0;
+    // primitive type dimension
+    for (; colCount < isNoDictColumn.length; colCount++) {
+      if (nullBitSet.get(colCount)) {
+        if (isFilterRequired[colCount]) {
+          filterValues[filterMap[colCount]] = CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY;
+        }
+        if (isProjectionRequired[colCount]) {
+          outputValues[projectionMap[colCount]] = null;
+        }
+      } else {
+        if (isNoDictColumn[colCount]) {
+          int v = input.readShort();
+          if (isRequired[colCount]) {
+            byte[] b = input.readBytes(v);
+            if (isFilterRequired[colCount]) {
+              filterValues[filterMap[colCount]] = b;
+            }
+            if (isProjectionRequired[colCount]) {
+              outputValues[projectionMap[colCount]] =
+                  DataTypeUtil.getDataBasedOnDataTypeForNoDictionaryColumn(b,
+                      storageColumns[colCount].getDataType());
+            }
+          } else {
+            input.skipBytes(v);
+          }
+        } else if (null != directDictionaryGenerators[colCount]) {
+          if (isRequired[colCount]) {
+            if (isFilterRequired[colCount]) {
+              filterValues[filterMap[colCount]] = input.copy(4);
+            }
+            if (isProjectionRequired[colCount]) {
+              outputValues[projectionMap[colCount]] =
+                  directDictionaryGenerators[colCount].getValueFromSurrogate(input.readInt());
+            } else {
+              input.skipBytes(4);
+            }
+          } else {
+            input.skipBytes(4);
+          }
+        } else {
+          if (isRequired[colCount]) {
+            if (isFilterRequired[colCount]) {
+              filterValues[filterMap[colCount]] = input.copy(4);
+            }
+            if (isProjectionRequired[colCount]) {
+              outputValues[projectionMap[colCount]] = input.readInt();
+            } else {
+              input.skipBytes(4);
+            }
+          } else {
+            input.skipBytes(4);
+          }
+        }
+      }
+    }
+    // complex type dimension
+    for (; colCount < dimensionCount; colCount++) {
+      if (nullBitSet.get(colCount)) {
+        if (isFilterRequired[colCount]) {
+          filterValues[filterMap[colCount]] = null;
+        }
+        if (isProjectionRequired[colCount]) {
+          outputValues[projectionMap[colCount]] = null;
+        }
+      } else {
+        short v = input.readShort();
+        if (isRequired[colCount]) {
+          byte[] b = input.readBytes(v);
+          if (isFilterRequired[colCount]) {
+            filterValues[filterMap[colCount]] = b;
+          }
+          if (isProjectionRequired[colCount]) {
+            outputValues[projectionMap[colCount]] = queryTypes[colCount]
+                .getDataBasedOnDataTypeFromSurrogates(ByteBuffer.wrap(b));
+          }
+        } else {
+          input.skipBytes(v);
+        }
+      }
+    }
+    // measure
+    int dataType;
+    for (int msrCount = 0; msrCount < measureCount; msrCount++, colCount++) {
+      if (nullBitSet.get(colCount)) {
+        if (isFilterRequired[colCount]) {
+          filterValues[filterMap[colCount]] = null;
+        }
+        if (isProjectionRequired[colCount]) {
+          outputValues[projectionMap[colCount]] = null;
+        }
+      } else {
+        dataType = measureDataTypes[msrCount];
+        if (dataType == DataTypes.BOOLEAN_TYPE_ID) {
+          if (isRequired[colCount]) {
+            boolean v = input.readBoolean();
+            if (isFilterRequired[colCount]) {
+              filterValues[filterMap[colCount]] = v;
+            }
+            if (isProjectionRequired[colCount]) {
+              outputValues[projectionMap[colCount]] = v;
+            }
+          } else {
+            input.skipBytes(1);
+          }
+        } else if (dataType == DataTypes.SHORT_TYPE_ID) {
+          if (isRequired[colCount]) {
+            short v = input.readShort();
+            if (isFilterRequired[colCount]) {
+              filterValues[filterMap[colCount]] = v;
+            }
+            if (isProjectionRequired[colCount]) {
+              outputValues[projectionMap[colCount]] = v;
+            }
+          } else {
+            input.skipBytes(2);
+          }
+        } else if (dataType == DataTypes.INT_TYPE_ID) {
+          if (isRequired[colCount]) {
+            int v = input.readInt();
+            if (isFilterRequired[colCount]) {
+              filterValues[filterMap[colCount]] = v;
+            }
+            if (isProjectionRequired[colCount]) {
+              outputValues[projectionMap[colCount]] = v;
+            }
+          } else {
+            input.skipBytes(4);
+          }
+        } else if (dataType == DataTypes.LONG_TYPE_ID) {
+          if (isRequired[colCount]) {
+            long v = input.readLong();
+            if (isFilterRequired[colCount]) {
+              filterValues[filterMap[colCount]] = v;
+            }
+            if (isProjectionRequired[colCount]) {
+              outputValues[projectionMap[colCount]] = v;
+            }
+          } else {
+            input.skipBytes(8);
+          }
+        } else if (dataType == DataTypes.DOUBLE_TYPE_ID) {
+          if (isRequired[colCount]) {
+            double v = input.readDouble();
+            if (isFilterRequired[colCount]) {
+              filterValues[filterMap[colCount]] = v;
+            }
+            if (isProjectionRequired[colCount]) {
+              outputValues[projectionMap[colCount]] = v;
+            }
+          } else {
+            input.skipBytes(8);
+          }
+        } else if (dataType == DataTypes.DECIMAL_TYPE_ID) {
+          int len = input.readShort();
+          if (isRequired[colCount]) {
+            BigDecimal v = DataTypeUtil.byteToBigDecimal(input.readBytes(len));
+            if (isFilterRequired[colCount]) {
+              filterValues[filterMap[colCount]] = v;
+            }
+            if (isProjectionRequired[colCount]) {
+              outputValues[projectionMap[colCount]] = v;
+            }
+          } else {
+            input.skipBytes(len);
+          }
+        }
+      }
+    }
+  }
+
+  private void putRowToColumnBatch(int rowId) {
+    for (int i = 0; i < projection.length; i++) {
+      Object value = outputValues[i];
+      ColumnVector col = columnarBatch.column(i);
+      DataType t = col.dataType();
+      if (null == value) {
+        col.putNull(rowId);
+      } else {
+        if (t == org.apache.spark.sql.types.DataTypes.BooleanType) {
+          col.putBoolean(rowId, (boolean)value);
+        } else if (t == org.apache.spark.sql.types.DataTypes.ByteType) {
+          col.putByte(rowId, (byte) value);
+        } else if (t == org.apache.spark.sql.types.DataTypes.ShortType) {
+          col.putShort(rowId, (short) value);
+        } else if (t == org.apache.spark.sql.types.DataTypes.IntegerType) {
+          col.putInt(rowId, (int) value);
+        } else if (t == org.apache.spark.sql.types.DataTypes.LongType) {
+          col.putLong(rowId, (long) value);
+        } else if (t == org.apache.spark.sql.types.DataTypes.FloatType) {
+          col.putFloat(rowId, (float) value);
+        } else if (t == org.apache.spark.sql.types.DataTypes.DoubleType) {
+          col.putDouble(rowId, (double) value);
+        } else if (t == org.apache.spark.sql.types.DataTypes.StringType) {
+          UTF8String v = (UTF8String) value;
+          col.putByteArray(rowId, v.getBytes());
+        } else if (t instanceof DecimalType) {
+          DecimalType dt = (DecimalType)t;
+          Decimal d = (Decimal) value;
+          if (dt.precision() <= Decimal.MAX_INT_DIGITS()) {
+            col.putInt(rowId, (int)d.toUnscaledLong());
+          } else if (dt.precision() <= Decimal.MAX_LONG_DIGITS()) {
+            col.putLong(rowId, d.toUnscaledLong());
+          } else {
+            final BigInteger integer = d.toJavaBigDecimal().unscaledValue();
+            byte[] bytes = integer.toByteArray();
+            col.putByteArray(rowId, bytes, 0, bytes.length);
+          }
+        } else if (t instanceof CalendarIntervalType) {
+          CalendarInterval c = (CalendarInterval) value;
+          col.getChildColumn(0).putInt(rowId, c.months);
+          col.getChildColumn(1).putLong(rowId, c.microseconds);
+        } else if (t instanceof DateType) {
+          col.putInt(rowId, (int) value);
+        } else if (t instanceof TimestampType) {
+          col.putLong(rowId, (long) value);
+        }
+      }
+    }
+  }
+
+  @Override public float getProgress() throws IOException, InterruptedException {
+    return 0;
+  }
+
+  public void setVectorReader(boolean isVectorReader) {
+    this.isVectorReader = isVectorReader;
+  }
+
+  @Override public void close() throws IOException {
+    if (null != input) {
+      input.close();
+    }
+    if (null != columnarBatch) {
+      columnarBatch.close();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/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
new file mode 100644
index 0000000..8d7a2e3
--- /dev/null
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordWriter.java
@@ -0,0 +1,305 @@
+/*
+ * 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.hadoop.streaming;
+
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.List;
+
+import org.apache.carbondata.common.logging.LogService;
+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.DataTypes;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
+import org.apache.carbondata.core.util.CarbonMetadataUtil;
+import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.core.util.DataTypeUtil;
+import org.apache.carbondata.core.util.path.CarbonStorePath;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
+import org.apache.carbondata.format.FileHeader;
+import org.apache.carbondata.processing.loading.BadRecordsLogger;
+import org.apache.carbondata.processing.loading.CarbonDataLoadConfiguration;
+import org.apache.carbondata.processing.loading.DataField;
+import org.apache.carbondata.processing.loading.DataLoadProcessBuilder;
+import org.apache.carbondata.processing.loading.converter.RowConverter;
+import org.apache.carbondata.processing.loading.converter.impl.RowConverterImpl;
+import org.apache.carbondata.processing.loading.model.CarbonLoadModel;
+import org.apache.carbondata.processing.loading.parser.RowParser;
+import org.apache.carbondata.processing.loading.parser.impl.RowParserImpl;
+import org.apache.carbondata.processing.loading.steps.DataConverterProcessorStepImpl;
+import org.apache.carbondata.processing.store.writer.AbstractFactDataWriter;
+import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskID;
+
+/**
+ * Stream record writer
+ */
+public class CarbonStreamRecordWriter extends RecordWriter<Void, Object> {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(CarbonStreamRecordWriter.class.getName());
+
+  // basic info
+  private Configuration hadoopConf;
+  private CarbonDataLoadConfiguration configuration;
+  private CarbonTable carbonTable;
+  private int maxRowNums;
+  private int maxCacheSize;
+
+  // parser and converter
+  private RowParser rowParser;
+  private RowConverter converter;
+  private CarbonRow currentRow = new CarbonRow(null);
+
+  // encoder
+  private DataField[] dataFields;
+  private BitSet nullBitSet;
+  private boolean[] isNoDictionaryDimensionColumn;
+  private int dimensionWithComplexCount;
+  private int measureCount;
+  private int[] measureDataTypes;
+  private StreamBlockletWriter output = null;
+
+  // data write
+  private String segmentDir;
+  private String fileName;
+  private DataOutputStream outputStream;
+  private boolean isFirstRow = true;
+  private boolean hasException = false;
+
+  CarbonStreamRecordWriter(TaskAttemptContext job) throws IOException {
+    initialize(job);
+  }
+
+  private void initialize(TaskAttemptContext job) throws IOException {
+    // set basic information
+    hadoopConf = job.getConfiguration();
+    CarbonLoadModel carbonLoadModel = CarbonStreamOutputFormat.getCarbonLoadModel(hadoopConf);
+    if (carbonLoadModel == null) {
+      throw new IOException(
+          "CarbonStreamRecordWriter require configuration: mapreduce.output.carbon.load.model");
+    }
+    carbonTable = carbonLoadModel.getCarbonDataLoadSchema().getCarbonTable();
+    int taskNo = TaskID.forName(hadoopConf.get("mapred.tip.id")).getId();
+    carbonLoadModel.setTaskNo("" + taskNo);
+    configuration = DataLoadProcessBuilder.createConfiguration(carbonLoadModel);
+    maxRowNums = hadoopConf.getInt(CarbonStreamOutputFormat.CARBON_STREAM_BLOCKLET_ROW_NUMS,
+        CarbonStreamOutputFormat.CARBON_STREAM_BLOCKLET_ROW_NUMS_DEFAULT) - 1;
+    maxCacheSize = hadoopConf.getInt(CarbonStreamOutputFormat.CARBON_STREAM_CACHE_SIZE,
+        CarbonStreamOutputFormat.CARBON_STREAM_CACHE_SIZE_DEFAULT);
+
+    CarbonTablePath tablePath =
+        CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier());
+    segmentDir = tablePath.getSegmentDir("0", carbonLoadModel.getSegmentId());
+    fileName = CarbonTablePath.getCarbonDataFileName(0, taskNo, 0, 0, "0");
+  }
+
+  private void initializeAtFirstRow() throws IOException, InterruptedException {
+    isFirstRow = false;
+
+    // initialize metadata
+    isNoDictionaryDimensionColumn =
+        CarbonDataProcessorUtil.getNoDictionaryMapping(configuration.getDataFields());
+    dimensionWithComplexCount = configuration.getDimensionCount();
+    measureCount = configuration.getMeasureCount();
+    dataFields = configuration.getDataFields();
+    measureDataTypes = new int[measureCount];
+    for (int i = 0; i < measureCount; i++) {
+      measureDataTypes[i] =
+          dataFields[dimensionWithComplexCount + i].getColumn().getDataType().getId();
+    }
+
+    // initialize parser and converter
+    rowParser = new RowParserImpl(dataFields, configuration);
+    BadRecordsLogger badRecordLogger =
+        DataConverterProcessorStepImpl.createBadRecordLogger(configuration);
+    converter = new RowConverterImpl(configuration.getDataFields(), configuration, badRecordLogger);
+    configuration.setCardinalityFinder(converter);
+    converter.initialize();
+
+    // initialize encoder
+    nullBitSet = new BitSet(dataFields.length);
+    int rowBufferSize = hadoopConf.getInt(CarbonStreamOutputFormat.CARBON_ENCODER_ROW_BUFFER_SIZE,
+        CarbonStreamOutputFormat.CARBON_ENCODER_ROW_BUFFER_SIZE_DEFAULT);
+    output = new StreamBlockletWriter(maxCacheSize, maxRowNums, rowBufferSize);
+
+    // initialize data writer
+    String filePath = segmentDir + File.separator + fileName;
+    FileFactory.FileType fileType = FileFactory.getFileType(filePath);
+    CarbonFile carbonFile = FileFactory.getCarbonFile(filePath, fileType);
+    if (carbonFile.exists()) {
+      // if the file is existed, use the append api
+      outputStream = FileFactory.getDataOutputStreamUsingAppend(filePath, fileType);
+    } else {
+      // IF the file is not existed, use the create api
+      outputStream = FileFactory.getDataOutputStream(filePath, fileType);
+      writeFileHeader();
+    }
+  }
+
+  @Override public void write(Void key, Object value) throws IOException, InterruptedException {
+    if (isFirstRow) {
+      initializeAtFirstRow();
+    }
+
+    // parse and convert row
+    currentRow.setData(rowParser.parseRow((Object[]) value));
+    converter.convert(currentRow);
+
+    // null bit set
+    nullBitSet.clear();
+    for (int i = 0; i < dataFields.length; i++) {
+      if (null == currentRow.getObject(i)) {
+        nullBitSet.set(i);
+      }
+    }
+    output.nextRow();
+    byte[] b = nullBitSet.toByteArray();
+    output.writeShort(b.length);
+    if (b.length > 0) {
+      output.writeBytes(b);
+    }
+    int dimCount = 0;
+    Object columnValue;
+
+    // primitive type dimension
+    for (; dimCount < isNoDictionaryDimensionColumn.length; dimCount++) {
+      columnValue = currentRow.getObject(dimCount);
+      if (null != columnValue) {
+        if (isNoDictionaryDimensionColumn[dimCount]) {
+          byte[] col = (byte[]) columnValue;
+          output.writeShort(col.length);
+          output.writeBytes(col);
+        } else {
+          output.writeInt((int) columnValue);
+        }
+      }
+    }
+    // complex type dimension
+    for (; dimCount < dimensionWithComplexCount; dimCount++) {
+      columnValue = currentRow.getObject(dimCount);
+      if (null != columnValue) {
+        byte[] col = (byte[]) columnValue;
+        output.writeShort(col.length);
+        output.writeBytes(col);
+      }
+    }
+    // measure
+    int 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) {
+          output.writeBoolean((boolean) columnValue);
+        } else if (dataType == DataTypes.SHORT_TYPE_ID) {
+          output.writeShort((short) columnValue);
+        } else if (dataType == DataTypes.INT_TYPE_ID) {
+          output.writeInt((int) columnValue);
+        } else if (dataType == DataTypes.LONG_TYPE_ID) {
+          output.writeLong((long) columnValue);
+        } else if (dataType == DataTypes.DOUBLE_TYPE_ID) {
+          output.writeDouble((double) columnValue);
+        } else if (dataType == DataTypes.DECIMAL_TYPE_ID) {
+          BigDecimal val = (BigDecimal) columnValue;
+          byte[] bigDecimalInBytes = DataTypeUtil.bigDecimalToByte(val);
+          output.writeShort(bigDecimalInBytes.length);
+          output.writeBytes(bigDecimalInBytes);
+        } else {
+          String msg =
+              "unsupported data type:" + dataFields[dimCount + msrCount].getColumn().getDataType()
+                  .getName();
+          LOGGER.error(msg);
+          throw new IOException(msg);
+        }
+      }
+    }
+
+    if (output.isFull()) {
+      appendBlockletToDataFile();
+    }
+  }
+
+  private void writeFileHeader() throws IOException {
+    List<ColumnSchema> wrapperColumnSchemaList = CarbonUtil
+        .getColumnSchemaList(carbonTable.getDimensionByTableName(carbonTable.getFactTableName()),
+            carbonTable.getMeasureByTableName(carbonTable.getFactTableName()));
+    int[] dimLensWithComplex = new int[wrapperColumnSchemaList.size()];
+    for (int i = 0; i < dimLensWithComplex.length; i++) {
+      dimLensWithComplex[i] = Integer.MAX_VALUE;
+    }
+    int[] dictionaryColumnCardinality =
+        CarbonUtil.getFormattedCardinality(dimLensWithComplex, wrapperColumnSchemaList);
+    List<Integer> cardinality = new ArrayList<>();
+    List<org.apache.carbondata.format.ColumnSchema> columnSchemaList = AbstractFactDataWriter
+        .getColumnSchemaListAndCardinality(cardinality, dictionaryColumnCardinality,
+            wrapperColumnSchemaList);
+    FileHeader fileHeader =
+        CarbonMetadataUtil.getFileHeader(true, columnSchemaList, System.currentTimeMillis());
+    fileHeader.setIs_footer_present(false);
+    fileHeader.setIs_splitable(true);
+    fileHeader.setSync_marker(CarbonStreamOutputFormat.CARBON_SYNC_MARKER);
+    outputStream.write(CarbonUtil.getByteArray(fileHeader));
+  }
+
+  /**
+   * write a blocklet to file
+   */
+  private void appendBlockletToDataFile() throws IOException {
+    if (output.getRowIndex() == -1) {
+      return;
+    }
+    output.apppendBlocklet(outputStream);
+    outputStream.flush();
+    // reset data
+    output.reset();
+  }
+
+  @Override public void close(TaskAttemptContext context) throws IOException, InterruptedException {
+    try {
+      // append remain buffer data
+      if (!hasException) {
+        appendBlockletToDataFile();
+        converter.finish();
+      }
+    } finally {
+      // close resource
+      CarbonUtil.closeStreams(outputStream);
+      output.close();
+    }
+  }
+
+  public String getSegmentDir() {
+    return segmentDir;
+  }
+
+  public String getFileName() {
+    return fileName;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/StreamBlockletReader.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/StreamBlockletReader.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/StreamBlockletReader.java
new file mode 100644
index 0000000..eafb142
--- /dev/null
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/StreamBlockletReader.java
@@ -0,0 +1,248 @@
+/*
+ * 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.hadoop.streaming;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.carbondata.core.datastore.compression.Compressor;
+import org.apache.carbondata.core.datastore.compression.CompressorFactory;
+import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.format.BlockletHeader;
+
+/**
+ * stream blocklet reader
+ */
+public class StreamBlockletReader {
+
+  private byte[] buffer;
+  private int offset;
+  private final byte[] syncMarker;
+  private final byte[] syncBuffer;
+  private final int syncLen;
+  private long pos = 0;
+  private final InputStream in;
+  private final long limitStart;
+  private final long limitEnd;
+  private boolean isAlreadySync = false;
+  private Compressor compressor = CompressorFactory.getInstance().getCompressor();
+  private int rowNums = 0;
+  private int rowIndex = 0;
+  private boolean isHeaderPresent;
+
+  StreamBlockletReader(byte[] syncMarker, InputStream in, long limit, boolean isHeaderPresent) {
+    this.syncMarker = syncMarker;
+    this.syncLen = syncMarker.length;
+    this.syncBuffer = new byte[syncMarker.length];
+    this.in = in;
+    this.limitStart = limit;
+    this.limitEnd = limitStart + syncLen;
+    this.isHeaderPresent = isHeaderPresent;
+  }
+
+  private void ensureCapacity(int capacity) {
+    if (buffer == null || capacity > buffer.length) {
+      buffer = new byte[capacity];
+    }
+  }
+
+  /**
+   * find the first position of sync_marker in input stream
+   */
+  private boolean sync() throws IOException {
+    int len = in.read(syncBuffer);
+    if (len < syncLen) {
+      return false;
+    }
+    pos += syncLen;
+    boolean skipHeader = false;
+    for (int i = 0; i < limitStart; i++) {
+      int j = 0;
+      for (; j < syncLen; j++) {
+        if (syncMarker[j] != syncBuffer[(i + j) % syncLen]) break;
+      }
+      if (syncLen == j) {
+        if (isHeaderPresent) {
+          if (skipHeader) {
+            return true;
+          } else {
+            skipHeader = true;
+          }
+        } else {
+          return true;
+        }
+      }
+      int value = in.read();
+      if (-1 == value) {
+        return false;
+      }
+      syncBuffer[i % syncLen] = (byte) value;
+      pos++;
+    }
+    return false;
+  }
+
+  BlockletHeader readBlockletHeader() throws IOException {
+    int len = readIntFromStream();
+    byte[] b = new byte[len];
+    readBytesFromStream(b);
+    BlockletHeader header = CarbonUtil.readBlockletHeader(b);
+    rowNums = header.getBlocklet_info().getNum_rows();
+    rowIndex = 0;
+    return header;
+  }
+
+  void readBlockletData(BlockletHeader header) throws IOException {
+    ensureCapacity(header.getBlocklet_length());
+    offset = 0;
+    int len = readIntFromStream();
+    byte[] b = new byte[len];
+    readBytesFromStream(b);
+    compressor.rawUncompress(b, buffer);
+  }
+
+  void skipBlockletData(boolean reset) throws IOException {
+    int len = readIntFromStream();
+    skip(len);
+    pos += len;
+    if (reset) {
+      this.rowNums = 0;
+      this.rowIndex = 0;
+    }
+  }
+
+  private void skip(int len) throws IOException {
+    long remaining = len;
+    do {
+      long skipLen = in.skip(remaining);
+      remaining -= skipLen;
+    } while (remaining > 0);
+  }
+
+  /**
+   * find the next blocklet
+   */
+  boolean nextBlocklet() throws IOException {
+    if (pos >= limitStart) {
+      return false;
+    }
+    if (isAlreadySync) {
+      int v = in.read(syncBuffer);
+      if (v < syncLen) {
+        return false;
+      }
+      pos += syncLen;
+    } else {
+      isAlreadySync = true;
+      if (!sync()) {
+        return false;
+      }
+    }
+
+    return pos < limitEnd;
+  }
+
+  boolean hasNext() throws IOException {
+    return rowIndex < rowNums;
+  }
+
+  void nextRow() {
+    rowIndex++;
+  }
+
+  int readIntFromStream() throws IOException {
+    int ch1 = in.read();
+    int ch2 = in.read();
+    int ch3 = in.read();
+    int ch4 = in.read();
+    if ((ch1 | ch2 | ch3 | ch4) < 0) throw new EOFException();
+    pos += 4;
+    return ((ch1 << 24) + (ch2 << 16) + (ch3 << 8) + (ch4 << 0));
+  }
+
+  void readBytesFromStream(byte[] b) throws IOException {
+    int len = in.read(b, 0, b.length);
+    if (len < b.length) {
+      throw new EOFException();
+    }
+    pos += b.length;
+  }
+
+  boolean readBoolean() {
+    return (buffer[offset++]) != 0;
+  }
+
+  short readShort() {
+    short v =  (short) ((buffer[offset + 1] & 255) +
+        ((buffer[offset]) << 8));
+    offset += 2;
+    return v;
+  }
+
+  byte[] copy(int len) {
+    byte[] b = new byte[len];
+    System.arraycopy(buffer, offset, b, 0, len);
+    return b;
+  }
+
+  int readInt() {
+    int v = ((buffer[offset + 3] & 255) +
+        ((buffer[offset + 2] & 255) << 8) +
+        ((buffer[offset + 1] & 255) << 16) +
+        ((buffer[offset]) << 24));
+    offset += 4;
+    return v;
+  }
+
+  long readLong() {
+    long v = ((long)(buffer[offset + 7] & 255)) +
+        ((long) (buffer[offset + 6] & 255) << 8) +
+        ((long) (buffer[offset + 5] & 255) << 16) +
+        ((long) (buffer[offset + 4] & 255) << 24) +
+        ((long) (buffer[offset + 3] & 255) << 32) +
+        ((long) (buffer[offset + 2] & 255) << 40) +
+        ((long) (buffer[offset + 1] & 255) << 48) +
+        ((long) (buffer[offset]) << 56);
+    offset += 8;
+    return v;
+  }
+
+  double readDouble() {
+    return Double.longBitsToDouble(readLong());
+  }
+
+  byte[] readBytes(int len) {
+    byte[] b = new byte[len];
+    System.arraycopy(buffer, offset, b, 0, len);
+    offset += len;
+    return b;
+  }
+
+  void skipBytes(int len) {
+    offset += len;
+  }
+
+  int getRowNums() {
+    return rowNums;
+  }
+
+  void close() {
+    CarbonUtil.closeStreams(in);
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/StreamBlockletWriter.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/StreamBlockletWriter.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/StreamBlockletWriter.java
new file mode 100644
index 0000000..a0328b3
--- /dev/null
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/StreamBlockletWriter.java
@@ -0,0 +1,152 @@
+/*
+ * 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.hadoop.streaming;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import org.apache.carbondata.core.datastore.compression.Compressor;
+import org.apache.carbondata.core.datastore.compression.CompressorFactory;
+import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.format.BlockletHeader;
+import org.apache.carbondata.format.BlockletInfo;
+import org.apache.carbondata.format.MutationType;
+
+/**
+ * stream blocklet writer
+ */
+public class StreamBlockletWriter {
+  private byte[] buffer;
+  private int maxSize;
+  private int maxRowNum;
+  private int rowSize;
+  private int count = 0;
+  private int rowIndex = -1;
+  private Compressor compressor = CompressorFactory.getInstance().getCompressor();
+
+  StreamBlockletWriter(int maxSize, int maxRowNum, int rowSize) {
+    buffer = new byte[maxSize];
+    this.maxSize = maxSize;
+    this.maxRowNum = maxRowNum;
+    this.rowSize = rowSize;
+  }
+
+  private void ensureCapacity(int space) {
+    int newcount = space + count;
+    if (newcount > buffer.length) {
+      byte[] newbuf = new byte[Math.max(newcount, buffer.length + rowSize)];
+      System.arraycopy(buffer, 0, newbuf, 0, count);
+      buffer = newbuf;
+    }
+  }
+
+  void reset() {
+    count = 0;
+    rowIndex = -1;
+  }
+
+  byte[] getBytes() {
+    return buffer;
+  }
+
+  int getCount() {
+    return count;
+  }
+
+  int getRowIndex() {
+    return rowIndex;
+  }
+
+  void nextRow() {
+    rowIndex++;
+  }
+
+  boolean isFull() {
+    return rowIndex == maxRowNum || count >= maxSize;
+  }
+
+  void writeBoolean(boolean val) {
+    ensureCapacity(1);
+    buffer[count] = (byte) (val ? 1 : 0);
+    count += 1;
+  }
+
+  void writeShort(int val) {
+    ensureCapacity(2);
+    buffer[count + 1] = (byte) (val);
+    buffer[count] = (byte) (val >>> 8);
+    count += 2;
+  }
+
+  void writeInt(int val) {
+    ensureCapacity(4);
+    buffer[count + 3] = (byte) (val);
+    buffer[count + 2] = (byte) (val >>> 8);
+    buffer[count + 1] = (byte) (val >>> 16);
+    buffer[count] = (byte) (val >>> 24);
+    count += 4;
+  }
+
+  void writeLong(long val) {
+    ensureCapacity(8);
+    buffer[count + 7] = (byte) (val);
+    buffer[count + 6] = (byte) (val >>> 8);
+    buffer[count + 5] = (byte) (val >>> 16);
+    buffer[count + 4] = (byte) (val >>> 24);
+    buffer[count + 3] = (byte) (val >>> 32);
+    buffer[count + 2] = (byte) (val >>> 40);
+    buffer[count + 1] = (byte) (val >>> 48);
+    buffer[count] = (byte) (val >>> 56);
+    count += 8;
+  }
+
+  void writeDouble(double val) {
+    writeLong(Double.doubleToLongBits(val));
+  }
+
+  void writeBytes(byte[] b) {
+    writeBytes(b, 0, b.length);
+  }
+
+  void writeBytes(byte[] b, int off, int len) {
+    ensureCapacity(len);
+    System.arraycopy(b, off, buffer, count, len);
+    count += len;
+  }
+
+  void apppendBlocklet(DataOutputStream outputStream) throws IOException {
+    outputStream.write(CarbonStreamOutputFormat.CARBON_SYNC_MARKER);
+
+    BlockletInfo blockletInfo = new BlockletInfo();
+    blockletInfo.setNum_rows(getRowIndex() + 1);
+    BlockletHeader blockletHeader = new BlockletHeader();
+    blockletHeader.setBlocklet_length(getCount());
+    blockletHeader.setMutation(MutationType.INSERT);
+    blockletHeader.setBlocklet_info(blockletInfo);
+    byte[] headerBytes = CarbonUtil.getByteArray(blockletHeader);
+    outputStream.writeInt(headerBytes.length);
+    outputStream.write(headerBytes);
+
+    byte[] compressed = compressor.compressByte(getBytes(), getCount());
+    outputStream.writeInt(compressed.length);
+    outputStream.write(compressed);
+  }
+
+  void close() {
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java
index a559cc4..b4444be 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java
@@ -109,12 +109,14 @@ public class CarbonInputFormatUtil {
     plan.addDimension(queryDimension);
   }
 
-  public static void processFilterExpression(Expression filterExpression, CarbonTable carbonTable) {
+  public static void processFilterExpression(Expression filterExpression, CarbonTable carbonTable,
+      boolean[] isFilterDimensions, boolean[] isFilterMeasures) {
     List<CarbonDimension> dimensions =
         carbonTable.getDimensionByTableName(carbonTable.getFactTableName());
     List<CarbonMeasure> measures =
         carbonTable.getMeasureByTableName(carbonTable.getFactTableName());
-    QueryModel.processFilterExpression(filterExpression, dimensions, measures);
+    QueryModel.processFilterExpression(filterExpression, dimensions, measures,
+        isFilterDimensions, isFilterMeasures);
 
     if (null != filterExpression) {
       // Optimize Filter Expression and fit RANGE filters is conditions apply.

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonTypeUtil.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonTypeUtil.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonTypeUtil.java
new file mode 100644
index 0000000..395015e
--- /dev/null
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonTypeUtil.java
@@ -0,0 +1,101 @@
+/*
+ * 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.hadoop.util;
+
+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.encoder.Encoding;
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
+
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.DecimalType;
+import org.apache.spark.sql.types.StructField;
+
+public class CarbonTypeUtil {
+
+  public static org.apache.spark.sql.types.DataType convertCarbonToSparkDataType(
+      DataType carbonDataType) {
+    if (carbonDataType == org.apache.carbondata.core.metadata.datatype.DataTypes.STRING) {
+      return DataTypes.StringType;
+    } else if (carbonDataType == org.apache.carbondata.core.metadata.datatype.DataTypes.SHORT) {
+      return DataTypes.ShortType;
+    } else if (carbonDataType == org.apache.carbondata.core.metadata.datatype.DataTypes.INT) {
+      return DataTypes.IntegerType;
+    } else if (carbonDataType == org.apache.carbondata.core.metadata.datatype.DataTypes.LONG) {
+      return DataTypes.LongType;
+    } else if (carbonDataType == org.apache.carbondata.core.metadata.datatype.DataTypes.DOUBLE) {
+      return DataTypes.DoubleType;
+    } else if (carbonDataType == org.apache.carbondata.core.metadata.datatype.DataTypes.BOOLEAN) {
+      return DataTypes.BooleanType;
+    } else if (org.apache.carbondata.core.metadata.datatype.DataTypes.isDecimal(carbonDataType)) {
+      return DataTypes.createDecimalType();
+    } else if (carbonDataType == org.apache.carbondata.core.metadata.datatype.DataTypes.TIMESTAMP) {
+      return DataTypes.TimestampType;
+    } else if (carbonDataType == org.apache.carbondata.core.metadata.datatype.DataTypes.DATE) {
+      return DataTypes.DateType;
+    } else {
+      return null;
+    }
+  }
+
+  public static StructField[] convertCarbonSchemaToSparkSchema(CarbonColumn[] carbonColumns) {
+    StructField[] fields = new StructField[carbonColumns.length];
+    for (int i = 0; i < carbonColumns.length; i++) {
+      CarbonColumn carbonColumn = carbonColumns[i];
+      if (carbonColumn.isDimension()) {
+        if (carbonColumn.hasEncoding(Encoding.DIRECT_DICTIONARY)) {
+          DirectDictionaryGenerator generator = DirectDictionaryKeyGeneratorFactory
+              .getDirectDictionaryGenerator(carbonColumn.getDataType());
+          fields[i] = new StructField(carbonColumn.getColName(),
+              CarbonTypeUtil.convertCarbonToSparkDataType(generator.getReturnType()), true, null);
+        } else if (!carbonColumn.hasEncoding(Encoding.DICTIONARY)) {
+          fields[i] = new StructField(carbonColumn.getColName(),
+              CarbonTypeUtil.convertCarbonToSparkDataType(carbonColumn.getDataType()), true, null);
+        } else if (carbonColumn.isComplex()) {
+          fields[i] = new StructField(carbonColumn.getColName(),
+              CarbonTypeUtil.convertCarbonToSparkDataType(carbonColumn.getDataType()), true, null);
+        } else {
+          fields[i] = new StructField(carbonColumn.getColName(), CarbonTypeUtil
+              .convertCarbonToSparkDataType(
+                  org.apache.carbondata.core.metadata.datatype.DataTypes.INT), true, null);
+        }
+      } else if (carbonColumn.isMeasure()) {
+        DataType dataType = carbonColumn.getDataType();
+        if (dataType == org.apache.carbondata.core.metadata.datatype.DataTypes.BOOLEAN
+            || dataType == org.apache.carbondata.core.metadata.datatype.DataTypes.SHORT
+            || dataType == org.apache.carbondata.core.metadata.datatype.DataTypes.INT
+            || dataType == org.apache.carbondata.core.metadata.datatype.DataTypes.LONG) {
+          fields[i] = new StructField(carbonColumn.getColName(),
+              CarbonTypeUtil.convertCarbonToSparkDataType(dataType), true, null);
+        } else if (org.apache.carbondata.core.metadata.datatype.DataTypes.isDecimal(dataType)) {
+          CarbonMeasure measure = (CarbonMeasure) carbonColumn;
+          fields[i] = new StructField(carbonColumn.getColName(),
+              new DecimalType(measure.getPrecision(), measure.getScale()), true, null);
+        } else {
+          fields[i] = new StructField(carbonColumn.getColName(), CarbonTypeUtil
+              .convertCarbonToSparkDataType(
+                  org.apache.carbondata.core.metadata.datatype.DataTypes.DOUBLE), true, null);
+        }
+      }
+    }
+    return fields;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/StoreCreator.java
----------------------------------------------------------------------
diff --git a/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/StoreCreator.java b/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/StoreCreator.java
index ea90bbf..29d8d03 100644
--- a/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/StoreCreator.java
+++ b/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/StoreCreator.java
@@ -116,6 +116,45 @@ public class StoreCreator {
     return absoluteTableIdentifier;
   }
 
+  public static CarbonLoadModel buildCarbonLoadModel(CarbonTable table, String factFilePath) {
+    CarbonDataLoadSchema schema = new CarbonDataLoadSchema(table);
+    CarbonLoadModel loadModel = new CarbonLoadModel();
+    loadModel.setCarbonDataLoadSchema(schema);
+    loadModel.setDatabaseName(absoluteTableIdentifier.getCarbonTableIdentifier().getDatabaseName());
+    loadModel.setTableName(absoluteTableIdentifier.getCarbonTableIdentifier().getTableName());
+    loadModel.setTableName(absoluteTableIdentifier.getCarbonTableIdentifier().getTableName());
+    loadModel.setFactFilePath(factFilePath);
+    loadModel.setLoadMetadataDetails(new ArrayList<LoadMetadataDetails>());
+    loadModel.setStorePath(absoluteTableIdentifier.getStorePath());
+    loadModel.setDateFormat(null);
+    loadModel.setDefaultTimestampFormat(CarbonProperties.getInstance().getProperty(
+        CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
+        CarbonCommonConstants.CARBON_TIMESTAMP_MILLIS));
+    loadModel.setDefaultDateFormat(CarbonProperties.getInstance().getProperty(
+        CarbonCommonConstants.CARBON_DATE_FORMAT,
+        CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT));
+    loadModel
+        .setSerializationNullFormat(
+            TableOptionConstant.SERIALIZATION_NULL_FORMAT.getName() + "," + "\\N");
+    loadModel
+        .setBadRecordsLoggerEnable(
+            TableOptionConstant.BAD_RECORDS_LOGGER_ENABLE.getName() + "," + "false");
+    loadModel
+        .setBadRecordsAction(
+            TableOptionConstant.BAD_RECORDS_ACTION.getName() + "," + "FORCE");
+    loadModel
+        .setIsEmptyDataBadRecord(
+            DataLoadProcessorConstants.IS_EMPTY_DATA_BAD_RECORD + "," + "false");
+    loadModel.setCsvHeader("ID,date,country,name,phonetype,serialname,salary");
+    loadModel.setCsvHeaderColumns(loadModel.getCsvHeader().split(","));
+    loadModel.setTaskNo("0");
+    loadModel.setSegmentId("0");
+    loadModel.setPartitionId("0");
+    loadModel.setFactTimeStamp(System.currentTimeMillis());
+    loadModel.setMaxColumns("10");
+    return loadModel;
+  }
+
   /**
    * Create store without any restructure
    */
@@ -131,42 +170,7 @@ public class StoreCreator {
 
       CarbonTable table = createTable();
       writeDictionary(factFilePath, table);
-      CarbonDataLoadSchema schema = new CarbonDataLoadSchema(table);
-      CarbonLoadModel loadModel = new CarbonLoadModel();
-      String partitionId = "0";
-      loadModel.setCarbonDataLoadSchema(schema);
-      loadModel.setDatabaseName(absoluteTableIdentifier.getCarbonTableIdentifier().getDatabaseName());
-      loadModel.setTableName(absoluteTableIdentifier.getCarbonTableIdentifier().getTableName());
-      loadModel.setTableName(absoluteTableIdentifier.getCarbonTableIdentifier().getTableName());
-      loadModel.setFactFilePath(factFilePath);
-      loadModel.setLoadMetadataDetails(new ArrayList<LoadMetadataDetails>());
-      loadModel.setStorePath(absoluteTableIdentifier.getStorePath());
-      loadModel.setDateFormat(null);
-      loadModel.setDefaultTimestampFormat(CarbonProperties.getInstance().getProperty(
-          CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
-          CarbonCommonConstants.CARBON_TIMESTAMP_MILLIS));
-      loadModel.setDefaultDateFormat(CarbonProperties.getInstance().getProperty(
-          CarbonCommonConstants.CARBON_DATE_FORMAT,
-          CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT));
-      loadModel
-          .setSerializationNullFormat(
-              TableOptionConstant.SERIALIZATION_NULL_FORMAT.getName() + "," + "\\N");
-      loadModel
-          .setBadRecordsLoggerEnable(
-              TableOptionConstant.BAD_RECORDS_LOGGER_ENABLE.getName() + "," + "false");
-      loadModel
-          .setBadRecordsAction(
-              TableOptionConstant.BAD_RECORDS_ACTION.getName() + "," + "FORCE");
-      loadModel
-          .setIsEmptyDataBadRecord(
-              DataLoadProcessorConstants.IS_EMPTY_DATA_BAD_RECORD + "," + "false");
-      loadModel.setCsvHeader("ID,date,country,name,phonetype,serialname,salary");
-      loadModel.setCsvHeaderColumns(loadModel.getCsvHeader().split(","));
-      loadModel.setTaskNo("0");
-      loadModel.setSegmentId("0");
-      loadModel.setPartitionId("0");
-      loadModel.setFactTimeStamp(System.currentTimeMillis());
-      loadModel.setMaxColumns("10");
+      CarbonLoadModel loadModel = buildCarbonLoadModel(table, factFilePath);
 
       executeGraph(loadModel, absoluteTableIdentifier.getStorePath());
 
@@ -176,7 +180,7 @@ public class StoreCreator {
 
   }
 
-  private static CarbonTable createTable() throws IOException {
+  public static CarbonTable createTable() throws IOException {
     TableInfo tableInfo = new TableInfo();
     tableInfo.setStorePath(absoluteTableIdentifier.getStorePath());
     tableInfo.setDatabaseName(absoluteTableIdentifier.getCarbonTableIdentifier().getDatabaseName());

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/integration/hive/src/main/java/org/apache/carbondata/hive/MapredCarbonInputFormat.java
----------------------------------------------------------------------
diff --git a/integration/hive/src/main/java/org/apache/carbondata/hive/MapredCarbonInputFormat.java b/integration/hive/src/main/java/org/apache/carbondata/hive/MapredCarbonInputFormat.java
index 2e840c0..4cbc692 100644
--- a/integration/hive/src/main/java/org/apache/carbondata/hive/MapredCarbonInputFormat.java
+++ b/integration/hive/src/main/java/org/apache/carbondata/hive/MapredCarbonInputFormat.java
@@ -135,7 +135,7 @@ public class MapredCarbonInputFormat extends CarbonInputFormat<ArrayWritable>
         QueryModel.createModel(identifier, queryPlan, carbonTable, new DataTypeConverterImpl());
     // set the filter to the query model in order to filter blocklet before scan
     Expression filter = getFilterPredicates(configuration);
-    CarbonInputFormatUtil.processFilterExpression(filter, carbonTable);
+    CarbonInputFormatUtil.processFilterExpression(filter, carbonTable, null, null);
     FilterResolverIntf filterIntf =
         CarbonInputFormatUtil.resolveFilter(filter, identifier, tableProvider);
     queryModel.setFilterExpressionResolverTree(filterIntf);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/integration/presto/src/main/java/org/apache/carbondata/presto/CarbonVectorizedRecordReader.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbonVectorizedRecordReader.java b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbonVectorizedRecordReader.java
index f129474..5e3e5b7 100644
--- a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbonVectorizedRecordReader.java
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbonVectorizedRecordReader.java
@@ -42,6 +42,7 @@ import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.hadoop.AbstractRecordReader;
 import org.apache.carbondata.hadoop.CarbonInputSplit;
 import org.apache.carbondata.hadoop.CarbonMultiBlockSplit;
+import org.apache.carbondata.hadoop.util.CarbonTypeUtil;
 
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
index fc34127..7ec6b7b 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
@@ -21,6 +21,8 @@ import java.text.SimpleDateFormat
 import java.util.{ArrayList, Date, List}
 
 import scala.collection.JavaConverters._
+import scala.collection.mutable
+import scala.collection.mutable.ArrayBuffer
 import scala.util.Random
 
 import org.apache.hadoop.conf.Configuration
@@ -38,9 +40,11 @@ import org.apache.carbondata.core.metadata.schema.table.TableInfo
 import org.apache.carbondata.core.scan.expression.Expression
 import org.apache.carbondata.core.scan.model.QueryModel
 import org.apache.carbondata.core.stats.{QueryStatistic, QueryStatisticsConstants, QueryStatisticsRecorder}
-import org.apache.carbondata.core.util.{CarbonProperties, CarbonTimeStatisticsFactory, TaskMetricsMap}
+import org.apache.carbondata.core.statusmanager.FileFormat
+import org.apache.carbondata.core.util.{CarbonProperties, CarbonTimeStatisticsFactory, DataTypeUtil, TaskMetricsMap}
 import org.apache.carbondata.hadoop._
 import org.apache.carbondata.hadoop.api.CarbonTableInputFormat
+import org.apache.carbondata.hadoop.streaming.{CarbonStreamInputFormat, CarbonStreamRecordReader}
 import org.apache.carbondata.processing.util.CarbonLoaderUtil
 import org.apache.carbondata.spark.InitInputMetrics
 import org.apache.carbondata.spark.util.SparkDataTypeConverterImpl
@@ -82,11 +86,45 @@ class CarbonScanRDD(
 
     // get splits
     val splits = format.getSplits(job)
-    val result = distributeSplits(splits)
-    result
+
+    // separate split
+    // 1. for batch splits, invoke distributeSplits method to create partitions
+    // 2. for stream splits, create partition for each split by default
+    val columnarSplits = new ArrayList[InputSplit]()
+    val streamSplits = new ArrayBuffer[InputSplit]()
+    splits.asScala.foreach { split =>
+      val carbonInputSplit = split.asInstanceOf[CarbonInputSplit]
+      if (FileFormat.rowformat == carbonInputSplit.getFileFormat) {
+        streamSplits += split
+      } else {
+        columnarSplits.add(split)
+      }
+    }
+    val batchPartitions = distributeColumnarSplits(columnarSplits)
+    if (streamSplits.isEmpty) {
+      batchPartitions.toArray
+    } else {
+      val index = batchPartitions.length
+      val streamPartitions: mutable.Buffer[Partition] =
+        streamSplits.zipWithIndex.map { splitWithIndex =>
+          val multiBlockSplit =
+            new CarbonMultiBlockSplit(identifier,
+              Seq(splitWithIndex._1.asInstanceOf[CarbonInputSplit]).asJava,
+              splitWithIndex._1.getLocations,
+              FileFormat.rowformat)
+          new CarbonSparkPartition(id, splitWithIndex._2 + index, multiBlockSplit)
+        }
+      if (batchPartitions.isEmpty) {
+        streamPartitions.toArray
+      } else {
+        // should keep the order by index of partition
+        batchPartitions.appendAll(streamPartitions)
+        batchPartitions.toArray
+      }
+    }
   }
 
-  private def distributeSplits(splits: List[InputSplit]): Array[Partition] = {
+  private def distributeColumnarSplits(splits: List[InputSplit]): mutable.Buffer[Partition] = {
     // this function distributes the split based on following logic:
     // 1. based on data locality, to make split balanced on all available nodes
     // 2. if the number of split for one
@@ -190,7 +228,7 @@ class CarbonScanRDD(
          | no.of.nodes: $noOfNodes,
          | parallelism: $parallelism
        """.stripMargin)
-    result.toArray(new Array[Partition](result.size()))
+    result.asScala
   }
 
   override def internalCompute(split: Partition, context: TaskContext): Iterator[InternalRow] = {
@@ -210,20 +248,34 @@ class CarbonScanRDD(
     inputMetricsStats.initBytesReadCallback(context, inputSplit)
     val iterator = if (inputSplit.getAllSplits.size() > 0) {
       val model = format.getQueryModel(inputSplit, attemptContext)
-      val reader = {
-        if (vectorReader) {
-          val carbonRecordReader = createVectorizedCarbonRecordReader(model, inputMetricsStats)
-          if (carbonRecordReader == null) {
-            new CarbonRecordReader(model,
-              format.getReadSupportClass(attemptContext.getConfiguration), inputMetricsStats)
+      // get RecordReader by FileFormat
+      val reader: RecordReader[Void, Object] = inputSplit.getFileFormat match {
+        case FileFormat.rowformat =>
+          // create record reader for row format
+          DataTypeUtil.setDataTypeConverter(new SparkDataTypeConverterImpl)
+          val inputFormat = new CarbonStreamInputFormat
+          val streamReader = inputFormat.createRecordReader(inputSplit, attemptContext)
+            .asInstanceOf[CarbonStreamRecordReader]
+          streamReader.setVectorReader(vectorReader)
+          model.setStatisticsRecorder(
+            CarbonTimeStatisticsFactory.createExecutorRecorder(model.getQueryId))
+          streamReader.setQueryModel(model)
+          streamReader
+        case _ =>
+          // create record reader for CarbonData file format
+          if (vectorReader) {
+            val carbonRecordReader = createVectorizedCarbonRecordReader(model, inputMetricsStats)
+            if (carbonRecordReader == null) {
+              new CarbonRecordReader(model,
+                format.getReadSupportClass(attemptContext.getConfiguration), inputMetricsStats)
+            } else {
+              carbonRecordReader
+            }
           } else {
-            carbonRecordReader
+            new CarbonRecordReader(model,
+              format.getReadSupportClass(attemptContext.getConfiguration),
+              inputMetricsStats)
           }
-        } else {
-          new CarbonRecordReader(model,
-            format.getReadSupportClass(attemptContext.getConfiguration),
-            inputMetricsStats)
-        }
       }
 
       reader.initialize(inputSplit, attemptContext)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/integration/spark2/pom.xml
----------------------------------------------------------------------
diff --git a/integration/spark2/pom.xml b/integration/spark2/pom.xml
index ed01728..18e37ad 100644
--- a/integration/spark2/pom.xml
+++ b/integration/spark2/pom.xml
@@ -36,7 +36,7 @@
   <dependencies>
     <dependency>
       <groupId>org.apache.carbondata</groupId>
-      <artifactId>carbondata-spark-common</artifactId>
+      <artifactId>carbondata-streaming</artifactId>
       <version>${project.version}</version>
     </dependency>
     <dependency>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/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 d5adc2f..10336eb 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
@@ -23,9 +23,11 @@ import org.apache.spark.CarbonInputMetrics
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.execution.command.management.LoadTableByInsertCommand
+import org.apache.spark.sql.execution.streaming.Sink
 import org.apache.spark.sql.hive.CarbonRelation
 import org.apache.spark.sql.optimizer.CarbonFilters
-import org.apache.spark.sql.sources.{BaseRelation, Filter, InsertableRelation}
+import org.apache.spark.sql.sources.{BaseRelation, Filter, InsertableRelation, StreamSinkProvider}
+import org.apache.spark.sql.streaming.OutputMode
 import org.apache.spark.sql.types.StructType
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
@@ -36,6 +38,7 @@ 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
 
 case class CarbonDatasourceHadoopRelation(
     sparkSession: SparkSession,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/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 eeca8b8..6020eee 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
@@ -22,6 +22,7 @@ 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.internal.{SessionState, SharedState}
 import org.apache.spark.util.Utils
@@ -168,6 +169,7 @@ object CarbonSession {
             SparkSession.sqlListener.set(null)
           }
         })
+        session.streams.addListener(new CarbonStreamingQueryListener(session))
       }
 
       return session


[30/49] carbondata git commit: [CARBONDATA-1592]Event listener added

Posted by ra...@apache.org.
[CARBONDATA-1592]Event listener added

This closes #1473


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

Branch: refs/heads/fgdatamap
Commit: 4aa0f493038a050f61af7c5d6c3af22cd20fd56b
Parents: fd0bdf6
Author: rahulforallp <ra...@knoldus.in>
Authored: Tue Nov 7 11:50:15 2017 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Fri Nov 10 10:19:46 2017 +0530

----------------------------------------------------------------------
 .../carbondata/core/datamap/TableDataMap.java   |  15 ++-
 .../core/datamap/dev/DataMapFactory.java        |   4 +-
 .../carbondata/core/events/ChangeEvent.java     |  35 -----
 .../carbondata/core/events/EventListener.java   |  25 ----
 .../blockletindex/BlockletDataMapFactory.java   |  11 +-
 .../org/apache/carbondata/events/Event.java     |  31 +++++
 .../carbondata/events/OperationContext.java     |  41 ++++++
 .../events/OperationEventListener.java          |  31 +++++
 .../carbondata/events/OperationListenerBus.java | 119 +++++++++++++++++
 .../testsuite/datamap/DataMapWriterSuite.scala  |   4 +-
 .../carbondata/events/AlterTableEvents.scala    | 132 +++++++++++++++++++
 .../carbondata/events/CarbonInitEvents.scala    |  38 ++++++
 .../carbondata/events/CleanFilesEvents.scala    |  46 +++++++
 .../carbondata/events/DeleteSegmentEvents.scala |  77 +++++++++++
 .../carbondata/events/DropTableEvents.scala     |  57 ++++++++
 .../org/apache/carbondata/events/Events.scala   | 131 ++++++++++++++++++
 .../apache/carbondata/events/IUDEvents.scala    |  62 +++++++++
 .../apache/carbondata/events/LoadEvents.scala   |  46 +++++++
 .../events/LookupRelationEvents.scala           |  51 +++++++
 .../apache/carbondata/spark/rdd/Compactor.scala |  14 +-
 .../spark/rdd/CarbonDataRDDFactory.scala        |   9 ++
 .../scala/org/apache/spark/sql/CarbonEnv.scala  |   6 +
 .../command/CarbonDropTableCommand.scala        |  22 +++-
 .../command/management/CleanFilesCommand.scala  |  11 ++
 .../management/DeleteLoadByIdCommand.scala      |  15 +++
 .../DeleteLoadByLoadDateCommand.scala           |  15 +++
 .../mutation/ProjectForDeleteCommand.scala      |  14 +-
 .../mutation/ProjectForUpdateCommand.scala      |  13 ++
 .../CarbonAlterTableDropColumnCommand.scala     |  19 +++
 .../schema/CarbonAlterTableRenameCommand.scala  |  16 +++
 .../spark/sql/hive/CarbonFileMetastore.scala    |   9 ++
 31 files changed, 1039 insertions(+), 80 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/4aa0f493/core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java b/core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java
index 1cf1def..705a9fd 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java
@@ -22,19 +22,20 @@ import java.util.List;
 
 import org.apache.carbondata.core.datamap.dev.DataMap;
 import org.apache.carbondata.core.datamap.dev.DataMapFactory;
-import org.apache.carbondata.core.events.ChangeEvent;
-import org.apache.carbondata.core.events.EventListener;
 import org.apache.carbondata.core.indexstore.Blocklet;
 import org.apache.carbondata.core.indexstore.BlockletDetailsFetcher;
 import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
+import org.apache.carbondata.events.Event;
+import org.apache.carbondata.events.OperationContext;
+import org.apache.carbondata.events.OperationEventListener;
 
 /**
  * DataMap at the table level, user can add any number of datamaps for one table. Depends
  * on the filter condition it can prune the blocklets.
  */
-public final class TableDataMap implements EventListener {
+public final class TableDataMap implements OperationEventListener {
 
   private AbsoluteTableIdentifier identifier;
 
@@ -132,10 +133,6 @@ public final class TableDataMap implements EventListener {
     return detailedBlocklets;
   }
 
-  @Override public void fireEvent(ChangeEvent event) {
-    dataMapFactory.fireEvent(event);
-  }
-
   /**
    * Clear only the datamaps of the segments
    * @param segmentIds
@@ -164,4 +161,8 @@ public final class TableDataMap implements EventListener {
   public DataMapFactory getDataMapFactory() {
     return dataMapFactory;
   }
+
+  @Override public void onEvent(Event event, OperationContext opContext) {
+    dataMapFactory.fireEvent(event);
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4aa0f493/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java
index cf0519b..f5a7404 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java
@@ -21,8 +21,8 @@ import java.util.List;
 
 import org.apache.carbondata.core.datamap.DataMapDistributable;
 import org.apache.carbondata.core.datamap.DataMapMeta;
-import org.apache.carbondata.core.events.ChangeEvent;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
+import org.apache.carbondata.events.Event;
 
 /**
  * Interface for datamap factory, it is responsible for creating the datamap.
@@ -59,7 +59,7 @@ public interface DataMapFactory {
    *
    * @param event
    */
-  void fireEvent(ChangeEvent event);
+  void fireEvent(Event event);
 
   /**
    * Clears datamap of the segment

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4aa0f493/core/src/main/java/org/apache/carbondata/core/events/ChangeEvent.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/events/ChangeEvent.java b/core/src/main/java/org/apache/carbondata/core/events/ChangeEvent.java
deleted file mode 100644
index de02c64..0000000
--- a/core/src/main/java/org/apache/carbondata/core/events/ChangeEvent.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * 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.events;
-
-/**
- * Change event for any updates in store.
- */
-public interface ChangeEvent<T> {
-
-  EventType getEventType();
-
-  T getEventData();
-
-  void setEventData(T data);
-
-  enum EventType {
-    INSERT,DELETE,UPDATE,REFRESH
-  }
-}
-
-

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4aa0f493/core/src/main/java/org/apache/carbondata/core/events/EventListener.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/events/EventListener.java b/core/src/main/java/org/apache/carbondata/core/events/EventListener.java
deleted file mode 100644
index adf45b2..0000000
--- a/core/src/main/java/org/apache/carbondata/core/events/EventListener.java
+++ /dev/null
@@ -1,25 +0,0 @@
-/*
- * 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.events;
-
-/**
- * Event listener
- */
-public interface EventListener {
-
-  void fireEvent(ChangeEvent event);
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4aa0f493/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java
index a1bbba5..e58c911 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java
@@ -32,13 +32,13 @@ import org.apache.carbondata.core.datamap.dev.DataMapFactory;
 import org.apache.carbondata.core.datamap.dev.DataMapWriter;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
-import org.apache.carbondata.core.events.ChangeEvent;
 import org.apache.carbondata.core.indexstore.Blocklet;
 import org.apache.carbondata.core.indexstore.BlockletDetailsFetcher;
 import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
 import org.apache.carbondata.core.indexstore.TableBlockIndexUniqueIdentifier;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
+import org.apache.carbondata.events.Event;
 
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocatedFileStatus;
@@ -164,6 +164,10 @@ public class BlockletDataMapFactory implements DataMapFactory, BlockletDetailsFe
     return distributables;
   }
 
+  @Override public void fireEvent(Event event) {
+
+  }
+
   @Override
   public void clear(String segmentId) {
     List<TableBlockIndexUniqueIdentifier> blockIndexes = segmentMap.remove(segmentId);
@@ -213,11 +217,6 @@ public class BlockletDataMapFactory implements DataMapFactory, BlockletDetailsFe
   }
 
   @Override
-  public void fireEvent(ChangeEvent event) {
-
-  }
-
-  @Override
   public DataMapMeta getMeta() {
     // TODO: pass SORT_COLUMNS into this class
     return null;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4aa0f493/core/src/main/java/org/apache/carbondata/events/Event.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/events/Event.java b/core/src/main/java/org/apache/carbondata/events/Event.java
new file mode 100644
index 0000000..4de6632
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/events/Event.java
@@ -0,0 +1,31 @@
+/*
+ * 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.events;
+
+/**
+ * Event class which will be used for retrieving the type of interface
+ */
+public abstract class Event {
+
+  /**
+   * Method for getting the event type. Used for invoking all listeners registered for an event
+   *
+   * @return
+   */
+  String getEventType() { return this.getClass().getName(); }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4aa0f493/core/src/main/java/org/apache/carbondata/events/OperationContext.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/events/OperationContext.java b/core/src/main/java/org/apache/carbondata/events/OperationContext.java
new file mode 100644
index 0000000..f6fe676
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/events/OperationContext.java
@@ -0,0 +1,41 @@
+/*
+ * 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.events;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * One OperationContext per one operation.
+ * OperationContext active till operation execution completes
+ */
+public class OperationContext {
+
+  private Map<String, Object> operationProperties = new HashMap<String, Object>();
+
+  public Map<String, Object> getProperties() {
+    return operationProperties;
+  }
+
+  public void setProperty(String key, Object value) {
+    this.operationProperties.put(key, value);
+  }
+
+  public Object getProperty(String key) {
+    return this.operationProperties.get(key);
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4aa0f493/core/src/main/java/org/apache/carbondata/events/OperationEventListener.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/events/OperationEventListener.java b/core/src/main/java/org/apache/carbondata/events/OperationEventListener.java
new file mode 100644
index 0000000..7007f9b
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/events/OperationEventListener.java
@@ -0,0 +1,31 @@
+/*
+ * 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.events;
+
+/**
+ * Event listener interface which describes the possible events
+ */
+public interface OperationEventListener {
+
+  /**
+   * Called on a specified event occurrence
+   *
+   * @param event
+   * @param operationContext
+   */
+  void onEvent(Event event, OperationContext operationContext) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4aa0f493/core/src/main/java/org/apache/carbondata/events/OperationListenerBus.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/events/OperationListenerBus.java b/core/src/main/java/org/apache/carbondata/events/OperationListenerBus.java
new file mode 100644
index 0000000..01ecb04
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/events/OperationListenerBus.java
@@ -0,0 +1,119 @@
+/*
+ * 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.events;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CopyOnWriteArrayList;
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+
+/**
+ * An event bus which posts events to its listeners.
+ */
+public class OperationListenerBus {
+
+  /**
+   * singleton instance
+   */
+  private static final OperationListenerBus INSTANCE = new OperationListenerBus();
+
+  /**
+   * Event map to hold all listeners corresponding to an event
+   */
+  protected Map<String, List<OperationEventListener>> eventMap =
+      new ConcurrentHashMap<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+
+  /**
+   * @return
+   */
+  public static OperationListenerBus getInstance() {
+    return INSTANCE;
+  }
+
+  /**
+   * Add a listener to listen events. This method is thread-safe and can be called in any thread.
+   *
+   * @param eventClass
+   * @param operationEventListener
+   */
+  public void addListener(Class<? extends Event> eventClass,
+      OperationEventListener operationEventListener) {
+
+    String eventType = eventClass.getName();
+    List<OperationEventListener> operationEventListeners = eventMap.get(eventType);
+    if (null == operationEventListeners) {
+      operationEventListeners = new CopyOnWriteArrayList<>();
+      eventMap.put(eventType, operationEventListeners);
+    }
+    operationEventListeners.add(operationEventListener);
+  }
+
+  /**
+   * Removes a given map from the event. This method is thread-safe and can be called
+   * in any thread.
+   *
+   * @param eventType
+   */
+  public void removeEvent(String eventType) {
+    eventMap.remove(eventType);
+  }
+
+  /**
+   * Removes a given listener for an event. This method is thread-safe and can be called
+   * in any thread.
+   *
+   * @param eventType
+   * @param operationEventListener
+   */
+  public void removeListener(String eventType, OperationEventListener operationEventListener) {
+    List<OperationEventListener> operationEventListeners = eventMap.get(eventType);
+    if (null != operationEventListeners) {
+      operationEventListeners.remove(operationEventListener);
+    }
+  }
+
+  /**
+   * Notify all registered listeners on occurrence of an event
+   * Should be used for stateless events which cannot be mapped to a operation
+   *
+   * @param event
+   */
+  public void fireEvent(Event event) throws Exception {
+    fireEvent(event, new OperationContext());
+  }
+
+  /**
+   * Notify all registered listeners on occurrence of an event
+   *
+   * @param event
+   * @param operationContext
+   */
+  public void fireEvent(Event event, OperationContext operationContext) throws Exception {
+    if (operationContext == null) {
+      throw new Exception("OperationContext cannot be null");
+    }
+    List<OperationEventListener> operationEventListeners = eventMap.get(event.getEventType());
+    if (null != operationEventListeners) {
+      for (OperationEventListener operationEventListener : operationEventListeners) {
+        operationEventListener.onEvent(event, operationContext);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4aa0f493/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala
index 888c97d..348d9a7 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala
@@ -26,18 +26,18 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datamap.dev.{DataMap, DataMapFactory, DataMapWriter}
 import org.apache.carbondata.core.datamap.{DataMapDistributable, DataMapMeta, DataMapStoreManager}
 import org.apache.carbondata.core.datastore.page.ColumnPage
-import org.apache.carbondata.core.events.ChangeEvent
 import org.apache.carbondata.core.indexstore.schema.FilterType
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
 import org.apache.carbondata.core.metadata.datatype.{DataType, DataTypes}
 import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.events.Event
 
 class C2DataMapFactory() extends DataMapFactory {
 
   override def init(identifier: AbsoluteTableIdentifier,
       dataMapName: String): Unit = {}
 
-  override def fireEvent(event: ChangeEvent[_]): Unit = ???
+  override def fireEvent(event: Event): Unit = ???
 
   override def clear(segmentId: String): Unit = {}
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4aa0f493/integration/spark-common/src/main/scala/org/apache/carbondata/events/AlterTableEvents.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/events/AlterTableEvents.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/events/AlterTableEvents.scala
new file mode 100644
index 0000000..ec79acc
--- /dev/null
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/events/AlterTableEvents.scala
@@ -0,0 +1,132 @@
+/*
+ * 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.events
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.SQLContext
+import org.apache.spark.sql.execution.command.{AlterTableDropColumnModel, AlterTableRenameModel}
+
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+import org.apache.carbondata.processing.loading.model.CarbonLoadModel
+
+/**
+ *
+ * @param carbonTable
+ * @param alterTableDropColumnModel
+ * @param sparkSession
+ */
+case class AlterTableDropColumnPreEvent(carbonTable: CarbonTable,
+    alterTableDropColumnModel: AlterTableDropColumnModel,
+    sparkSession: SparkSession) extends Event with AlterTableDropColumnEventInfo
+
+
+/**
+ *
+ * @param carbonTable
+ * @param alterTableDropColumnModel
+ * @param sparkSession
+ */
+case class AlterTableDropColumnPostEvent(carbonTable: CarbonTable,
+    alterTableDropColumnModel: AlterTableDropColumnModel,
+    sparkSession: SparkSession) extends Event with AlterTableDropColumnEventInfo
+
+
+/**
+ *
+ * @param carbonTable
+ * @param alterTableDropColumnModel
+ * @param sparkSession
+ */
+case class AlterTableDropColumnAbortEvent(carbonTable: CarbonTable,
+    alterTableDropColumnModel: AlterTableDropColumnModel,
+    sparkSession: SparkSession) extends Event with AlterTableDropColumnEventInfo
+
+
+/**
+ *
+ * @param carbonTable
+ * @param alterTableRenameModel
+ * @param newTablePath
+ * @param sparkSession
+ */
+case class AlterTableRenamePreEvent(carbonTable: CarbonTable,
+    alterTableRenameModel: AlterTableRenameModel, newTablePath: String,
+    sparkSession: SparkSession) extends Event with AlterTableRenameEventInfo
+
+
+/**
+ *
+ * @param carbonTable
+ * @param alterTableRenameModel
+ * @param newTablePath
+ * @param sparkSession
+ */
+case class AlterTableRenamePostEvent(carbonTable: CarbonTable,
+    alterTableRenameModel: AlterTableRenameModel, newTablePath: String,
+    sparkSession: SparkSession) extends Event with AlterTableRenameEventInfo
+
+
+/**
+ *
+ * @param carbonTable
+ * @param alterTableRenameModel
+ * @param newTablePath
+ * @param sparkSession
+ */
+case class AlterTableRenameAbortEvent(carbonTable: CarbonTable,
+    alterTableRenameModel: AlterTableRenameModel, newTablePath: String,
+    sparkSession: SparkSession) extends Event with AlterTableRenameEventInfo
+
+
+/**
+ *
+ * @param carbonTable
+ * @param carbonLoadModel
+ * @param mergedLoadName
+ * @param sQLContext
+ */
+case class AlterTableCompactionPreEvent(carbonTable: CarbonTable,
+    carbonLoadModel: CarbonLoadModel,
+    mergedLoadName: String,
+    sQLContext: SQLContext) extends Event with AlterTableCompactionEventInfo
+
+
+/**
+ *
+ * @param carbonTable
+ * @param carbonLoadModel
+ * @param mergedLoadName
+ * @param sQLContext
+ */
+case class AlterTableCompactionPostEvent(carbonTable: CarbonTable,
+    carbonLoadModel: CarbonLoadModel,
+    mergedLoadName: String,
+    sQLContext: SQLContext) extends Event with AlterTableCompactionEventInfo
+
+
+/**
+ * Class for handling clean up in case of any failure and abort the operation
+ *
+ * @param carbonTable
+ * @param carbonLoadModel
+ * @param mergedLoadName
+ * @param sQLContext
+ */
+case class AlterTableCompactionAbortEvent(carbonTable: CarbonTable,
+    carbonLoadModel: CarbonLoadModel,
+    mergedLoadName: String,
+    sQLContext: SQLContext) extends Event with AlterTableCompactionEventInfo

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4aa0f493/integration/spark-common/src/main/scala/org/apache/carbondata/events/CarbonInitEvents.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/events/CarbonInitEvents.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/events/CarbonInitEvents.scala
new file mode 100644
index 0000000..b76c534
--- /dev/null
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/events/CarbonInitEvents.scala
@@ -0,0 +1,38 @@
+/*
+ * 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.events
+
+import org.apache.spark.sql.SparkSession
+
+/**
+ *
+ * @param sparkSession
+ * @param storePath
+ */
+case class CarbonEnvInitPreEvent(
+    sparkSession: SparkSession, storePath: String)
+  extends Event with SessionEventInfo
+
+
+/**
+ *
+ * @param sparkSession
+ * @param storePath
+ */
+case class CarbonEnvInitPostEvent(
+    sparkSession: SparkSession, storePath: String)
+  extends Event with SessionEventInfo

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4aa0f493/integration/spark-common/src/main/scala/org/apache/carbondata/events/CleanFilesEvents.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/events/CleanFilesEvents.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/events/CleanFilesEvents.scala
new file mode 100644
index 0000000..1a9c5f6
--- /dev/null
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/events/CleanFilesEvents.scala
@@ -0,0 +1,46 @@
+/*
+ * 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.events
+
+import org.apache.spark.sql.SparkSession
+
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+
+/**
+ *
+ * @param carbonTable
+ * @param sparkSession
+ */
+case class CleanFilesPreEvent(carbonTable: CarbonTable, sparkSession: SparkSession)
+  extends Event with CleanFilesEventInfo
+
+
+/**
+ *
+ * @param carbonTable
+ * @param sparkSession
+ */
+case class CleanFilesPostEvent(carbonTable: CarbonTable, sparkSession: SparkSession)
+  extends Event with CleanFilesEventInfo
+
+/**
+ *
+ * @param carbonTable
+ * @param sparkSession
+ */
+case class CleanFilesAbortEvent(carbonTable: CarbonTable, sparkSession: SparkSession)
+  extends Event with CleanFilesEventInfo

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4aa0f493/integration/spark-common/src/main/scala/org/apache/carbondata/events/DeleteSegmentEvents.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/events/DeleteSegmentEvents.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/events/DeleteSegmentEvents.scala
new file mode 100644
index 0000000..0008492
--- /dev/null
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/events/DeleteSegmentEvents.scala
@@ -0,0 +1,77 @@
+/*
+ * 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.events
+
+import org.apache.spark.sql.SparkSession
+
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+
+/**
+ *
+ * @param carbonTable
+ * @param loadIds
+ * @param sparkSession
+ */
+case class DeleteSegmentByIdPreEvent(carbonTable: CarbonTable, loadIds: Seq[String],
+    sparkSession: SparkSession) extends Event with DeleteSegmentbyIdEventInfo
+
+
+/**
+ *
+ * @param carbonTable
+ * @param loadIds
+ * @param sparkSession
+ */
+case class DeleteSegmentByIdPostEvent(carbonTable: CarbonTable, loadIds: Seq[String],
+    sparkSession: SparkSession) extends Event with DeleteSegmentbyIdEventInfo
+
+
+/**
+ *
+ * @param carbonTable
+ * @param loadIds
+ * @param sparkSession
+ */
+case class DeleteSegmentByIdAbortEvent(carbonTable: CarbonTable, loadIds: Seq[String],
+    sparkSession: SparkSession) extends Event with DeleteSegmentbyIdEventInfo
+
+/**
+ *
+ * @param carbonTable
+ * @param loadDates
+ * @param sparkSession
+ */
+case class DeleteSegmentByDatePreEvent(carbonTable: CarbonTable, loadDates: String,
+    sparkSession: SparkSession) extends Event with DeleteSegmentbyDateEventInfo
+
+/**
+ *
+ * @param carbonTable
+ * @param loadDates
+ * @param sparkSession
+ */
+case class DeleteSegmentByDatePostEvent(carbonTable: CarbonTable, loadDates: String,
+    sparkSession: SparkSession) extends Event with DeleteSegmentbyDateEventInfo
+
+/**
+ *
+ * @param carbonTable
+ * @param loadDates
+ * @param sparkSession
+ */
+case class DeleteSegmentByDateAbortEvent(carbonTable: CarbonTable, loadDates: String,
+    sparkSession: SparkSession) extends Event with DeleteSegmentbyDateEventInfo

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4aa0f493/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
new file mode 100644
index 0000000..ed43de6
--- /dev/null
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/events/DropTableEvents.scala
@@ -0,0 +1,57 @@
+/*
+ * 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.events
+
+import org.apache.spark.sql.SparkSession
+
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+
+
+/**
+ *
+ * @param carbonTable
+ * @param ifExistsSet
+ * @param sparkSession
+ */
+case class DropTablePreEvent(carbonTable: CarbonTable,
+    ifExistsSet: Boolean,
+    sparkSession: SparkSession)
+  extends Event with DropTableEventInfo
+
+
+/**
+ *
+ * @param carbonTable
+ * @param ifExistsSet
+ * @param sparkSession
+ */
+case class DropTablePostEvent(carbonTable: CarbonTable,
+    ifExistsSet: Boolean,
+    sparkSession: SparkSession)
+  extends Event with DropTableEventInfo
+
+
+/**
+ *
+ * @param carbonTable
+ * @param ifExistsSet
+ * @param sparkSession
+ */
+case class DropTableAbortEvent(carbonTable: CarbonTable,
+    ifExistsSet: Boolean,
+    sparkSession: SparkSession)
+  extends Event with DropTableEventInfo

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4aa0f493/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
new file mode 100644
index 0000000..0d923ed
--- /dev/null
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/events/Events.scala
@@ -0,0 +1,131 @@
+/*
+ * 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.events
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.execution.command.{AlterTableDropColumnModel, AlterTableRenameModel}
+
+import org.apache.carbondata.core.metadata.CarbonTableIdentifier
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+import org.apache.carbondata.processing.loading.model.CarbonLoadModel
+
+/**
+ * event for database operations
+ */
+trait DatabaseEventInfo {
+  val databaseName: String
+}
+
+/**
+ * event for table related operations
+ */
+trait TableEventInfo {
+  val carbonTableIdentifier: CarbonTableIdentifier
+}
+
+/**
+ * event for load operations
+ */
+trait LoadEventInfo {
+  val carbonLoadModel: CarbonLoadModel
+}
+
+/**
+ * event for lookup
+ */
+trait LookupRelationEventInfo {
+  val carbonTable: CarbonTable
+}
+
+
+/**
+ * event for drop table
+ */
+trait DropTableEventInfo {
+  val carbonTable: CarbonTable
+  val ifExistsSet: Boolean
+}
+
+/**
+ * event for alter_table_drop_column
+ */
+trait AlterTableDropColumnEventInfo {
+  val carbonTable: CarbonTable
+  val alterTableDropColumnModel: AlterTableDropColumnModel
+}
+
+/**
+ * event for alter_table_rename
+ */
+trait AlterTableRenameEventInfo {
+  val carbonTable: CarbonTable
+  val alterTableRenameModel: AlterTableRenameModel
+}
+
+/**
+ * event for alter_table_rename
+ */
+trait AlterTableCompactionEventInfo {
+  val carbonTable: CarbonTable
+  val carbonLoadModel: CarbonLoadModel
+  val mergedLoadName: String
+}
+
+/**
+ * event for DeleteSegmentById
+ */
+trait DeleteSegmentbyIdEventInfo {
+  val carbonTable: CarbonTable
+  val loadIds: Seq[String]
+}
+
+/**
+ * event for DeleteSegmentByDate
+ */
+trait DeleteSegmentbyDateEventInfo {
+  val carbonTable: CarbonTable
+  val loadDates: String
+}
+
+/**
+ * event for Clean Files
+ */
+trait CleanFilesEventInfo {
+  val carbonTable: CarbonTable
+}
+
+/**
+ * event for update table
+ */
+trait UpdateTableEventInfo {
+  val carbonTable: CarbonTable
+}
+
+/**
+ * event for delete from table
+ */
+trait DeleteFromTableEventInfo {
+  val carbonTable: CarbonTable
+}
+
+/**
+ * event to initiate CarbonEnv
+ */
+trait SessionEventInfo {
+  val sparkSession: SparkSession
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4aa0f493/integration/spark-common/src/main/scala/org/apache/carbondata/events/IUDEvents.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/events/IUDEvents.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/events/IUDEvents.scala
new file mode 100644
index 0000000..deebb65
--- /dev/null
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/events/IUDEvents.scala
@@ -0,0 +1,62 @@
+/*
+ * 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.events
+
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+
+/**
+ *
+ * @param carbonTable
+ */
+case class UpdateTablePreEvent(carbonTable: CarbonTable) extends Event with UpdateTableEventInfo
+
+
+/**
+ *
+ * @param carbonTable
+ */
+case class UpdateTablePostEvent(carbonTable: CarbonTable) extends Event with UpdateTableEventInfo
+
+
+/**
+ *
+ * @param carbonTable
+ */
+case class UpdateTableAbortEvent(carbonTable: CarbonTable) extends Event with UpdateTableEventInfo
+
+/**
+ *
+ * @param carbonTable
+ */
+case class DeleteFromTablePreEvent(carbonTable: CarbonTable)
+  extends Event with DeleteFromTableEventInfo
+
+
+/**
+ *
+ * @param carbonTable
+ */
+case class DeleteFromTablePostEvent(carbonTable: CarbonTable)
+  extends Event with DeleteFromTableEventInfo
+
+
+/**
+ *
+ * @param carbonTable
+ */
+case class DeleteFromTableAbortEvent(carbonTable: CarbonTable)
+  extends Event with DeleteFromTableEventInfo

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4aa0f493/integration/spark-common/src/main/scala/org/apache/carbondata/events/LoadEvents.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/events/LoadEvents.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/events/LoadEvents.scala
new file mode 100644
index 0000000..e3833d8
--- /dev/null
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/events/LoadEvents.scala
@@ -0,0 +1,46 @@
+/*
+ * 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.events
+
+import org.apache.spark.sql.SparkSession
+
+import org.apache.carbondata.core.metadata.CarbonTableIdentifier
+import org.apache.carbondata.processing.loading.model.CarbonLoadModel
+
+/**
+ * Class for handling operations before start of a load process.
+ * Example usage: For validation purpose
+ */
+case class LoadTablePreExecutionEvent(sparkSession: SparkSession,
+    carbonTableIdentifier: CarbonTableIdentifier,
+    carbonLoadModel: CarbonLoadModel) extends Event with LoadEventInfo
+
+/**
+ * Class for handling operations after data load completion and before final
+ * commit of load operation. Example usage: For loading pre-aggregate tables
+ */
+case class LoadTablePostExecutionEvent(sparkSession: SparkSession,
+    carbonTableIdentifier: CarbonTableIdentifier,
+    carbonLoadModel: CarbonLoadModel) extends Event with LoadEventInfo
+
+/**
+ * Class for handling clean up in case of any failure and abort the operation.
+ */
+case class LoadTableAbortExecutionEvent(sparkSession: SparkSession,
+    carbonTableIdentifier: CarbonTableIdentifier,
+    carbonLoadModel: CarbonLoadModel) extends Event with LoadEventInfo

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4aa0f493/integration/spark-common/src/main/scala/org/apache/carbondata/events/LookupRelationEvents.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/events/LookupRelationEvents.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/events/LookupRelationEvents.scala
new file mode 100644
index 0000000..178ff58
--- /dev/null
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/events/LookupRelationEvents.scala
@@ -0,0 +1,51 @@
+/*
+ * 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.events
+
+import org.apache.spark.sql.SparkSession
+
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+
+/**
+ *
+ * @param carbonTable
+ * @param sparkSession
+ */
+case class LookupRelationPreEvent(
+    carbonTable: CarbonTable,
+    sparkSession: SparkSession) extends Event with LookupRelationEventInfo
+
+
+/**
+ *
+ * @param carbonTable
+ * @param sparkSession
+ */
+case class LookupRelationPostEvent(
+    carbonTable: CarbonTable,
+    sparkSession: SparkSession) extends Event with LookupRelationEventInfo
+
+
+/**
+ *
+ * @param carbonTable
+ * @param sparkSession
+ */
+case class LookupRelationAbortEvent(
+    carbonTable: CarbonTable,
+    sparkSession: SparkSession) extends Event with LookupRelationEventInfo

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4aa0f493/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/Compactor.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/Compactor.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/Compactor.scala
index cb25756..adc71cf 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/Compactor.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/Compactor.scala
@@ -22,9 +22,9 @@ import scala.collection.JavaConverters._
 import org.apache.spark.sql.execution.command.{CarbonMergerMapping, CompactionCallableModel}
 
 import org.apache.carbondata.common.logging.LogServiceFactory
-import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.mutate.CarbonUpdateUtil
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager
+import org.apache.carbondata.events._
 import org.apache.carbondata.processing.merger.{CarbonDataMergerUtil, CompactionType}
 import org.apache.carbondata.spark.MergeResultImpl
 import org.apache.carbondata.spark.util.CommonUtil
@@ -68,6 +68,12 @@ object Compactor {
     carbonLoadModel.setStorePath(carbonMergerMapping.hdfsStoreLocation)
     carbonLoadModel.setLoadMetadataDetails(
       SegmentStatusManager.readLoadMetadata(carbonTable.getMetaDataFilepath).toList.asJava)
+    // trigger event for compaction
+    val operationContext = new OperationContext
+    val alterTableCompactionPreEvent: AlterTableCompactionPreEvent =
+      AlterTableCompactionPreEvent(carbonTable, carbonLoadModel, mergedLoadName, sc)
+    OperationListenerBus.getInstance.fireEvent(alterTableCompactionPreEvent, operationContext)
+
     var execInstance = "1"
     // in case of non dynamic executor allocation, number of executors are fixed.
     if (sc.sparkContext.getConf.contains("spark.executor.instances")) {
@@ -110,6 +116,12 @@ object Compactor {
     if (finalMergeStatus) {
       val mergedLoadNumber = CarbonDataMergerUtil.getLoadNumberFromLoadName(mergedLoadName)
       CommonUtil.mergeIndexFiles(sc.sparkContext, Seq(mergedLoadNumber), storePath, carbonTable)
+
+      // trigger event for compaction
+      val alterTableCompactionPostEvent: AlterTableCompactionPostEvent =
+        AlterTableCompactionPostEvent(carbonTable, carbonLoadModel, mergedLoadName, sc)
+      OperationListenerBus.getInstance.fireEvent(alterTableCompactionPostEvent, operationContext)
+
       val endTime = System.nanoTime()
       logger.info(s"time taken to merge $mergedLoadName is ${ endTime - startTime }")
       val statusFileUpdation =

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4aa0f493/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
index 47d7c95..1083669 100644
--- a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
@@ -52,6 +52,7 @@ import org.apache.carbondata.core.scan.partition.PartitionUtil
 import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
 import org.apache.carbondata.core.util.{ByteUtil, CarbonProperties}
 import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.events.{LoadTablePostExecutionEvent, OperationContext, OperationListenerBus}
 import org.apache.carbondata.processing.exception.DataLoadingException
 import org.apache.carbondata.processing.loading.FailureCauses
 import org.apache.carbondata.processing.loading.csvinput.BlockDetails
@@ -267,6 +268,7 @@ object CarbonDataRDDFactory {
       updateModel: Option[UpdateTableModel] = None
   ): Unit = {
     val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
+    val operationContext = new OperationContext
     // for handling of the segment Merging.
 
     LOGGER.audit(s"Data load request has been received for table" +
@@ -475,6 +477,13 @@ object CarbonDataRDDFactory {
         throw new Exception("No Data to load")
       }
       writeDictionary(carbonLoadModel, result, writeAll = false)
+      // Register a handler here for executing tasks required before committing
+      // the load operation to a table status file
+      val loadTablePostExecutionEvent: LoadTablePostExecutionEvent =
+      LoadTablePostExecutionEvent(sqlContext.sparkSession,
+        carbonTable.getCarbonTableIdentifier,
+        carbonLoadModel)
+      OperationListenerBus.getInstance.fireEvent(loadTablePostExecutionEvent, operationContext)
       updateTableStatus(status, carbonLoadModel, loadStatus, overwriteTable)
 
       if (CarbonCommonConstants.STORE_LOADSTATUS_PARTIAL_SUCCESS.equals(loadStatus)) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4aa0f493/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 9d10ea0..b324b10 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
@@ -26,6 +26,7 @@ import org.apache.spark.sql.internal.CarbonSQLConf
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.util.{CarbonProperties, CarbonSessionInfo, SessionParams, ThreadLocalSessionInfo}
+import org.apache.carbondata.events.{CarbonEnvInitPreEvent, OperationContext, OperationListenerBus}
 import org.apache.carbondata.spark.rdd.SparkReadSupport
 import org.apache.carbondata.spark.readsupport.SparkRowReadSupportImpl
 
@@ -70,6 +71,11 @@ class CarbonEnv {
           properties.addProperty(CarbonCommonConstants.STORE_LOCATION, storePath)
         }
         LOGGER.info(s"carbon env initial: $storePath")
+        // trigger event for CarbonEnv init
+        val carbonEnvInitPreEvent: CarbonEnvInitPreEvent =
+          CarbonEnvInitPreEvent(sparkSession, storePath)
+        OperationListenerBus.getInstance.fireEvent(carbonEnvInitPreEvent)
+
         CarbonMetaStoreFactory.createCarbonMetaStore(sparkSession.conf)
       }
       CarbonProperties.getInstance.addProperty(CarbonCommonConstants.IS_DRIVER_INSTANCE, "true")

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4aa0f493/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 5f70771..5905493 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,13 +21,15 @@ 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.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, CarbonTableIdentifier}
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonMetadata, CarbonTableIdentifier}
 import org.apache.carbondata.core.util.CarbonUtil
 import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.events.{DropTablePostEvent, DropTablePreEvent, OperationContext, OperationListenerBus}
 
 case class CarbonDropTableCommand(
     ifExistsSet: Boolean,
@@ -59,8 +61,26 @@ case class CarbonDropTableCommand(
       }
       LOGGER.audit(s"Deleting table [$tableName] under database [$dbName]")
 
+      // fires the event before dropping main table
+      val carbonTable = CarbonMetadata.getInstance.getCarbonTable(dbName + "_" + tableName)
+      val operationContext = new OperationContext
+      val dropTablePreEvent: DropTablePreEvent =
+        DropTablePreEvent(
+          carbonTable,
+          ifExistsSet,
+          sparkSession)
+      OperationListenerBus.getInstance.fireEvent(dropTablePreEvent, operationContext)
+
       CarbonEnv.getInstance(sparkSession).carbonMetastore
         .dropTable(tableIdentifier.getTablePath, identifier)(sparkSession)
+
+      val dropTablePostEvent: DropTablePostEvent =
+        DropTablePostEvent(
+          carbonTable,
+          ifExistsSet,
+          sparkSession)
+      OperationListenerBus.getInstance.fireEvent(dropTablePreEvent, operationContext)
+
       LOGGER.audit(s"Deleted table [$tableName] under database [$dbName]")
     } catch {
       case ex: Exception =>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4aa0f493/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CleanFilesCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CleanFilesCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CleanFilesCommand.scala
index 9406335..1b16b88 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CleanFilesCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CleanFilesCommand.scala
@@ -22,6 +22,7 @@ import org.apache.spark.sql.execution.command.{Checker, DataProcessCommand, Runn
 import org.apache.spark.sql.hive.CarbonRelation
 
 import org.apache.carbondata.api.CarbonStore
+import org.apache.carbondata.events.{CleanFilesPostEvent, CleanFilesPreEvent, OperationContext, OperationListenerBus}
 
 case class CleanFilesCommand(
     databaseNameOp: Option[String],
@@ -46,12 +47,22 @@ case class CleanFilesCommand(
       val relation = catalog
         .lookupRelation(databaseNameOp, tableName)(sparkSession).asInstanceOf[CarbonRelation]
       val carbonTable = relation.tableMeta.carbonTable
+      val cleanFilesPreEvent: CleanFilesPreEvent =
+        CleanFilesPreEvent(carbonTable,
+          sparkSession)
+      OperationListenerBus.getInstance.fireEvent(cleanFilesPreEvent)
+
       CarbonStore.cleanFiles(
         GetDB.getDatabaseName(databaseNameOp, sparkSession),
         tableName,
         relation.asInstanceOf[CarbonRelation].tableMeta.storePath,
         carbonTable,
         forceTableClean)
+
+      val cleanFilesPostEvent: CleanFilesPostEvent =
+        CleanFilesPostEvent(carbonTable,
+          sparkSession)
+      OperationListenerBus.getInstance.fireEvent(cleanFilesPreEvent)
     }
     Seq.empty
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4aa0f493/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/DeleteLoadByIdCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/DeleteLoadByIdCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/DeleteLoadByIdCommand.scala
index 1ea4508..9ea4018 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/DeleteLoadByIdCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/DeleteLoadByIdCommand.scala
@@ -22,6 +22,7 @@ import org.apache.spark.sql.execution.command.{Checker, DataProcessCommand, Runn
 import org.apache.spark.sql.hive.CarbonRelation
 
 import org.apache.carbondata.api.CarbonStore
+import org.apache.carbondata.events.{DeleteSegmentByIdPostEvent, DeleteSegmentByIdPreEvent, OperationContext, OperationListenerBus}
 
 case class DeleteLoadByIdCommand(
     loadIds: Seq[String],
@@ -37,12 +38,26 @@ case class DeleteLoadByIdCommand(
     val carbonTable = CarbonEnv.getInstance(sparkSession).carbonMetastore.
       lookupRelation(databaseNameOp, tableName)(sparkSession).asInstanceOf[CarbonRelation].
       tableMeta.carbonTable
+    val operationContext = new OperationContext
+
+    val deleteSegmentByIdPreEvent: DeleteSegmentByIdPreEvent =
+      DeleteSegmentByIdPreEvent(carbonTable,
+        loadIds,
+        sparkSession)
+    OperationListenerBus.getInstance.fireEvent(deleteSegmentByIdPreEvent, operationContext)
+
     CarbonStore.deleteLoadById(
       loadIds,
       GetDB.getDatabaseName(databaseNameOp, sparkSession),
       tableName,
       carbonTable
     )
+
+    val deleteSegmentPostEvent: DeleteSegmentByIdPostEvent =
+      DeleteSegmentByIdPostEvent(carbonTable,
+        loadIds,
+        sparkSession)
+    OperationListenerBus.getInstance.fireEvent(deleteSegmentByIdPreEvent, operationContext)
     Seq.empty
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4aa0f493/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/DeleteLoadByLoadDateCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/DeleteLoadByLoadDateCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/DeleteLoadByLoadDateCommand.scala
index 3d06b18..58d8236 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/DeleteLoadByLoadDateCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/DeleteLoadByLoadDateCommand.scala
@@ -22,6 +22,7 @@ import org.apache.spark.sql.execution.command.{Checker, DataProcessCommand, Runn
 import org.apache.spark.sql.hive.CarbonRelation
 
 import org.apache.carbondata.api.CarbonStore
+import org.apache.carbondata.events.{DeleteSegmentByDatePostEvent, DeleteSegmentByDatePreEvent, OperationContext, OperationListenerBus}
 
 case class DeleteLoadByLoadDateCommand(
     databaseNameOp: Option[String],
@@ -39,12 +40,26 @@ case class DeleteLoadByLoadDateCommand(
     val carbonTable = CarbonEnv.getInstance(sparkSession).carbonMetastore.
       lookupRelation(databaseNameOp, tableName)(sparkSession).asInstanceOf[CarbonRelation].
       tableMeta.carbonTable
+    val operationContext = new OperationContext
+    val deleteSegmentByDatePreEvent: DeleteSegmentByDatePreEvent =
+      DeleteSegmentByDatePreEvent(carbonTable,
+        loadDate,
+        sparkSession)
+    OperationListenerBus.getInstance.fireEvent(deleteSegmentByDatePreEvent, operationContext)
+
     CarbonStore.deleteLoadByDate(
       loadDate,
       GetDB.getDatabaseName(databaseNameOp, sparkSession),
       tableName,
       carbonTable
     )
+
+    val deleteSegmentPostEvent: DeleteSegmentByDatePostEvent =
+      DeleteSegmentByDatePostEvent(carbonTable,
+        loadDate,
+        sparkSession)
+    OperationListenerBus.getInstance.fireEvent(deleteSegmentByDatePreEvent, operationContext)
+
     Seq.empty
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4aa0f493/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForDeleteCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForDeleteCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForDeleteCommand.scala
index af971d0..764deb7 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForDeleteCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForDeleteCommand.scala
@@ -25,8 +25,8 @@ import org.apache.spark.sql.hive.CarbonRelation
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.locks.{CarbonLockFactory, CarbonLockUtil, LockUsage}
 import org.apache.carbondata.core.mutate.CarbonUpdateUtil
+import org.apache.carbondata.events.{DeleteFromTablePostEvent, DeleteFromTablePreEvent, OperationContext, OperationListenerBus}
 import org.apache.carbondata.processing.loading.FailureCauses
-
 /**
  * IUD update delete and compaction framework.
  *
@@ -52,6 +52,13 @@ private[sql] case class ProjectForDeleteCommand(
       .lookupRelation(DeleteExecution.getTableIdentifier(identifier))(sparkSession).
       asInstanceOf[CarbonRelation]
     val carbonTable = relation.tableMeta.carbonTable
+
+    // trigger event for Delete from table
+    val operationContext = new OperationContext
+    val deleteFromTablePreEvent: DeleteFromTablePreEvent =
+      DeleteFromTablePreEvent(carbonTable)
+    OperationListenerBus.getInstance.fireEvent(deleteFromTablePreEvent, operationContext)
+
     val metadataLock = CarbonLockFactory
       .getCarbonLockObj(carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
         LockUsage.METADATA_LOCK)
@@ -76,6 +83,11 @@ private[sql] case class ProjectForDeleteCommand(
         // call IUD Compaction.
         HorizontalCompaction.tryHorizontalCompaction(sparkSession, relation,
           isUpdateOperation = false)
+
+        // trigger post event for Delete from table
+        val deleteFromTablePostEvent: DeleteFromTablePostEvent =
+          DeleteFromTablePostEvent(carbonTable)
+        OperationListenerBus.getInstance.fireEvent(deleteFromTablePostEvent, operationContext)
       }
     } catch {
       case e: HorizontalCompactionException =>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4aa0f493/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForUpdateCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForUpdateCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForUpdateCommand.scala
index faeb3af..e48693b 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForUpdateCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForUpdateCommand.scala
@@ -31,6 +31,7 @@ import org.apache.carbondata.core.locks.{CarbonLockFactory, CarbonLockUtil, Lock
 import org.apache.carbondata.core.mutate.CarbonUpdateUtil
 import org.apache.carbondata.core.util.CarbonProperties
 import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.events.{OperationContext, OperationListenerBus, UpdateTablePostEvent, UpdateTablePreEvent}
 import org.apache.carbondata.processing.loading.FailureCauses
 
 private[sql] case class ProjectForUpdateCommand(
@@ -58,6 +59,13 @@ private[sql] case class ProjectForUpdateCommand(
       .lookupRelation(DeleteExecution.getTableIdentifier(tableIdentifier))(sparkSession).
       asInstanceOf[CarbonRelation]
     val carbonTable = relation.tableMeta.carbonTable
+
+    // trigger event for Update table
+    val operationContext = new OperationContext
+    val updateTablePreEvent: UpdateTablePreEvent =
+      UpdateTablePreEvent(carbonTable)
+    OperationListenerBus.getInstance.fireEvent(updateTablePreEvent, operationContext)
+
     val metadataLock = CarbonLockFactory
       .getCarbonLockObj(carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
         LockUsage.METADATA_LOCK)
@@ -111,6 +119,11 @@ private[sql] case class ProjectForUpdateCommand(
 
       // Do IUD Compaction.
       HorizontalCompaction.tryHorizontalCompaction(sparkSession, relation, isUpdateOperation = true)
+
+      // trigger event for Update table
+      val updateTablePostEvent: UpdateTablePostEvent =
+        UpdateTablePostEvent(carbonTable)
+      OperationListenerBus.getInstance.fireEvent(updateTablePostEvent, operationContext)
     } catch {
       case e: HorizontalCompactionException =>
         LOGGER.error(

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4aa0f493/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDropColumnCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDropColumnCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDropColumnCommand.scala
index 3ac23f7..0b737bf 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDropColumnCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDropColumnCommand.scala
@@ -30,6 +30,7 @@ import org.apache.carbondata.core.locks.{ICarbonLock, LockUsage}
 import org.apache.carbondata.core.metadata.encoder.Encoding
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.events.{AlterTableDropColumnPostEvent, AlterTableDropColumnPreEvent, OperationContext, OperationListenerBus}
 import org.apache.carbondata.format.SchemaEvolutionEntry
 import org.apache.carbondata.spark.rdd.AlterTableDropColumnRDD
 
@@ -99,6 +100,15 @@ private[sql] case class CarbonAlterTableDropColumnCommand(
       if (keyColumnCountToBeDeleted >= totalKeyColumnInSchema) {
         sys.error(s"Alter drop operation failed. AtLeast one key column should exist after drop.")
       }
+
+      val operationContext = new OperationContext
+      // event will be fired before dropping the columns
+      val alterTableDropColumnPreEvent: AlterTableDropColumnPreEvent = AlterTableDropColumnPreEvent(
+        carbonTable,
+        alterTableDropColumnModel,
+        sparkSession)
+      OperationListenerBus.getInstance().fireEvent(alterTableDropColumnPreEvent, operationContext)
+
       // read the latest schema file
       val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getStorePath,
         carbonTable.getCarbonTableIdentifier)
@@ -130,6 +140,15 @@ private[sql] case class CarbonAlterTableDropColumnCommand(
         dictionaryColumns,
         carbonTable.getCarbonTableIdentifier,
         carbonTable.getStorePath).collect()
+
+      // event will be fired before dropping the columns
+      val alterTableDropColumnPostEvent: AlterTableDropColumnPostEvent =
+        AlterTableDropColumnPostEvent(
+        carbonTable,
+        alterTableDropColumnModel,
+        sparkSession)
+      OperationListenerBus.getInstance().fireEvent(alterTableDropColumnPostEvent, operationContext)
+
       LOGGER.info(s"Alter table for drop columns is successful for table $dbName.$tableName")
       LOGGER.audit(s"Alter table for drop columns is successful for table $dbName.$tableName")
     } catch {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4aa0f493/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 88cf212..c000488 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
@@ -31,6 +31,7 @@ import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTable
 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.{AlterTableRenamePostEvent, AlterTableRenamePreEvent, OperationContext, OperationListenerBus}
 import org.apache.carbondata.format.SchemaEvolutionEntry
 import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 
@@ -110,6 +111,14 @@ private[sql] case class CarbonAlterTableRenameCommand(
         tableInfo,
         schemaEvolutionEntry,
         tableMeta.tablePath)(sparkSession)
+      val operationContext = new OperationContext
+      val alterTableRenamePreEvent: AlterTableRenamePreEvent = AlterTableRenamePreEvent(
+        carbonTable,
+        alterTableRenameModel,
+        newTablePath,
+        sparkSession)
+      OperationListenerBus.getInstance().fireEvent(alterTableRenamePreEvent, operationContext)
+
       metastore.removeTableFromMetadata(oldDatabaseName, oldTableName)
       sparkSession.sessionState.asInstanceOf[CarbonSessionState].metadataHive
         .runSqlHive(
@@ -119,6 +128,13 @@ private[sql] case class CarbonAlterTableRenameCommand(
           s"ALTER TABLE $oldDatabaseName.$newTableName SET SERDEPROPERTIES" +
           s"('tableName'='$newTableName', " +
           s"'dbName'='$oldDatabaseName', 'tablePath'='$newTablePath')")
+      val alterTableRenamePostEvent: AlterTableRenamePostEvent = AlterTableRenamePostEvent(
+        carbonTable,
+        alterTableRenameModel,
+        newTablePath,
+        sparkSession)
+      OperationListenerBus.getInstance().fireEvent(alterTableRenamePreEvent, operationContext)
+
       sparkSession.catalog.refreshTable(TableIdentifier(newTableName,
         Some(oldDatabaseName)).quotedString)
       LOGGER.audit(s"Table $oldTableName has been successfully renamed to $newTableName")

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4aa0f493/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 16724fc..9822d8f 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
@@ -42,6 +42,7 @@ import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 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.{SchemaEvolutionEntry, TableInfo}
 import org.apache.carbondata.processing.merger.TableMeta
 import org.apache.carbondata.spark.util.CarbonSparkUtil
@@ -129,6 +130,14 @@ class CarbonFileMetastore extends CarbonMetaStore {
         carbonDatasourceHadoopRelation.carbonRelation
       case _ => throw new NoSuchTableException(database, tableIdentifier.table)
     }
+
+    // fire post event after lookup relation
+    val operationContext = new OperationContext
+    val lookupRelationPostEvent: LookupRelationPostEvent =
+      LookupRelationPostEvent(
+        relation.tableMeta.carbonTable,
+        sparkSession)
+    OperationListenerBus.getInstance.fireEvent(lookupRelationPostEvent, operationContext)
     relation
   }
 


[21/49] carbondata git commit: [CARBONDATA-1618]Table comment support for alter table

Posted by ra...@apache.org.
[CARBONDATA-1618]Table comment support for alter table

This closes #1472


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

Branch: refs/heads/fgdatamap
Commit: 46731137579750d8389f3f9c4ec58547457fda2d
Parents: fd28b15
Author: Pawan Malwal <pm...@gmail.com>
Authored: Mon Nov 6 18:10:14 2017 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Wed Nov 8 17:56:54 2017 +0530

----------------------------------------------------------------------
 .../core/constants/CarbonCommonConstants.java   |   5 +
 .../core/metadata/schema/table/TableInfo.java   |  14 --
 .../TestAlterTableWithTableComment.scala        | 130 +++++++++++++++++++
 .../command/carbonTableSchemaCommon.scala       |   1 -
 .../CarbonDescribeFormattedCommand.scala        |   4 +-
 .../command/schema/AlterTableSetCommand.scala   |  44 +++++++
 .../command/schema/AlterTableUnsetCommand.scala |  45 +++++++
 .../sql/execution/strategy/DDLStrategy.scala    |  12 +-
 .../org/apache/spark/util/AlterTableUtil.scala  |  82 ++++++++++++
 9 files changed, 320 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/46731137/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
index 3fed18f..711b237 100644
--- a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
+++ b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
@@ -1374,6 +1374,11 @@ public final class CarbonCommonConstants {
   @CarbonProperty
   public static final String BITSET_PIPE_LINE = "carbon.use.bitset.pipe.line";
 
+  /**
+   * this will be used to provide comment for table
+   */
+  public static final String TABLE_COMMENT = "comment";
+
   public static final String BITSET_PIPE_LINE_DEFAULT = "true";
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/46731137/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 717eada..d1a7e5b 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
@@ -79,12 +79,6 @@ 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;
 
-  // table comment
-  private String tableComment;
-
-  public TableInfo() {
-  }
-
   /**
    * @return the factTable
    */
@@ -163,14 +157,6 @@ public class TableInfo implements Serializable, Writable {
     this.storePath = storePath;
   }
 
-  public String getTableComment() {
-    return tableComment;
-  }
-
-  public void setTableComment(String tableComment) {
-    this.tableComment = tableComment;
-  }
-
   /**
    * to generate the hash code
    */

http://git-wip-us.apache.org/repos/asf/carbondata/blob/46731137/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestAlterTableWithTableComment.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestAlterTableWithTableComment.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestAlterTableWithTableComment.scala
new file mode 100644
index 0000000..b01fe4f
--- /dev/null
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestAlterTableWithTableComment.scala
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.spark.testsuite.createTable
+
+import org.apache.spark.sql.test.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+/**
+  * test functionality for alter table with table comment
+  */
+class TestAlterTableWithTableComment extends QueryTest with BeforeAndAfterAll {
+
+  override def beforeAll {
+    sql("use default")
+    sql("drop table if exists alterTableWithTableComment")
+    sql("drop table if exists alterTableWithoutTableComment")
+    sql("drop table if exists alterTableUnsetTableComment")
+  }
+
+  test("test add table comment using alter table set ") {
+    sql(
+      s"""
+         | create table alterTableWithTableComment(
+         | id int,
+         | name string
+         | )
+         | STORED BY 'carbondata'
+       """.stripMargin
+    )
+
+    val create_result = sql("describe formatted alterTableWithTableComment")
+
+    checkExistence(create_result, true, "Comment:")
+    checkExistence(create_result, false, "This is table comment")
+
+    sql(
+      s"""
+         | alter table alterTableWithTableComment
+         | SET TBLPROPERTIES (
+         | 'comment'='This table comment is added by alter table'
+         | )
+       """.stripMargin
+    )
+
+    val alter_result = sql("describe formatted alterTableWithTableComment")
+
+    checkExistence(alter_result, true, "Comment:")
+    checkExistence(alter_result, true, "This table comment is added by alter table")
+  }
+
+  test("test modifiy table comment using alter table set ") {
+    sql(
+      s"""
+         | create table alterTableWithoutTableComment(
+         | id int,
+         | name string
+         | comment "This is table comment"
+         | )
+         | STORED BY 'carbondata'
+       """.stripMargin
+    )
+
+    sql(
+      s"""
+         | alter table alterTableWithoutTableComment
+         | SET TBLPROPERTIES (
+         | 'comment'='This table comment is modified by alter table set'
+         | )
+       """.stripMargin
+    )
+
+    val alter_result = sql("describe formatted alterTableWithoutTableComment")
+
+    checkExistence(alter_result, true, "Comment:")
+    checkExistence(alter_result, true, "This table comment is modified by alter table set")
+  }
+
+  test("test remove table comment using alter table unset ") {
+    sql(
+      s"""
+         | create table alterTableUnsetTableComment(
+         | id int,
+         | name string
+         | )
+         | comment "This is table comment"
+         | STORED BY 'carbondata'
+       """.stripMargin
+    )
+
+    val create_result = sql("describe formatted alterTableUnsetTableComment")
+
+    checkExistence(create_result, true, "Comment:")
+    checkExistence(create_result, true, "This is table comment")
+
+    sql(
+      s"""
+         | alter table alterTableUnsetTableComment
+         | UNSET TBLPROPERTIES ('comment')
+       """.stripMargin
+    )
+
+    val alter_result = sql("describe formatted alterTableUnsetTableComment")
+
+    checkExistence(alter_result, true, "Comment:")
+    checkExistence(alter_result, false, "This is table comment")
+  }
+
+  override def afterAll: Unit = {
+    sql("use default")
+    sql("drop table if exists alterTableWithTableComment")
+    sql("drop table if exists alterTableWithoutTableComment")
+    sql("drop table if exists alterTableUnsetTableComment")
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/46731137/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 fba3085..1188b59 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
@@ -551,7 +551,6 @@ class TableNewProcessor(cm: TableModel) {
     tableInfo.setTableUniqueName(cm.databaseName + "_" + cm.tableName)
     tableInfo.setLastUpdatedTime(System.currentTimeMillis())
     tableInfo.setFactTable(tableSchema)
-    tableInfo.setTableComment(cm.tableComment.getOrElse(""))
     tableInfo
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/46731137/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonDescribeFormattedCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonDescribeFormattedCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonDescribeFormattedCommand.scala
index e57f490..519fbea 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonDescribeFormattedCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonDescribeFormattedCommand.scala
@@ -106,7 +106,9 @@ private[sql] case class CarbonDescribeFormattedCommand(
     results ++= Seq(("CARBON Store Path: ", relation.tableMeta.storePath, ""))
     val carbonTable = relation.tableMeta.carbonTable
     // Carbon table support table comment
-    results ++= Seq(("Comment: ", carbonTable.getTableInfo.getTableComment, ""))
+    val tableComment = carbonTable.getTableInfo.getFactTable.getTableProperties
+      .getOrDefault(CarbonCommonConstants.TABLE_COMMENT, "")
+    results ++= Seq(("Comment: ", tableComment, ""))
     results ++= Seq(("Table Block Size : ", carbonTable.getBlockSizeInMB + " MB", ""))
     results ++= Seq(("SORT_SCOPE", carbonTable.getTableInfo.getFactTable
       .getTableProperties.getOrDefault("sort_scope", CarbonCommonConstants

http://git-wip-us.apache.org/repos/asf/carbondata/blob/46731137/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableSetCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableSetCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableSetCommand.scala
new file mode 100644
index 0000000..afbf8f6
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableSetCommand.scala
@@ -0,0 +1,44 @@
+/*
+ * 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.schema
+
+import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.execution.command._
+import org.apache.spark.sql.hive.{CarbonRelation, CarbonSessionState}
+import org.apache.spark.util.AlterTableUtil
+
+import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
+import org.apache.carbondata.format.TableInfo
+
+private[sql] case class AlterTableSetCommand(val tableIdentifier: TableIdentifier,
+                                             val properties: Map[String, String],
+                                             val isView: Boolean)
+  extends RunnableCommand with SchemaProcessCommand {
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    processSchema(sparkSession)
+  }
+
+  override def processSchema(sparkSession: SparkSession): Seq[Row] = {
+    val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+    AlterTableUtil.modifyTableComment(tableIdentifier, properties, Nil,
+      true)(sparkSession, sparkSession.sessionState.asInstanceOf[CarbonSessionState])
+    Seq.empty
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/46731137/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableUnsetCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableUnsetCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableUnsetCommand.scala
new file mode 100644
index 0000000..0bcae1e
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableUnsetCommand.scala
@@ -0,0 +1,45 @@
+/*
+ * 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.schema
+
+import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.execution.command._
+import org.apache.spark.sql.hive.{CarbonRelation, CarbonSessionState}
+import org.apache.spark.util.AlterTableUtil
+
+import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
+import org.apache.carbondata.format.TableInfo
+
+private[sql] case class AlterTableUnsetCommand(val tableIdentifier: TableIdentifier,
+                                               val propKeys: Seq[String],
+                                               val ifExists: Boolean,
+                                               val isView: Boolean)
+  extends RunnableCommand with SchemaProcessCommand {
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    processSchema(sparkSession)
+  }
+
+  override def processSchema(sparkSession: SparkSession): Seq[Row] = {
+    val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+    AlterTableUtil.modifyTableComment(tableIdentifier, Map.empty[String, String],
+      propKeys, false)(sparkSession, sparkSession.sessionState.asInstanceOf[CarbonSessionState])
+    Seq.empty
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/46731137/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala
index bf13e41..e39ba73 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala
@@ -24,7 +24,7 @@ import org.apache.spark.sql.execution.{SparkPlan, SparkStrategy}
 import org.apache.spark.sql.execution.command._
 import org.apache.spark.sql.execution.command.management.{AlterTableCompactionCommand, CarbonShowLoadsCommand, LoadTableByInsertCommand, LoadTableCommand}
 import org.apache.spark.sql.execution.command.partition.ShowCarbonPartitionsCommand
-import org.apache.spark.sql.execution.command.schema.{CarbonAlterTableAddColumnCommand, CarbonAlterTableDataTypeChangeCommand, CarbonAlterTableDropColumnCommand, CarbonAlterTableRenameCommand}
+import org.apache.spark.sql.execution.command.schema._
 import org.apache.spark.sql.hive.execution.command.{CarbonDropDatabaseCommand, CarbonResetCommand, CarbonSetCommand}
 
 import org.apache.carbondata.core.util.CarbonUtil
@@ -156,6 +156,16 @@ class DDLStrategy(sparkSession: SparkSession) extends SparkStrategy {
         val cmd =
           CreateDataSourceTableCommand(updatedCatalog, ignoreIfExists = mode == SaveMode.Ignore)
         ExecutedCommandExec(cmd) :: Nil
+      case AlterTableSetPropertiesCommand(tableName, properties, isView)
+        if (CarbonEnv.getInstance(sparkSession).carbonMetastore
+        .tableExists(tableName)(sparkSession)) => {
+        ExecutedCommandExec(AlterTableSetCommand(tableName, properties, isView)) :: Nil
+      }
+      case AlterTableUnsetPropertiesCommand(tableName, propKeys, ifExists, isView)
+        if (CarbonEnv.getInstance(sparkSession).carbonMetastore
+        .tableExists(tableName)(sparkSession)) => {
+        ExecutedCommandExec(AlterTableUnsetCommand(tableName, propKeys, ifExists, isView)) :: Nil
+      }
       case _ => Nil
     }
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/46731137/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 74f4dd0..44f5a36 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
@@ -18,6 +18,7 @@
 package org.apache.spark.util
 
 import scala.collection.JavaConverters._
+import scala.collection.mutable
 import scala.collection.mutable.ListBuffer
 
 import org.apache.spark.SparkConf
@@ -30,10 +31,13 @@ import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.locks.{CarbonLockUtil, ICarbonLock}
+import org.apache.carbondata.core.locks.{ICarbonLock, LockUsage}
 import org.apache.carbondata.core.metadata.CarbonTableIdentifier
+import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.util.path.CarbonStorePath
 import org.apache.carbondata.format.{SchemaEvolutionEntry, TableInfo}
+import org.apache.carbondata.format.TableInfo
 
 object AlterTableUtil {
 
@@ -317,4 +321,82 @@ object AlterTableUtil {
     }
   }
 
+  /**
+   * This method add/modify the table comments.
+   *
+   * @param tableIdentifier
+   * @param properties
+   * @param propKeys
+   * @param set
+   * @param sparkSession
+   * @param sessionState
+   */
+  def modifyTableComment(tableIdentifier: TableIdentifier, properties: Map[String, String],
+                         propKeys: Seq[String], set: Boolean)
+                        (sparkSession: SparkSession, sessionState: CarbonSessionState): Unit = {
+    val tableName = tableIdentifier.table
+    val dbName = tableIdentifier.database.getOrElse(sparkSession.catalog.currentDatabase)
+    LOGGER.audit(s"Alter table comment request has been received for $dbName.$tableName")
+    val locksToBeAcquired = List(LockUsage.METADATA_LOCK, LockUsage.COMPACTION_LOCK)
+    var locks = List.empty[ICarbonLock]
+    var timeStamp = 0L
+    var newCols = Seq[org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema]()
+    var carbonTable: CarbonTable = null
+    try {
+      locks = AlterTableUtil
+        .validateTableAndAcquireLock(dbName, tableName, locksToBeAcquired)(sparkSession)
+      val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
+      carbonTable = metastore
+        .lookupRelation(Some(dbName), tableName)(sparkSession).asInstanceOf[CarbonRelation]
+        .tableMeta.carbonTable
+      // get the latest carbon table
+      // 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)
+      val schemaEvolutionEntry = new org.apache.carbondata.core.metadata.schema.SchemaEvolutionEntry
+      schemaEvolutionEntry.setTimeStamp(timeStamp)
+      val thriftTable = schemaConverter
+        .fromWrapperToExternalTableInfo(wrapperTableInfo, dbName, tableName)
+      val tblPropertiesMap: mutable.Map[String, String] =
+        thriftTable.fact_table.getTableProperties.asScala
+      if (set) {
+        //       This overrides old properties and update the comment parameter of thriftTable
+        //       with the newly added/modified comment since thriftTable also holds comment as its
+        //       direct property.
+
+        properties.foreach { x =>
+          if (x._1.equalsIgnoreCase(CarbonCommonConstants.TABLE_COMMENT)) {
+            tblPropertiesMap.put(x._1, x._2)
+          }
+        }
+      } else {
+        // This removes the comment parameter from thriftTable
+        // since thriftTable also holds comment as its property.
+        propKeys.foreach { x =>
+          if (x.equalsIgnoreCase(CarbonCommonConstants.TABLE_COMMENT)) {
+            tblPropertiesMap.remove(x)
+          }
+        }
+      }
+      updateSchemaInfo(carbonTable,
+        schemaConverter.fromWrapperToExternalSchemaEvolutionEntry(schemaEvolutionEntry),
+        thriftTable)(sparkSession, sessionState)
+      LOGGER.info(s"Alter table comment is successful for table $dbName.$tableName")
+      LOGGER.audit(s"Alter table comment is successful for table $dbName.$tableName")
+    } catch {
+      case e: Exception =>
+        LOGGER.error(e, "Alter table comment failed")
+        sys.error(s"Alter table comment operation failed: ${e.getMessage}")
+    } finally {
+      // release lock after command execution completion
+      AlterTableUtil.releaseLocks(locks)
+    }
+  }
 }


[38/49] 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/fgdatamap
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")
+
+  }
+}


[27/49] carbondata git commit: [CARBONDATA-1624]Set the default value of 'carbon.number.of.cores.while.loading' as per the spark conf 'spark.executor.cores'

Posted by ra...@apache.org.
[CARBONDATA-1624]Set the default value of 'carbon.number.of.cores.while.loading' as per the spark conf 'spark.executor.cores'

1.Use 'spark.executor.cores' as the default value for 'carbon.number.of.cores.while.loading'
2.Use 'CarbonProperties.getNumberOfCores()' to get 'carbon.number.of.cores.while.loading' uniformly

This closes #1455


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

Branch: refs/heads/fgdatamap
Commit: 9e9d68988e29a9c3a2520189d822835562f4a34d
Parents: f3b507c
Author: Zhang Zhichao <44...@qq.com>
Authored: Tue Oct 31 10:55:38 2017 +0800
Committer: Jacky Li <ja...@qq.com>
Committed: Thu Nov 9 22:44:38 2017 +0800

----------------------------------------------------------------------
 .../dictionary/AbstractDictionaryCache.java     |  8 +-------
 .../generator/TableDictionaryGenerator.java     | 10 +---------
 .../reader/CarbonDeleteFilesDataReader.java     |  8 +-------
 .../carbondata/core/util/CarbonProperties.java  |  6 ++++--
 .../testsuite/datamap/DataMapWriterSuite.scala  |  7 ++++---
 .../command/management/LoadTableCommand.scala   | 20 ++++++++++++++++++++
 .../sort/sortdata/SortParameters.java           | 20 ++------------------
 .../store/CarbonFactDataHandlerColumnar.java    | 11 +----------
 8 files changed, 34 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/9e9d6898/core/src/main/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCache.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCache.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCache.java
index 4046364..e145cb8 100644
--- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCache.java
+++ b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCache.java
@@ -72,13 +72,7 @@ public abstract class AbstractDictionaryCache<K extends DictionaryColumnUniqueId
    * max number of threads for a job
    */
   private void initThreadPoolSize() {
-    try {
-      thread_pool_size = Integer.parseInt(CarbonProperties.getInstance()
-          .getProperty(CarbonCommonConstants.NUM_CORES_LOADING,
-              CarbonCommonConstants.NUM_CORES_DEFAULT_VAL));
-    } catch (NumberFormatException e) {
-      thread_pool_size = Integer.parseInt(CarbonCommonConstants.NUM_CORES_DEFAULT_VAL);
-    }
+    thread_pool_size = CarbonProperties.getInstance().getNumberOfCores();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/9e9d6898/core/src/main/java/org/apache/carbondata/core/dictionary/generator/TableDictionaryGenerator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/dictionary/generator/TableDictionaryGenerator.java b/core/src/main/java/org/apache/carbondata/core/dictionary/generator/TableDictionaryGenerator.java
index f08ba1f..905d2fa 100644
--- a/core/src/main/java/org/apache/carbondata/core/dictionary/generator/TableDictionaryGenerator.java
+++ b/core/src/main/java/org/apache/carbondata/core/dictionary/generator/TableDictionaryGenerator.java
@@ -25,7 +25,6 @@ import java.util.concurrent.TimeUnit;
 
 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.devapi.BiDictionary;
 import org.apache.carbondata.core.devapi.DictionaryGenerationException;
 import org.apache.carbondata.core.devapi.DictionaryGenerator;
@@ -72,14 +71,7 @@ public class TableDictionaryGenerator
   }
 
   @Override public void writeDictionaryData() {
-    int numOfCores = 1;
-    try {
-      numOfCores = Integer.parseInt(CarbonProperties.getInstance()
-              .getProperty(CarbonCommonConstants.NUM_CORES_LOADING,
-                      CarbonCommonConstants.NUM_CORES_DEFAULT_VAL));
-    } catch (NumberFormatException e) {
-      numOfCores = Integer.parseInt(CarbonCommonConstants.NUM_CORES_DEFAULT_VAL);
-    }
+    int numOfCores = CarbonProperties.getInstance().getNumberOfCores();
     long start = System.currentTimeMillis();
     ExecutorService executorService = Executors.newFixedThreadPool(numOfCores);
     for (final DictionaryGenerator generator : columnMap.values()) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/9e9d6898/core/src/main/java/org/apache/carbondata/core/reader/CarbonDeleteFilesDataReader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/reader/CarbonDeleteFilesDataReader.java b/core/src/main/java/org/apache/carbondata/core/reader/CarbonDeleteFilesDataReader.java
index 6739b41..cc6e53f 100644
--- a/core/src/main/java/org/apache/carbondata/core/reader/CarbonDeleteFilesDataReader.java
+++ b/core/src/main/java/org/apache/carbondata/core/reader/CarbonDeleteFilesDataReader.java
@@ -65,13 +65,7 @@ public class CarbonDeleteFilesDataReader {
    * max number of threads for a job
    */
   private void initThreadPoolSize() {
-    try {
-      thread_pool_size = Integer.parseInt(CarbonProperties.getInstance()
-          .getProperty(CarbonCommonConstants.NUM_CORES_LOADING,
-              CarbonCommonConstants.NUM_CORES_DEFAULT_VAL));
-    } catch (NumberFormatException e) {
-      thread_pool_size = Integer.parseInt(CarbonCommonConstants.NUM_CORES_DEFAULT_VAL);
-    }
+    thread_pool_size = CarbonProperties.getInstance().getNumberOfCores();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/9e9d6898/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java b/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java
index cdd6183..678a6f7 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java
@@ -669,9 +669,11 @@ public final class CarbonProperties {
     int numberOfCores;
     try {
       numberOfCores = Integer.parseInt(CarbonProperties.getInstance()
-          .getProperty(CarbonCommonConstants.NUM_CORES_LOADING,
-              CarbonCommonConstants.NUM_CORES_DEFAULT_VAL));
+          .getProperty(CarbonCommonConstants.NUM_CORES_LOADING));
     } catch (NumberFormatException exc) {
+      LOGGER.error("Configured value for property " + CarbonCommonConstants.NUM_CORES_LOADING
+          + " is wrong. Falling back to the default value "
+          + CarbonCommonConstants.NUM_CORES_DEFAULT_VAL);
       numberOfCores = Integer.parseInt(CarbonCommonConstants.NUM_CORES_DEFAULT_VAL);
     }
     return numberOfCores;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/9e9d6898/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala
index ff900ce..888c97d 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala
@@ -18,13 +18,11 @@
 package org.apache.carbondata.spark.testsuite.datamap
 
 import java.util
-
 import scala.collection.JavaConverters._
-
 import org.apache.spark.sql.{DataFrame, SaveMode}
 import org.apache.spark.sql.test.util.QueryTest
 import org.scalatest.BeforeAndAfterAll
-
+import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datamap.dev.{DataMap, DataMapFactory, DataMapWriter}
 import org.apache.carbondata.core.datamap.{DataMapDistributable, DataMapMeta, DataMapStoreManager}
 import org.apache.carbondata.core.datastore.page.ColumnPage
@@ -117,6 +115,9 @@ class DataMapWriterSuite extends QueryTest with BeforeAndAfterAll {
 
     CarbonProperties.getInstance()
       .addProperty("carbon.blockletgroup.size.in.mb", "1")
+    CarbonProperties.getInstance()
+      .addProperty("carbon.number.of.cores.while.loading",
+          CarbonCommonConstants.NUM_CORES_DEFAULT_VAL)
 
     val df = buildTestData(300000)
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/9e9d6898/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 630ee27..bda6829 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
@@ -84,6 +84,26 @@ case class LoadTableCommand(
 
     val carbonProperty: CarbonProperties = CarbonProperties.getInstance()
     carbonProperty.addProperty("zookeeper.enable.lock", "false")
+
+    // get the value of 'spark.executor.cores' from spark conf, default value is 1
+    val sparkExecutorCores = sparkSession.sparkContext.conf.get("spark.executor.cores", "1")
+    // get the value of 'carbon.number.of.cores.while.loading' from carbon properties,
+    // default value is the value of 'spark.executor.cores'
+    val numCoresLoading =
+      try {
+        CarbonProperties.getInstance()
+            .getProperty(CarbonCommonConstants.NUM_CORES_LOADING, sparkExecutorCores)
+      } catch {
+        case exc: NumberFormatException =>
+          LOGGER.error("Configured value for property " + CarbonCommonConstants.NUM_CORES_LOADING
+              + " is wrong. Falling back to the default value "
+              + sparkExecutorCores)
+          sparkExecutorCores
+      }
+
+    // update the property with new value
+    carbonProperty.addProperty(CarbonCommonConstants.NUM_CORES_LOADING, numCoresLoading)
+
     val optionsFinal = DataLoadingUtil.getDataLoadingOptions(carbonProperty, options)
 
     val tableProperties = relation.tableMeta.carbonTable.getTableInfo

http://git-wip-us.apache.org/repos/asf/carbondata/blob/9e9d6898/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortParameters.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortParameters.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortParameters.java
index 39e1049..4da4c84 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortParameters.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortParameters.java
@@ -418,15 +418,7 @@ public class SortParameters implements Serializable {
     parameters.setTempFileLocation(sortTempDirs);
     LOGGER.info("temp file location: " + StringUtils.join(parameters.getTempFileLocation(), ","));
 
-    int numberOfCores;
-    try {
-      numberOfCores = Integer.parseInt(carbonProperties
-          .getProperty(CarbonCommonConstants.NUM_CORES_LOADING,
-              CarbonCommonConstants.NUM_CORES_DEFAULT_VAL));
-      numberOfCores = numberOfCores / 2;
-    } catch (NumberFormatException exc) {
-      numberOfCores = Integer.parseInt(CarbonCommonConstants.NUM_CORES_DEFAULT_VAL);
-    }
+    int numberOfCores = carbonProperties.getNumberOfCores() / 2;
     parameters.setNumberOfCores(numberOfCores > 0 ? numberOfCores : 1);
 
     parameters.setFileWriteBufferSize(Integer.parseInt(carbonProperties
@@ -539,15 +531,7 @@ public class SortParameters implements Serializable {
     parameters.setTempFileLocation(sortTempDirs);
     LOGGER.info("temp file location: " + StringUtils.join(parameters.getTempFileLocation(), ","));
 
-    int numberOfCores;
-    try {
-      numberOfCores = Integer.parseInt(carbonProperties
-          .getProperty(CarbonCommonConstants.NUM_CORES_LOADING,
-              CarbonCommonConstants.NUM_CORES_DEFAULT_VAL));
-      numberOfCores = numberOfCores / 2;
-    } catch (NumberFormatException exc) {
-      numberOfCores = Integer.parseInt(CarbonCommonConstants.NUM_CORES_DEFAULT_VAL);
-    }
+    int numberOfCores = carbonProperties.getNumberOfCores() / 2;
     parameters.setNumberOfCores(numberOfCores > 0 ? numberOfCores : 1);
 
     parameters.setFileWriteBufferSize(Integer.parseInt(carbonProperties

http://git-wip-us.apache.org/repos/asf/carbondata/blob/9e9d6898/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerColumnar.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerColumnar.java b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerColumnar.java
index 7882cd4..504e7ec 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerColumnar.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerColumnar.java
@@ -223,16 +223,7 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
         numberOfCores = Integer.parseInt(CarbonCommonConstants.NUM_CORES_DEFAULT_VAL);
       }
     } else {
-      try {
-        numberOfCores = Integer.parseInt(CarbonProperties.getInstance()
-            .getProperty(CarbonCommonConstants.NUM_CORES_LOADING,
-                CarbonCommonConstants.NUM_CORES_DEFAULT_VAL));
-      } catch (NumberFormatException exc) {
-        LOGGER.error("Configured value for property " + CarbonCommonConstants.NUM_CORES_LOADING
-            + "is wrong.Falling back to the default value "
-            + CarbonCommonConstants.NUM_CORES_DEFAULT_VAL);
-        numberOfCores = Integer.parseInt(CarbonCommonConstants.NUM_CORES_DEFAULT_VAL);
-      }
+      numberOfCores = CarbonProperties.getInstance().getNumberOfCores();
     }
 
     if (sortScope != null && sortScope.equals(SortScopeOptions.SortScope.GLOBAL_SORT)) {


[25/49] carbondata git commit: [CARBONDATA-1572][Streaming] Support streaming ingest and query

Posted by ra...@apache.org.
[CARBONDATA-1572][Streaming] Support streaming ingest and query

This PR supports streaming ingest from spark structured streaming:
1.row format writer and support to append batch data

2.support StreamSinkProvider and append batch data to row format file

3.row format reader and support to split row format file to small blocks

4.query with streaming row format file.

This closes #1470


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

Branch: refs/heads/fgdatamap
Commit: d7393da9890c2360831d17d23145b78f8da70575
Parents: fa19331
Author: QiangCai <qi...@qq.com>
Authored: Wed Oct 18 11:13:00 2017 +0800
Committer: Jacky Li <ja...@qq.com>
Committed: Thu Nov 9 16:26:53 2017 +0800

----------------------------------------------------------------------
 .../core/constants/CarbonCommonConstants.java   |  10 +
 .../core/datastore/compression/Compressor.java  |   4 +
 .../datastore/compression/SnappyCompressor.java |  13 +
 .../core/datastore/impl/FileFactory.java        |  49 +-
 .../apache/carbondata/core/locks/LockUsage.java |   2 +-
 .../core/metadata/datatype/DataTypes.java       |  37 +-
 .../core/metadata/schema/table/CarbonTable.java |  37 +
 .../carbondata/core/scan/filter/FilterUtil.java |  22 +
 .../filter/executer/AndFilterExecuterImpl.java  |   7 +
 .../executer/ExcludeFilterExecuterImpl.java     |  34 +
 .../scan/filter/executer/FilterExecuter.java    |   4 +
 .../executer/IncludeFilterExecuterImpl.java     |  31 +
 .../filter/executer/OrFilterExecuterImpl.java   |   7 +
 .../RestructureExcludeFilterExecutorImpl.java   |   7 +
 .../RestructureIncludeFilterExecutorImpl.java   |   7 +
 .../executer/RowLevelFilterExecuterImpl.java    |   9 +
 .../filter/executer/TrueFilterExecutor.java     |   5 +
 .../executer/ValueBasedFilterExecuterImpl.java  |   6 +
 .../carbondata/core/scan/model/QueryModel.java  |  49 +-
 .../core/statusmanager/FileFormat.java          |  40 ++
 .../core/statusmanager/LoadMetadataDetails.java |  13 +
 .../statusmanager/SegmentStatusManager.java     |  29 +-
 .../apache/carbondata/core/util/CarbonUtil.java |   8 +
 .../core/util/path/CarbonTablePath.java         |  22 +-
 examples/spark2/pom.xml                         |   5 +
 .../spark2/src/main/resources/streamSample.csv  |   6 +
 .../carbondata/examples/StreamExample.scala     | 213 ++++++
 format/src/main/thrift/carbondata.thrift        |   6 +-
 format/src/main/thrift/carbondata_index.thrift  |   1 +
 .../carbondata/hadoop/CarbonInputFormat.java    |  23 +-
 .../carbondata/hadoop/CarbonInputSplit.java     |  35 +
 .../hadoop/CarbonMultiBlockSplit.java           |  19 +
 .../hadoop/api/CarbonTableInputFormat.java      | 110 ++-
 .../streaming/CarbonStreamInputFormat.java      | 114 ++++
 .../streaming/CarbonStreamOutputFormat.java     |  75 ++
 .../streaming/CarbonStreamRecordReader.java     | 676 +++++++++++++++++++
 .../streaming/CarbonStreamRecordWriter.java     | 305 +++++++++
 .../hadoop/streaming/StreamBlockletReader.java  | 248 +++++++
 .../hadoop/streaming/StreamBlockletWriter.java  | 152 +++++
 .../hadoop/util/CarbonInputFormatUtil.java      |   6 +-
 .../carbondata/hadoop/util/CarbonTypeUtil.java  | 101 +++
 .../hadoop/test/util/StoreCreator.java          |  78 ++-
 .../hive/MapredCarbonInputFormat.java           |   2 +-
 .../presto/CarbonVectorizedRecordReader.java    |   1 +
 .../carbondata/spark/rdd/CarbonScanRDD.scala    |  86 ++-
 integration/spark2/pom.xml                      |   2 +-
 .../sql/CarbonDatasourceHadoopRelation.scala    |   5 +-
 .../org/apache/spark/sql/CarbonSession.scala    |   2 +
 .../org/apache/spark/sql/CarbonSource.scala     |  36 +-
 pom.xml                                         |  11 +-
 .../loading/csvinput/CSVInputFormat.java        |  45 +-
 .../store/writer/AbstractFactDataWriter.java    |   2 +-
 .../util/CarbonDataProcessorUtil.java           |  19 +
 .../processing/util/CarbonLoaderUtil.java       |   2 +-
 streaming/pom.xml                               | 127 ++++
 .../streaming/CarbonStreamException.java        |  32 +
 .../streaming/parser/CSVStreamParserImp.java    |  45 ++
 .../streaming/parser/CarbonStreamParser.java    |  38 ++
 .../streaming/segment/StreamSegment.java        | 373 ++++++++++
 .../streaming/StreamSinkFactory.scala           | 160 +++++
 .../streaming/CarbonAppendableStreamSink.scala  | 292 ++++++++
 .../CarbonStreamingQueryListener.scala          |  67 ++
 62 files changed, 3831 insertions(+), 141 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
index 711b237..17936d9 100644
--- a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
+++ b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
@@ -532,6 +532,16 @@ public final class CarbonCommonConstants {
    * LOAD_STATUS PARTIAL_SUCCESS
    */
   public static final String STORE_LOADSTATUS_PARTIAL_SUCCESS = "Partial Success";
+
+  /**
+   * STORE_LOADSTATUS_STREAMING
+   */
+  public static final String STORE_LOADSTATUS_STREAMING = "Streaming";
+
+  /**
+   * STORE_LOADSTATUS_STREAMING
+   */
+  public static final String STORE_LOADSTATUS_STREAMING_FINISH = "Streaming Finish";
   /**
    * LOAD_STATUS
    */

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/core/src/main/java/org/apache/carbondata/core/datastore/compression/Compressor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/compression/Compressor.java b/core/src/main/java/org/apache/carbondata/core/datastore/compression/Compressor.java
index cd31984..a32651a 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/compression/Compressor.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/compression/Compressor.java
@@ -25,6 +25,8 @@ public interface Compressor {
 
   byte[] compressByte(byte[] unCompInput);
 
+  byte[] compressByte(byte[] unCompInput, int byteSize);
+
   byte[] unCompressByte(byte[] compInput);
 
   byte[] unCompressByte(byte[] compInput, int offset, int length);
@@ -61,5 +63,7 @@ public interface Compressor {
 
   long rawCompress(long inputAddress, int inputSize, long outputAddress) throws IOException;
 
+  long rawUncompress(byte[] input, byte[] output) throws IOException;
+
   int maxCompressedLength(int inputSize);
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/core/src/main/java/org/apache/carbondata/core/datastore/compression/SnappyCompressor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/compression/SnappyCompressor.java b/core/src/main/java/org/apache/carbondata/core/datastore/compression/SnappyCompressor.java
index 4022680..f234f80 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/compression/SnappyCompressor.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/compression/SnappyCompressor.java
@@ -63,6 +63,15 @@ public class SnappyCompressor implements Compressor {
     }
   }
 
+  @Override public byte[] compressByte(byte[] unCompInput, int byteSize) {
+    try {
+      return Snappy.rawCompress(unCompInput, byteSize);
+    } catch (IOException e) {
+      LOGGER.error(e, e.getMessage());
+      return null;
+    }
+  }
+
   @Override public byte[] unCompressByte(byte[] compInput) {
     try {
       return Snappy.uncompress(compInput);
@@ -228,6 +237,10 @@ public class SnappyCompressor implements Compressor {
     return snappyNative.rawCompress(inputAddress, inputSize, outputAddress);
   }
 
+  public long rawUncompress(byte[] input, byte[] output) throws IOException {
+    return snappyNative.rawUncompress(input, 0, input.length, output, 0);
+  }
+
   @Override
   public int maxCompressedLength(int inputSize) {
     return snappyNative.maxCompressedLength(inputSize);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileFactory.java b/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileFactory.java
index 97f0b3f..e4e4ae2 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileFactory.java
@@ -26,6 +26,7 @@ import java.io.FileInputStream;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
+import java.nio.channels.FileChannel;
 import java.util.zip.GZIPInputStream;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
@@ -81,11 +82,9 @@ public final class FileFactory {
   public static FileType getFileType(String path) {
     if (path.startsWith(CarbonCommonConstants.HDFSURL_PREFIX)) {
       return FileType.HDFS;
-    }
-    else if (path.startsWith(CarbonCommonConstants.ALLUXIOURL_PREFIX)) {
+    } else if (path.startsWith(CarbonCommonConstants.ALLUXIOURL_PREFIX)) {
       return FileType.ALLUXIO;
-    }
-    else if (path.startsWith(CarbonCommonConstants.VIEWFSURL_PREFIX)) {
+    } else if (path.startsWith(CarbonCommonConstants.VIEWFSURL_PREFIX)) {
       return FileType.VIEWFS;
     }
     return FileType.LOCAL;
@@ -438,6 +437,48 @@ public final class FileFactory {
   }
 
   /**
+   * this method will truncate the file to the new size.
+   * @param path
+   * @param fileType
+   * @param newSize
+   * @throws IOException
+   */
+  public static void truncateFile(String path, FileType fileType, long newSize) throws IOException {
+    path = path.replace("\\", "/");
+    FileChannel fileChannel = null;
+    switch (fileType) {
+      case LOCAL:
+        path = getUpdatedFilePath(path, fileType);
+        fileChannel = new FileOutputStream(path, true).getChannel();
+        try {
+          fileChannel.truncate(newSize);
+        } finally {
+          if (fileChannel != null) {
+            fileChannel.close();
+          }
+        }
+        return;
+      case HDFS:
+      case ALLUXIO:
+      case VIEWFS:
+        Path pt = new Path(path);
+        FileSystem fs = pt.getFileSystem(configuration);
+        fs.truncate(pt, newSize);
+        return;
+      default:
+        fileChannel = new FileOutputStream(path, true).getChannel();
+        try {
+          fileChannel.truncate(newSize);
+        } finally {
+          if (fileChannel != null) {
+            fileChannel.close();
+          }
+        }
+        return;
+    }
+  }
+
+  /**
    * for creating a new Lock file and if it is successfully created
    * then in case of abrupt shutdown then the stream to that file will be closed.
    *

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/core/src/main/java/org/apache/carbondata/core/locks/LockUsage.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/locks/LockUsage.java b/core/src/main/java/org/apache/carbondata/core/locks/LockUsage.java
index 1738c64..434129c 100644
--- a/core/src/main/java/org/apache/carbondata/core/locks/LockUsage.java
+++ b/core/src/main/java/org/apache/carbondata/core/locks/LockUsage.java
@@ -32,5 +32,5 @@ public class LockUsage {
   public static final String DELETE_SEGMENT_LOCK = "delete_segment.lock";
   public static final String CLEAN_FILES_LOCK = "clean_files.lock";
   public static final String DROP_TABLE_LOCK = "droptable.lock";
-
+  public static final String STREAMING_LOCK = "streaming.lock";
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/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 43dad72..8686583 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
@@ -48,25 +48,24 @@ public class DataTypes {
   public static final DataType STRUCT = StructType.STRUCT;
   public static final DataType MAP = MapType.MAP;
 
-  // 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;
+  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;
 
   /**
    * create a DataType instance from uniqueId of the DataType

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/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 e1a7143..d4aaa29 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
@@ -124,6 +124,8 @@ public class CarbonTable implements Serializable {
    */
   private int numberOfNoDictSortColumns;
 
+  private int dimensionOrdinalMax;
+
   private CarbonTable() {
     this.tableDimensionsMap = new HashMap<String, List<CarbonDimension>>();
     this.tableImplicitDimensionsMap = new HashMap<String, List<CarbonDimension>>();
@@ -259,6 +261,8 @@ public class CarbonTable implements Serializable {
     fillVisibleDimensions(tableSchema.getTableName());
     fillVisibleMeasures(tableSchema.getTableName());
     addImplicitDimension(dimensionOrdinal, implicitDimensions);
+
+    dimensionOrdinalMax = dimensionOrdinal;
   }
 
   /**
@@ -431,6 +435,30 @@ public class CarbonTable implements Serializable {
   }
 
   /**
+   * This method will give storage order column list
+   */
+  public List<CarbonColumn> getStreamStorageOrderColumn(String tableName) {
+    List<CarbonDimension> dimensions = tableDimensionsMap.get(tableName);
+    List<CarbonMeasure> measures = tableMeasuresMap.get(tableName);
+    List<CarbonColumn> columnList = new ArrayList<>(dimensions.size() + measures.size());
+    List<CarbonColumn> complexdimensionList = new ArrayList<>(dimensions.size());
+    for (CarbonColumn column : dimensions) {
+      if (column.isComplex()) {
+        complexdimensionList.add(column);
+      } else {
+        columnList.add(column);
+      }
+    }
+    columnList.addAll(complexdimensionList);
+    for (CarbonColumn column : measures) {
+      if (!(column.getColName().equals("default_dummy_measure"))) {
+        columnList.add(column);
+      }
+    }
+    return columnList;
+  }
+
+  /**
    * to get particular measure from a table
    *
    * @param tableName
@@ -665,4 +693,13 @@ public class CarbonTable implements Serializable {
     String streaming = getTableInfo().getFactTable().getTableProperties().get("streaming");
     return streaming != null && streaming.equalsIgnoreCase("true");
   }
+
+  public int getDimensionOrdinalMax() {
+    return dimensionOrdinalMax;
+  }
+
+  public void setDimensionOrdinalMax(int dimensionOrdinalMax) {
+    this.dimensionOrdinalMax = dimensionOrdinalMax;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterUtil.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterUtil.java
index 6943b8b..56b27aa 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterUtil.java
@@ -57,6 +57,7 @@ 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.CarbonColumn;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
 import org.apache.carbondata.core.scan.expression.ColumnExpression;
@@ -1639,4 +1640,25 @@ public final class FilterUtil {
       }
     }
   }
+
+  public static void updateIndexOfColumnExpression(Expression exp, int dimOridnalMax) {
+    if (exp.getChildren() == null || exp.getChildren().size() == 0) {
+      if (exp instanceof ColumnExpression) {
+        ColumnExpression ce = (ColumnExpression) exp;
+        CarbonColumn column = ce.getCarbonColumn();
+        if (column.isDimension()) {
+          ce.setColIndex(column.getOrdinal());
+        } else {
+          ce.setColIndex(dimOridnalMax + column.getOrdinal());
+        }
+      }
+    } else {
+      if (exp.getChildren().size() > 0) {
+        List<Expression> children = exp.getChildren();
+        for (int i = 0; i < children.size(); i++) {
+          updateIndexOfColumnExpression(children.get(i), dimOridnalMax);
+        }
+      }
+    }
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/AndFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/AndFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/AndFilterExecuterImpl.java
index f79e788..6b256f1 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/AndFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/AndFilterExecuterImpl.java
@@ -20,6 +20,7 @@ import java.io.IOException;
 import java.util.BitSet;
 
 import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
+import org.apache.carbondata.core.scan.filter.intf.RowIntf;
 import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
 import org.apache.carbondata.core.util.BitSetGroup;
 
@@ -49,6 +50,12 @@ public class AndFilterExecuterImpl implements FilterExecuter {
     return leftFilters;
   }
 
+  @Override public boolean applyFilter(RowIntf value, int dimOrdinalMax)
+      throws FilterUnsupportedException, IOException {
+    return leftExecuter.applyFilter(value, dimOrdinalMax) &&
+        rightExecuter.applyFilter(value, dimOrdinalMax);
+  }
+
   @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
     BitSet leftFilters = leftExecuter.isScanRequired(blockMaxValue, blockMinValue);
     if (leftFilters.isEmpty()) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImpl.java
index 87e7dea..df94d46 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImpl.java
@@ -27,10 +27,12 @@ import org.apache.carbondata.core.datastore.page.ColumnPage;
 import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.scan.filter.FilterUtil;
+import org.apache.carbondata.core.scan.filter.intf.RowIntf;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
 import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
 import org.apache.carbondata.core.util.BitSetGroup;
+import org.apache.carbondata.core.util.ByteUtil;
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.core.util.DataTypeUtil;
 import org.apache.carbondata.core.util.comparator.Comparator;
@@ -45,6 +47,7 @@ public class ExcludeFilterExecuterImpl implements FilterExecuter {
   protected SegmentProperties segmentProperties;
   protected boolean isDimensionPresentInCurrentBlock = false;
   protected boolean isMeasurePresentInCurrentBlock = false;
+  private SerializableComparator comparator;
   /**
    * is dimension column data is natural sorted
    */
@@ -71,6 +74,9 @@ public class ExcludeFilterExecuterImpl implements FilterExecuter {
           .prepareKeysFromSurrogates(msrColumnEvaluatorInfo.getFilterValues(), segmentProperties,
               null, null, msrColumnEvaluatorInfo.getMeasure(), msrColumnExecutorInfo);
       isMeasurePresentInCurrentBlock = true;
+
+      DataType msrType = getMeasureDataType(msrColumnEvaluatorInfo);
+      comparator = Comparator.getComparatorByDataTypeForMeasure(msrType);
     }
 
   }
@@ -127,6 +133,34 @@ public class ExcludeFilterExecuterImpl implements FilterExecuter {
     return null;
   }
 
+  @Override public boolean applyFilter(RowIntf value, int dimOrdinalMax) {
+    if (isDimensionPresentInCurrentBlock) {
+      byte[][] filterValues = dimColumnExecuterInfo.getFilterKeys();
+      byte[] col = (byte[])value.getVal(dimColEvaluatorInfo.getDimension().getOrdinal());
+      for (int i = 0; i < filterValues.length; i++) {
+        if (0 == ByteUtil.UnsafeComparer.INSTANCE.compareTo(col, 0, col.length,
+            filterValues[i], 0, filterValues[i].length)) {
+          return false;
+        }
+      }
+    } else if (isMeasurePresentInCurrentBlock) {
+      Object[] filterValues = msrColumnExecutorInfo.getFilterKeys();
+      Object col = value.getVal(msrColumnEvaluatorInfo.getMeasure().getOrdinal() + dimOrdinalMax);
+      for (int i = 0; i < filterValues.length; i++) {
+        if (filterValues[i] == null) {
+          if (null == col) {
+            return false;
+          }
+          continue;
+        }
+        if (comparator.compare(col, filterValues[i]) == 0) {
+          return false;
+        }
+      }
+    }
+    return true;
+  }
+
   private DataType getMeasureDataType(MeasureColumnResolvedFilterInfo msrColumnEvaluatorInfo) {
     if (msrColumnEvaluatorInfo.getType() == DataTypes.BOOLEAN) {
       return DataTypes.BOOLEAN;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/FilterExecuter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/FilterExecuter.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/FilterExecuter.java
index 93640fa..85891dc 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/FilterExecuter.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/FilterExecuter.java
@@ -20,6 +20,7 @@ import java.io.IOException;
 import java.util.BitSet;
 
 import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
+import org.apache.carbondata.core.scan.filter.intf.RowIntf;
 import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
 import org.apache.carbondata.core.util.BitSetGroup;
 
@@ -34,6 +35,9 @@ public interface FilterExecuter {
   BitSetGroup applyFilter(BlocksChunkHolder blocksChunkHolder, boolean useBitsetPipeLine)
       throws FilterUnsupportedException, IOException;
 
+  boolean applyFilter(RowIntf value, int dimOrdinalMax)
+      throws FilterUnsupportedException, IOException;
+
   /**
    * API will verify whether the block can be shortlisted based on block
    * max and min key.

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImpl.java
index 0022a72..fe1421c 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImpl.java
@@ -27,6 +27,7 @@ import org.apache.carbondata.core.datastore.page.ColumnPage;
 import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.scan.filter.FilterUtil;
+import org.apache.carbondata.core.scan.filter.intf.RowIntf;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
 import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
@@ -78,6 +79,8 @@ public class IncludeFilterExecuterImpl implements FilterExecuter {
               null, null, msrColumnEvaluatorInfo.getMeasure(), msrColumnExecutorInfo);
       isMeasurePresentInCurrentBlock = true;
 
+      DataType msrType = getMeasureDataType(msrColumnEvaluatorInfo);
+      comparator = Comparator.getComparatorByDataTypeForMeasure(msrType);
     }
 
   }
@@ -147,6 +150,34 @@ public class IncludeFilterExecuterImpl implements FilterExecuter {
     return null;
   }
 
+  @Override public boolean applyFilter(RowIntf value, int dimOrdinalMax) {
+    if (isDimensionPresentInCurrentBlock) {
+      byte[][] filterValues = dimColumnExecuterInfo.getFilterKeys();
+      byte[] col = (byte[])value.getVal(dimColumnEvaluatorInfo.getDimension().getOrdinal());
+      for (int i = 0; i < filterValues.length; i++) {
+        if (0 == ByteUtil.UnsafeComparer.INSTANCE.compareTo(col, 0, col.length,
+            filterValues[i], 0, filterValues[i].length)) {
+          return true;
+        }
+      }
+    } else if (isMeasurePresentInCurrentBlock) {
+      Object[] filterValues = msrColumnExecutorInfo.getFilterKeys();
+      Object col = value.getVal(msrColumnEvaluatorInfo.getMeasure().getOrdinal() + dimOrdinalMax);
+      for (int i = 0; i < filterValues.length; i++) {
+        if (filterValues[i] == null) {
+          if (null == col) {
+            return true;
+          }
+          continue;
+        }
+        if (comparator.compare(col, filterValues[i]) == 0) {
+          return true;
+        }
+      }
+    }
+    return false;
+  }
+
   private DataType getMeasureDataType(MeasureColumnResolvedFilterInfo msrColumnEvaluatorInfo) {
     if (msrColumnEvaluatorInfo.getType() == DataTypes.BOOLEAN) {
       return DataTypes.BOOLEAN;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/OrFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/OrFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/OrFilterExecuterImpl.java
index 7eed8ee..87273bb 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/OrFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/OrFilterExecuterImpl.java
@@ -20,6 +20,7 @@ import java.io.IOException;
 import java.util.BitSet;
 
 import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
+import org.apache.carbondata.core.scan.filter.intf.RowIntf;
 import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
 import org.apache.carbondata.core.util.BitSetGroup;
 
@@ -43,6 +44,12 @@ public class OrFilterExecuterImpl implements FilterExecuter {
     return leftFilters;
   }
 
+  @Override public boolean applyFilter(RowIntf value, int dimOrdinalMax)
+      throws FilterUnsupportedException, IOException {
+    return leftExecuter.applyFilter(value, dimOrdinalMax) ||
+        rightExecuter.applyFilter(value, dimOrdinalMax);
+  }
+
   @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
     BitSet leftFilters = leftExecuter.isScanRequired(blockMaxValue, blockMinValue);
     BitSet rightFilters = rightExecuter.isScanRequired(blockMaxValue, blockMinValue);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RestructureExcludeFilterExecutorImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RestructureExcludeFilterExecutorImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RestructureExcludeFilterExecutorImpl.java
index 2c0d39f..5707eb4 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RestructureExcludeFilterExecutorImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RestructureExcludeFilterExecutorImpl.java
@@ -19,7 +19,9 @@ package org.apache.carbondata.core.scan.filter.executer;
 import java.io.IOException;
 import java.util.BitSet;
 
+import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
 import org.apache.carbondata.core.scan.filter.FilterUtil;
+import org.apache.carbondata.core.scan.filter.intf.RowIntf;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
 import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
@@ -54,6 +56,11 @@ public class RestructureExcludeFilterExecutorImpl extends RestructureEvaluatorIm
             numberOfRows, !isDefaultValuePresentInFilterValues);
   }
 
+  @Override public boolean applyFilter(RowIntf value, int dimOrdinalMax)
+      throws FilterUnsupportedException {
+    throw new FilterUnsupportedException("Unsupported RestructureExcludeFilterExecutorImpl on row");
+  }
+
   @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
     BitSet bitSet = new BitSet(1);
     bitSet.flip(0, 1);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RestructureIncludeFilterExecutorImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RestructureIncludeFilterExecutorImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RestructureIncludeFilterExecutorImpl.java
index 5ec6971..8bcc53f 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RestructureIncludeFilterExecutorImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RestructureIncludeFilterExecutorImpl.java
@@ -19,7 +19,9 @@ package org.apache.carbondata.core.scan.filter.executer;
 import java.io.IOException;
 import java.util.BitSet;
 
+import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
 import org.apache.carbondata.core.scan.filter.FilterUtil;
+import org.apache.carbondata.core.scan.filter.intf.RowIntf;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
 import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
@@ -53,6 +55,11 @@ public class RestructureIncludeFilterExecutorImpl extends RestructureEvaluatorIm
             numberOfRows, isDefaultValuePresentInFilterValues);
   }
 
+  @Override public boolean applyFilter(RowIntf value, int dimOrdinalMax)
+      throws FilterUnsupportedException {
+    throw new FilterUnsupportedException("Unsupported RestructureIncludeFilterExecutorImpl on row");
+  }
+
   public BitSet isScanRequired(byte[][] blkMaxVal, byte[][] blkMinVal) {
     BitSet bitSet = new BitSet(1);
     bitSet.set(0, isDefaultValuePresentInFilterValues);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/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 10664c8..777f564 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
@@ -266,6 +266,15 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
     return bitSetGroup;
   }
 
+  @Override public boolean applyFilter(RowIntf value, int dimOrdinalMax)
+      throws FilterUnsupportedException, IOException {
+    try {
+      return exp.evaluate(value).getBoolean();
+    } catch (FilterIllegalMemberException e) {
+      throw new FilterUnsupportedException(e);
+    }
+  }
+
   /**
    * Method will read the members of particular dimension block and create
    * a row instance for further processing of the filters

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/TrueFilterExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/TrueFilterExecutor.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/TrueFilterExecutor.java
index 91cebc5..92396ae 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/TrueFilterExecutor.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/TrueFilterExecutor.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.util.BitSet;
 
 import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
+import org.apache.carbondata.core.scan.filter.intf.RowIntf;
 import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
 import org.apache.carbondata.core.util.BitSetGroup;
 
@@ -44,6 +45,10 @@ public class TrueFilterExecutor implements FilterExecuter {
     return group;
   }
 
+  @Override public boolean applyFilter(RowIntf value, int dimOrdinalMax) {
+    return true;
+  }
+
   /**
    * API will verify whether the block can be shortlisted based on block
    * max and min key.

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ValueBasedFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ValueBasedFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ValueBasedFilterExecuterImpl.java
index 6dc1375..516447f 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ValueBasedFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ValueBasedFilterExecuterImpl.java
@@ -20,6 +20,7 @@ import java.io.IOException;
 import java.util.BitSet;
 
 import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
+import org.apache.carbondata.core.scan.filter.intf.RowIntf;
 import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
 import org.apache.carbondata.core.util.BitSetGroup;
 
@@ -32,6 +33,11 @@ public class ValueBasedFilterExecuterImpl implements FilterExecuter {
     return new BitSetGroup(0);
   }
 
+  @Override public boolean applyFilter(RowIntf value, int dimOrdinalMax)
+      throws FilterUnsupportedException, IOException {
+    throw new FilterUnsupportedException("Unsupported ValueBasedFilterExecuterImpl on row");
+  }
+
   @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
     return new BitSet(1);
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/core/src/main/java/org/apache/carbondata/core/scan/model/QueryModel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/model/QueryModel.java b/core/src/main/java/org/apache/carbondata/core/scan/model/QueryModel.java
index 20be2fd..66dfa61 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/model/QueryModel.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/model/QueryModel.java
@@ -109,6 +109,9 @@ public class QueryModel implements Serializable {
   private Map<String, UpdateVO> invalidSegmentBlockIdMap =
       new HashMap<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
 
+  private boolean[] isFilterDimensions;
+  private boolean[] isFilterMeasures;
+
   public QueryModel() {
     tableBlockInfos = new ArrayList<TableBlockInfo>();
     queryDimension = new ArrayList<QueryDimension>();
@@ -136,9 +139,14 @@ public class QueryModel implements Serializable {
     queryModel.setQueryDimension(queryPlan.getDimensions());
     queryModel.setQueryMeasures(queryPlan.getMeasures());
     if (null != queryPlan.getFilterExpression()) {
+      boolean[] isFilterDimensions = new boolean[carbonTable.getDimensionOrdinalMax()];
+      boolean[] isFilterMeasures =
+          new boolean[carbonTable.getNumberOfMeasures(carbonTable.getFactTableName())];
       processFilterExpression(queryPlan.getFilterExpression(),
           carbonTable.getDimensionByTableName(factTableName),
-          carbonTable.getMeasureByTableName(factTableName));
+          carbonTable.getMeasureByTableName(factTableName), isFilterDimensions, isFilterMeasures);
+      queryModel.setIsFilterDimensions(isFilterDimensions);
+      queryModel.setIsFilterMeasures(isFilterMeasures);
     }
     //TODO need to remove this code, and executor will load the table
     // from file metadata
@@ -146,28 +154,32 @@ public class QueryModel implements Serializable {
   }
 
   public static void processFilterExpression(Expression filterExpression,
-      List<CarbonDimension> dimensions, List<CarbonMeasure> measures) {
+      List<CarbonDimension> dimensions, List<CarbonMeasure> measures,
+      final boolean[] isFilterDimensions, final boolean[] isFilterMeasures) {
     if (null != filterExpression) {
       if (null != filterExpression.getChildren() && filterExpression.getChildren().size() == 0) {
         if (filterExpression instanceof ConditionalExpression) {
           List<ColumnExpression> listOfCol =
               ((ConditionalExpression) filterExpression).getColumnList();
           for (ColumnExpression expression : listOfCol) {
-            setDimAndMsrColumnNode(dimensions, measures, expression);
+            setDimAndMsrColumnNode(dimensions, measures, expression, isFilterDimensions,
+                isFilterMeasures);
           }
         }
       }
       for (Expression expression : filterExpression.getChildren()) {
         if (expression instanceof ColumnExpression) {
-          setDimAndMsrColumnNode(dimensions, measures, (ColumnExpression) expression);
+          setDimAndMsrColumnNode(dimensions, measures, (ColumnExpression) expression,
+              isFilterDimensions, isFilterMeasures);
         } else if (expression instanceof UnknownExpression) {
           UnknownExpression exp = ((UnknownExpression) expression);
           List<ColumnExpression> listOfColExpression = exp.getAllColumnList();
           for (ColumnExpression col : listOfColExpression) {
-            setDimAndMsrColumnNode(dimensions, measures, col);
+            setDimAndMsrColumnNode(dimensions, measures, col, isFilterDimensions, isFilterMeasures);
           }
         } else {
-          processFilterExpression(expression, dimensions, measures);
+          processFilterExpression(expression, dimensions, measures, isFilterDimensions,
+              isFilterMeasures);
         }
       }
     }
@@ -184,7 +196,8 @@ public class QueryModel implements Serializable {
   }
 
   private static void setDimAndMsrColumnNode(List<CarbonDimension> dimensions,
-      List<CarbonMeasure> measures, ColumnExpression col) {
+      List<CarbonMeasure> measures, ColumnExpression col, boolean[] isFilterDimensions,
+      boolean[] isFilterMeasures) {
     CarbonDimension dim;
     CarbonMeasure msr;
     String columnName;
@@ -199,10 +212,16 @@ public class QueryModel implements Serializable {
       col.setCarbonColumn(dim);
       col.setDimension(dim);
       col.setDimension(true);
+      if (null != isFilterDimensions) {
+        isFilterDimensions[dim.getOrdinal()] = true;
+      }
     } else {
       col.setCarbonColumn(msr);
       col.setMeasure(msr);
       col.setMeasure(true);
+      if (null != isFilterMeasures) {
+        isFilterMeasures[msr.getOrdinal()] = true;
+      }
     }
   }
 
@@ -378,4 +397,20 @@ public class QueryModel implements Serializable {
   public void setConverter(DataTypeConverter converter) {
     this.converter = converter;
   }
+
+  public boolean[] getIsFilterDimensions() {
+    return isFilterDimensions;
+  }
+
+  public void setIsFilterDimensions(boolean[] isFilterDimensions) {
+    this.isFilterDimensions = isFilterDimensions;
+  }
+
+  public boolean[] getIsFilterMeasures() {
+    return isFilterMeasures;
+  }
+
+  public void setIsFilterMeasures(boolean[] isFilterMeasures) {
+    this.isFilterMeasures = isFilterMeasures;
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/core/src/main/java/org/apache/carbondata/core/statusmanager/FileFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/statusmanager/FileFormat.java b/core/src/main/java/org/apache/carbondata/core/statusmanager/FileFormat.java
new file mode 100644
index 0000000..83a4813
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/statusmanager/FileFormat.java
@@ -0,0 +1,40 @@
+/*
+ * 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.statusmanager;
+
+/**
+ * the data file format which was supported
+ */
+public enum FileFormat {
+  carbondata, rowformat;
+
+  public static FileFormat getByOrdinal(int ordinal) {
+    if (ordinal < 0 || ordinal >= FileFormat.values().length) {
+      return carbondata;
+    }
+
+    switch (ordinal) {
+      case 0:
+        return carbondata;
+      case 1:
+        return rowformat;
+    }
+
+    return carbondata;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/core/src/main/java/org/apache/carbondata/core/statusmanager/LoadMetadataDetails.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/statusmanager/LoadMetadataDetails.java b/core/src/main/java/org/apache/carbondata/core/statusmanager/LoadMetadataDetails.java
index 3f83c72..7748d17 100644
--- a/core/src/main/java/org/apache/carbondata/core/statusmanager/LoadMetadataDetails.java
+++ b/core/src/main/java/org/apache/carbondata/core/statusmanager/LoadMetadataDetails.java
@@ -70,6 +70,11 @@ public class LoadMetadataDetails implements Serializable {
    */
   private String majorCompacted;
 
+  /**
+   * the file format of this segment
+   */
+  private FileFormat fileFormat = FileFormat.carbondata;
+
   public String getPartitionCount() {
     return partitionCount;
   }
@@ -339,4 +344,12 @@ public class LoadMetadataDetails implements Serializable {
   public void setUpdateStatusFileName(String updateStatusFileName) {
     this.updateStatusFileName = updateStatusFileName;
   }
+
+  public FileFormat getFileFormat() {
+    return fileFormat;
+  }
+
+  public void setFileFormat(FileFormat fileFormat) {
+    this.fileFormat = fileFormat;
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
index 28d3f18..30304d9 100644
--- a/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
+++ b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
@@ -100,9 +100,10 @@ public class SegmentStatusManager {
   public ValidAndInvalidSegmentsInfo getValidAndInvalidSegments() throws IOException {
 
     // @TODO: move reading LoadStatus file to separate class
-    List<String> listOfValidSegments = new ArrayList<String>(10);
-    List<String> listOfValidUpdatedSegments = new ArrayList<String>(10);
-    List<String> listOfInvalidSegments = new ArrayList<String>(10);
+    List<String> listOfValidSegments = new ArrayList<>(10);
+    List<String> listOfValidUpdatedSegments = new ArrayList<>(10);
+    List<String> listOfInvalidSegments = new ArrayList<>(10);
+    List<String> listOfStreamSegments = new ArrayList<>(10);
     CarbonTablePath carbonTablePath = CarbonStorePath
             .getCarbonTablePath(absoluteTableIdentifier.getStorePath(),
                     absoluteTableIdentifier.getCarbonTableIdentifier());
@@ -125,6 +126,10 @@ public class SegmentStatusManager {
                   || CarbonCommonConstants.MARKED_FOR_UPDATE
                   .equalsIgnoreCase(loadMetadataDetails.getLoadStatus())
                   || CarbonCommonConstants.STORE_LOADSTATUS_PARTIAL_SUCCESS
+                  .equalsIgnoreCase(loadMetadataDetails.getLoadStatus())
+                  || CarbonCommonConstants.STORE_LOADSTATUS_STREAMING
+                  .equalsIgnoreCase(loadMetadataDetails.getLoadStatus())
+                  || CarbonCommonConstants.STORE_LOADSTATUS_STREAMING_FINISH
                   .equalsIgnoreCase(loadMetadataDetails.getLoadStatus())) {
             // check for merged loads.
             if (null != loadMetadataDetails.getMergedLoadName()) {
@@ -144,6 +149,13 @@ public class SegmentStatusManager {
 
               listOfValidUpdatedSegments.add(loadMetadataDetails.getLoadName());
             }
+            if (CarbonCommonConstants.STORE_LOADSTATUS_STREAMING
+                .equalsIgnoreCase(loadMetadataDetails.getLoadStatus())
+                || CarbonCommonConstants.STORE_LOADSTATUS_STREAMING_FINISH
+                .equalsIgnoreCase(loadMetadataDetails.getLoadStatus())) {
+              listOfStreamSegments.add(loadMetadataDetails.getLoadName());
+              continue;
+            }
             listOfValidSegments.add(loadMetadataDetails.getLoadName());
           } else if ((CarbonCommonConstants.STORE_LOADSTATUS_FAILURE
                   .equalsIgnoreCase(loadMetadataDetails.getLoadStatus())
@@ -169,7 +181,7 @@ public class SegmentStatusManager {
       }
     }
     return new ValidAndInvalidSegmentsInfo(listOfValidSegments, listOfValidUpdatedSegments,
-            listOfInvalidSegments);
+            listOfInvalidSegments, listOfStreamSegments);
   }
 
   /**
@@ -642,12 +654,15 @@ public class SegmentStatusManager {
     private final List<String> listOfValidSegments;
     private final List<String> listOfValidUpdatedSegments;
     private final List<String> listOfInvalidSegments;
+    private final List<String> listOfStreamSegments;
 
     private ValidAndInvalidSegmentsInfo(List<String> listOfValidSegments,
-        List<String> listOfValidUpdatedSegments, List<String> listOfInvalidUpdatedSegments) {
+        List<String> listOfValidUpdatedSegments, List<String> listOfInvalidUpdatedSegments,
+        List<String> listOfStreamSegments) {
       this.listOfValidSegments = listOfValidSegments;
       this.listOfValidUpdatedSegments = listOfValidUpdatedSegments;
       this.listOfInvalidSegments = listOfInvalidUpdatedSegments;
+      this.listOfStreamSegments = listOfStreamSegments;
     }
     public List<String> getInvalidSegments() {
       return listOfInvalidSegments;
@@ -655,5 +670,9 @@ public class SegmentStatusManager {
     public List<String> getValidSegments() {
       return listOfValidSegments;
     }
+
+    public List<String> getStreamSegments() {
+      return listOfStreamSegments;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
index 17a4b5f..016b8b3 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
@@ -78,6 +78,7 @@ import org.apache.carbondata.core.statusmanager.SegmentUpdateStatusManager;
 import org.apache.carbondata.core.util.path.CarbonStorePath;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.core.writer.ThriftWriter;
+import org.apache.carbondata.format.BlockletHeader;
 import org.apache.carbondata.format.DataChunk2;
 import org.apache.carbondata.format.DataChunk3;
 
@@ -1358,6 +1359,13 @@ public final class CarbonUtil {
     return thriftByteArray;
   }
 
+  public static BlockletHeader readBlockletHeader(byte[] data) throws IOException {
+    return (BlockletHeader) read(data, new ThriftReader.TBaseCreator() {
+      @Override public TBase create() {
+        return new BlockletHeader();
+      }
+    }, 0, data.length);
+  }
 
   public static DataChunk3 readDataChunk3(ByteBuffer dataChunkBuffer, int offset, int length)
       throws IOException {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java b/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java
index 02a000a..caa046f 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java
@@ -53,6 +53,10 @@ public class CarbonTablePath extends Path {
   public static final String INDEX_FILE_EXT = ".carbonindex";
   public static final String MERGE_INDEX_FILE_EXT = ".carbonindexmerge";
 
+  private static final String STREAMING_DIR = ".streaming";
+  private static final String STREAMING_LOG_DIR = "log";
+  private static final String STREAMING_CHECKPOINT_DIR = "checkpoint";
+
   private String tablePath;
   private CarbonTableIdentifier carbonTableIdentifier;
 
@@ -428,6 +432,14 @@ public class CarbonTablePath extends Path {
         + INDEX_FILE_EXT;
   }
 
+  public static String getCarbonStreamIndexFileName() {
+    return getCarbonIndexFileName(0, 0, 0, "0");
+  }
+
+  public static String getCarbonStreamIndexFilePath(String segmentDir) {
+    return segmentDir + File.separator + getCarbonStreamIndexFileName();
+  }
+
   /**
    * Below method will be used to get the carbon index filename
    *
@@ -440,7 +452,7 @@ public class CarbonTablePath extends Path {
     return taskNo + "-" + factUpdatedTimeStamp + indexFileExtension;
   }
 
-  private String getSegmentDir(String partitionId, String segmentId) {
+  public String getSegmentDir(String partitionId, String segmentId) {
     return getPartitionDir(partitionId) + File.separator + SEGMENT_PREFIX + segmentId;
   }
 
@@ -456,6 +468,14 @@ public class CarbonTablePath extends Path {
     return tablePath + File.separator + FACT_DIR;
   }
 
+  public String getStreamingLogDir() {
+    return tablePath + File.separator + STREAMING_DIR + File.separator + STREAMING_LOG_DIR;
+  }
+
+  public String getStreamingCheckpointDir() {
+    return tablePath + File.separator + STREAMING_DIR + File.separator + STREAMING_CHECKPOINT_DIR;
+  }
+
   public CarbonTableIdentifier getCarbonTableIdentifier() {
     return carbonTableIdentifier;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/examples/spark2/pom.xml
----------------------------------------------------------------------
diff --git a/examples/spark2/pom.xml b/examples/spark2/pom.xml
index af25771..227da7d 100644
--- a/examples/spark2/pom.xml
+++ b/examples/spark2/pom.xml
@@ -40,6 +40,11 @@
       <version>${project.version}</version>
     </dependency>
     <dependency>
+      <groupId>org.apache.carbondata</groupId>
+      <artifactId>carbondata-streaming</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
       <groupId>org.apache.spark</groupId>
       <artifactId>spark-sql_${scala.binary.version}</artifactId>
     </dependency>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/examples/spark2/src/main/resources/streamSample.csv
----------------------------------------------------------------------
diff --git a/examples/spark2/src/main/resources/streamSample.csv b/examples/spark2/src/main/resources/streamSample.csv
new file mode 100644
index 0000000..590ea90
--- /dev/null
+++ b/examples/spark2/src/main/resources/streamSample.csv
@@ -0,0 +1,6 @@
+id,name,city,salary,file
+100000001,batch_1,city_1,0.1,school_1:school_11$20
+100000002,batch_2,city_2,0.2,school_2:school_22$30
+100000003,batch_3,city_3,0.3,school_3:school_33$40
+100000004,batch_4,city_4,0.4,school_4:school_44$50
+100000005,batch_5,city_5,0.5,school_5:school_55$60

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/examples/spark2/src/main/scala/org/apache/carbondata/examples/StreamExample.scala
----------------------------------------------------------------------
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/StreamExample.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/StreamExample.scala
new file mode 100644
index 0000000..c31a0aa
--- /dev/null
+++ b/examples/spark2/src/main/scala/org/apache/carbondata/examples/StreamExample.scala
@@ -0,0 +1,213 @@
+/*
+ * 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.examples
+
+import java.io.{File, PrintWriter}
+import java.net.ServerSocket
+
+import org.apache.spark.sql.{CarbonEnv, SparkSession}
+import org.apache.spark.sql.hive.CarbonRelation
+import org.apache.spark.sql.streaming.{ProcessingTime, StreamingQuery}
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
+
+// scalastyle:off println
+object StreamExample {
+  def main(args: Array[String]) {
+
+    // setup paths
+    val rootPath = new File(this.getClass.getResource("/").getPath
+                            + "../../../..").getCanonicalPath
+    val storeLocation = s"$rootPath/examples/spark2/target/store"
+    val warehouse = s"$rootPath/examples/spark2/target/warehouse"
+    val metastoredb = s"$rootPath/examples/spark2/target"
+    val streamTableName = s"stream_table"
+
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/MM/dd")
+
+    import org.apache.spark.sql.CarbonSession._
+    val spark = SparkSession
+      .builder()
+      .master("local")
+      .appName("StreamExample")
+      .config("spark.sql.warehouse.dir", warehouse)
+      .getOrCreateCarbonSession(storeLocation, metastoredb)
+
+    spark.sparkContext.setLogLevel("ERROR")
+
+    val requireCreateTable = true
+    val useComplexDataType = false
+
+    if (requireCreateTable) {
+      // drop table if exists previously
+      spark.sql(s"DROP TABLE IF EXISTS ${ streamTableName }")
+      // Create target carbon table and populate with initial data
+      if (useComplexDataType) {
+        spark.sql(
+          s"""
+             | CREATE TABLE ${ streamTableName }(
+             | id INT,
+             | name STRING,
+             | city STRING,
+             | salary FLOAT,
+             | file struct<school:array<string>, age:int>
+             | )
+             | STORED BY 'carbondata'
+             | TBLPROPERTIES('sort_columns'='name', 'dictionary_include'='city')
+             | """.stripMargin)
+      } else {
+        spark.sql(
+          s"""
+             | CREATE TABLE ${ streamTableName }(
+             | id INT,
+             | name STRING,
+             | city STRING,
+             | salary FLOAT
+             | )
+             | STORED BY 'carbondata'
+             | """.stripMargin)
+      }
+
+      val carbonTable = CarbonEnv.getInstance(spark).carbonMetastore.
+        lookupRelation(Some("default"), streamTableName)(spark).asInstanceOf[CarbonRelation].
+        tableMeta.carbonTable
+      val tablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
+      // batch load
+      val path = s"$rootPath/examples/spark2/src/main/resources/streamSample.csv"
+      spark.sql(
+        s"""
+           | LOAD DATA LOCAL INPATH '$path'
+           | INTO TABLE $streamTableName
+           | OPTIONS('HEADER'='true')
+         """.stripMargin)
+
+      // streaming ingest
+      val serverSocket = new ServerSocket(7071)
+      val thread1 = startStreaming(spark, tablePath)
+      val thread2 = writeSocket(serverSocket)
+      val thread3 = showTableCount(spark, streamTableName)
+
+      System.out.println("type enter to interrupt streaming")
+      System.in.read()
+      thread1.interrupt()
+      thread2.interrupt()
+      thread3.interrupt()
+      serverSocket.close()
+    }
+
+    spark.sql(s"select count(*) from ${ streamTableName }").show(100, truncate = false)
+
+    spark.sql(s"select * from ${ streamTableName }").show(100, truncate = false)
+
+    // record(id = 100000001) comes from batch segment_0
+    // record(id = 1) comes from stream segment_1
+    spark.sql(s"select * " +
+              s"from ${ streamTableName } " +
+              s"where id = 100000001 or id = 1 limit 100").show(100, truncate = false)
+
+    // not filter
+    spark.sql(s"select * " +
+              s"from ${ streamTableName } " +
+              s"where id < 10 limit 100").show(100, truncate = false)
+
+    if (useComplexDataType) {
+      // complex
+      spark.sql(s"select file.age, file.school " +
+                s"from ${ streamTableName } " +
+                s"where where file.age = 30 ").show(100, truncate = false)
+    }
+
+    spark.stop()
+    System.out.println("streaming finished")
+  }
+
+  def showTableCount(spark: SparkSession, tableName: String): Thread = {
+    val thread = new Thread() {
+      override def run(): Unit = {
+        for (_ <- 0 to 1000) {
+          spark.sql(s"select count(*) from $tableName").show(truncate = false)
+          Thread.sleep(1000 * 3)
+        }
+      }
+    }
+    thread.start()
+    thread
+  }
+
+  def startStreaming(spark: SparkSession, tablePath: CarbonTablePath): Thread = {
+    val thread = new Thread() {
+      override def run(): Unit = {
+        var qry: StreamingQuery = null
+        try {
+          val readSocketDF = spark.readStream
+            .format("socket")
+            .option("host", "localhost")
+            .option("port", 7071)
+            .load()
+
+          // Write data from socket stream to carbondata file
+          qry = readSocketDF.writeStream
+            .format("carbondata")
+            .trigger(ProcessingTime("5 seconds"))
+            .option("checkpointLocation", tablePath.getStreamingCheckpointDir)
+            .option("tablePath", tablePath.getPath)
+            .start()
+
+          qry.awaitTermination()
+        } catch {
+          case _: InterruptedException =>
+            println("Done reading and writing streaming data")
+        } finally {
+          qry.stop()
+        }
+      }
+    }
+    thread.start()
+    thread
+  }
+
+  def writeSocket(serverSocket: ServerSocket): Thread = {
+    val thread = new Thread() {
+      override def run(): Unit = {
+        // wait for client to connection request and accept
+        val clientSocket = serverSocket.accept()
+        val socketWriter = new PrintWriter(clientSocket.getOutputStream())
+        var index = 0
+        for (_ <- 1 to 1000) {
+          // write 5 records per iteration
+          for (_ <- 0 to 100) {
+            index = index + 1
+            socketWriter.println(index.toString + ",name_" + index
+                                 + ",city_" + index + "," + (index * 10000.00).toString +
+                                 ",school_" + index + ":school_" + index + index + "$" + index)
+          }
+          socketWriter.flush()
+          Thread.sleep(2000)
+        }
+        socketWriter.close()
+        System.out.println("Socket closed")
+      }
+    }
+    thread.start()
+    thread
+  }
+}
+// scalastyle:on println

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/format/src/main/thrift/carbondata.thrift
----------------------------------------------------------------------
diff --git a/format/src/main/thrift/carbondata.thrift b/format/src/main/thrift/carbondata.thrift
index 8acd0b1..1c15f3d 100644
--- a/format/src/main/thrift/carbondata.thrift
+++ b/format/src/main/thrift/carbondata.thrift
@@ -152,7 +152,7 @@ struct DataChunk3{
  */
 struct BlockletInfo{
     1: required i32 num_rows;	// Number of rows in this blocklet
-    2: required list<DataChunk> column_data_chunks;	// Information about all column chunks in this blocklet
+    2: optional list<DataChunk> column_data_chunks;	// Information about all column chunks in this blocklet
 }
 
 /**
@@ -209,6 +209,8 @@ struct FileHeader{
 	2: required list<schema.ColumnSchema> column_schema;  // Description of columns in this file
 	3: optional bool is_footer_present; //  To check whether footer is present or not
 	4: optional i64 time_stamp; // Timestamp to compare column schema against master schema
+	5: optional bool is_splitable; // Whether file is splitable or not
+	6: optional binary sync_marker; // 16 bytes sync marker
 }
 
 /**
@@ -225,7 +227,7 @@ enum MutationType {
 struct BlockletHeader{
 	1: required i32 blocklet_length; // Length of blocklet data
 	2: required MutationType mutation; // Mutation type of this blocklet
-	3: required BlockletIndex blocklet_index;  // Index for the following blocklet
+	3: optional BlockletIndex blocklet_index;  // Index for the following blocklet
 	4: required BlockletInfo blocklet_info;  // Info for the following blocklet
 	5: optional dictionary.ColumnDictionaryChunk dictionary; // Blocklet local dictionary
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/format/src/main/thrift/carbondata_index.thrift
----------------------------------------------------------------------
diff --git a/format/src/main/thrift/carbondata_index.thrift b/format/src/main/thrift/carbondata_index.thrift
index 4df085a..60ec769 100644
--- a/format/src/main/thrift/carbondata_index.thrift
+++ b/format/src/main/thrift/carbondata_index.thrift
@@ -42,4 +42,5 @@ struct BlockIndex{
   3: required i64 offset; // Offset of the footer
   4: required carbondata.BlockletIndex block_index;	// Blocklet index
   5: optional carbondata.BlockletInfo3 blocklet_info;
+  6: optional i64 file_size // Record the valid size for appendable carbon file
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputFormat.java
index 4e8591e..e5aac84 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputFormat.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputFormat.java
@@ -133,7 +133,7 @@ public class CarbonInputFormat<T> extends FileInputFormat<Void, T> {
   /**
    * Get TableInfo object from `configuration`
    */
-  private TableInfo getTableInfo(Configuration configuration) throws IOException {
+  private static TableInfo getTableInfo(Configuration configuration) throws IOException {
     String tableInfoStr = configuration.get(TABLE_INFO);
     if (tableInfoStr == null) {
       return null;
@@ -192,6 +192,21 @@ public class CarbonInputFormat<T> extends FileInputFormat<Void, T> {
       return this.carbonTable;
     }
   }
+
+  public static CarbonTable createCarbonTable(Configuration configuration) throws IOException {
+    // carbon table should be created either from deserialized table info (schema saved in
+    // hive metastore) or by reading schema in HDFS (schema saved in HDFS)
+    TableInfo tableInfo = getTableInfo(configuration);
+    CarbonTable carbonTable;
+    if (tableInfo != null) {
+      carbonTable = CarbonTable.buildFromTableInfo(tableInfo);
+    } else {
+      carbonTable = SchemaReader.readCarbonTableFromStore(
+          getAbsoluteTableIdentifier(configuration));
+    }
+    return carbonTable;
+  }
+
   public static void setTablePath(Configuration configuration, String tablePath)
       throws IOException {
     configuration.set(FileInputFormat.INPUT_DIR, tablePath);
@@ -295,7 +310,7 @@ public class CarbonInputFormat<T> extends FileInputFormat<Void, T> {
     configuration.set(CarbonInputFormat.INPUT_FILES, CarbonUtil.getSegmentString(validFiles));
   }
 
-  private AbsoluteTableIdentifier getAbsoluteTableIdentifier(Configuration configuration)
+  private static AbsoluteTableIdentifier getAbsoluteTableIdentifier(Configuration configuration)
       throws IOException {
     String dirs = configuration.get(INPUT_DIR, "");
     String[] inputPaths = StringUtils.split(dirs);
@@ -351,7 +366,7 @@ public class CarbonInputFormat<T> extends FileInputFormat<Void, T> {
       Expression filter = getFilterPredicates(job.getConfiguration());
       CarbonTable carbonTable = getOrCreateCarbonTable(job.getConfiguration());
       TableProvider tableProvider = new SingleTableProvider(carbonTable);
-      CarbonInputFormatUtil.processFilterExpression(filter, carbonTable);
+      CarbonInputFormatUtil.processFilterExpression(filter, carbonTable, null, null);
       BitSet matchedPartitions = null;
       PartitionInfo partitionInfo = carbonTable.getPartitionInfo(carbonTable.getFactTableName());
       if (partitionInfo != null) {
@@ -812,7 +827,7 @@ public class CarbonInputFormat<T> extends FileInputFormat<Void, T> {
 
     // set the filter to the query model in order to filter blocklet before scan
     Expression filter = getFilterPredicates(configuration);
-    CarbonInputFormatUtil.processFilterExpression(filter, carbonTable);
+    CarbonInputFormatUtil.processFilterExpression(filter, carbonTable, null, null);
     FilterResolverIntf filterIntf = CarbonInputFormatUtil
         .resolveFilter(filter, carbonTable.getAbsoluteTableIdentifier(), tableProvider);
     queryModel.setFilterExpressionResolverTree(filterIntf);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputSplit.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputSplit.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputSplit.java
index dde4c76..f7b372f 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputSplit.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputSplit.java
@@ -33,6 +33,7 @@ import org.apache.carbondata.core.datastore.block.TableBlockInfo;
 import org.apache.carbondata.core.indexstore.BlockletDetailInfo;
 import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
 import org.apache.carbondata.core.mutate.UpdateVO;
+import org.apache.carbondata.core.statusmanager.FileFormat;
 import org.apache.carbondata.core.util.ByteUtil;
 import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
@@ -81,6 +82,8 @@ public class CarbonInputSplit extends FileSplit
 
   private BlockletDetailInfo detailInfo;
 
+  private FileFormat fileFormat = FileFormat.carbondata;
+
   public CarbonInputSplit() {
     segmentId = null;
     taskId = "0";
@@ -111,6 +114,30 @@ public class CarbonInputSplit extends FileSplit
     this.numberOfBlocklets = numberOfBlocklets;
   }
 
+  public CarbonInputSplit(String segmentId, Path path, long start, long length, String[] locations,
+      FileFormat fileFormat) {
+    super(path, start, length, locations);
+    this.segmentId = segmentId;
+    this.fileFormat = fileFormat;
+    taskId = "0";
+    bucketId = "0";
+    numberOfBlocklets = 0;
+    invalidSegments = new ArrayList<>();
+    version = CarbonProperties.getInstance().getFormatVersion();
+  }
+
+  public CarbonInputSplit(String segmentId, Path path, long start, long length, String[] locations,
+      String[] inMemoryHosts, FileFormat fileFormat) {
+    super(path, start, length, locations, inMemoryHosts);
+    this.segmentId = segmentId;
+    this.fileFormat = fileFormat;
+    taskId = "0";
+    bucketId = "0";
+    numberOfBlocklets = 0;
+    invalidSegments = new ArrayList<>();
+    version = CarbonProperties.getInstance().getFormatVersion();
+  }
+
   /**
    * Constructor to initialize the CarbonInputSplit with blockStorageIdMap
    * @param segmentId
@@ -363,4 +390,12 @@ public class CarbonInputSplit extends FileSplit
   public void setDetailInfo(BlockletDetailInfo detailInfo) {
     this.detailInfo = detailInfo;
   }
+
+  public FileFormat getFileFormat() {
+    return fileFormat;
+  }
+
+  public void setFormat(FileFormat fileFormat) {
+    this.fileFormat = fileFormat;
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonMultiBlockSplit.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonMultiBlockSplit.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonMultiBlockSplit.java
index 1f8ccfc..d3fa2c2 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonMultiBlockSplit.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonMultiBlockSplit.java
@@ -24,6 +24,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
+import org.apache.carbondata.core.statusmanager.FileFormat;
 
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.mapreduce.InputSplit;
@@ -44,6 +45,8 @@ public class CarbonMultiBlockSplit extends InputSplit implements Writable {
    */
   private String[] locations;
 
+  private FileFormat fileFormat = FileFormat.carbondata;
+
   public CarbonMultiBlockSplit() {
     splitList = null;
     locations = null;
@@ -55,6 +58,13 @@ public class CarbonMultiBlockSplit extends InputSplit implements Writable {
     this.locations = locations;
   }
 
+  public CarbonMultiBlockSplit(AbsoluteTableIdentifier identifier, List<CarbonInputSplit> splitList,
+      String[] locations, FileFormat fileFormat) throws IOException {
+    this.splitList = splitList;
+    this.locations = locations;
+    this.fileFormat = fileFormat;
+  }
+
   /**
    * Return all splits for scan
    * @return split list for scan
@@ -88,6 +98,7 @@ public class CarbonMultiBlockSplit extends InputSplit implements Writable {
     for (int i = 0; i < locations.length; i++) {
       out.writeUTF(locations[i]);
     }
+    out.writeInt(fileFormat.ordinal());
   }
 
   @Override
@@ -105,6 +116,14 @@ public class CarbonMultiBlockSplit extends InputSplit implements Writable {
     for (int i = 0; i < len; i++) {
       locations[i] = in.readUTF();
     }
+    fileFormat = FileFormat.getByOrdinal(in.readInt());
+  }
+
+  public FileFormat getFileFormat() {
+    return fileFormat;
   }
 
+  public void setFileFormat(FileFormat fileFormat) {
+    this.fileFormat = fileFormat;
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
index 9fbeb8a..e22a5c6 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
@@ -19,6 +19,7 @@ package org.apache.carbondata.hadoop.api;
 
 import java.io.ByteArrayInputStream;
 import java.io.DataInputStream;
+import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.lang.reflect.Constructor;
@@ -33,6 +34,8 @@ import java.util.Map;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datamap.DataMapStoreManager;
 import org.apache.carbondata.core.datamap.TableDataMap;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
 import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataMap;
 import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataMapFactory;
@@ -46,6 +49,7 @@ import org.apache.carbondata.core.mutate.CarbonUpdateUtil;
 import org.apache.carbondata.core.mutate.SegmentUpdateDetails;
 import org.apache.carbondata.core.mutate.UpdateVO;
 import org.apache.carbondata.core.mutate.data.BlockMappingVO;
+import org.apache.carbondata.core.reader.CarbonIndexFileReader;
 import org.apache.carbondata.core.scan.expression.Expression;
 import org.apache.carbondata.core.scan.filter.FilterExpressionProcessor;
 import org.apache.carbondata.core.scan.filter.SingleTableProvider;
@@ -56,6 +60,7 @@ import org.apache.carbondata.core.scan.model.QueryModel;
 import org.apache.carbondata.core.stats.QueryStatistic;
 import org.apache.carbondata.core.stats.QueryStatisticsConstants;
 import org.apache.carbondata.core.stats.QueryStatisticsRecorder;
+import org.apache.carbondata.core.statusmanager.FileFormat;
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
 import org.apache.carbondata.core.statusmanager.SegmentUpdateStatusManager;
 import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory;
@@ -64,6 +69,7 @@ import org.apache.carbondata.core.util.DataTypeConverter;
 import org.apache.carbondata.core.util.DataTypeConverterImpl;
 import org.apache.carbondata.core.util.path.CarbonStorePath;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
+import org.apache.carbondata.format.BlockIndex;
 import org.apache.carbondata.hadoop.CarbonInputSplit;
 import org.apache.carbondata.hadoop.CarbonMultiBlockSplit;
 import org.apache.carbondata.hadoop.CarbonProjection;
@@ -77,6 +83,8 @@ import org.apache.carbondata.hadoop.util.SchemaReader;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.InvalidPathException;
 import org.apache.hadoop.fs.LocalFileSystem;
@@ -128,7 +136,7 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
   /**
    * Get TableInfo object from `configuration`
    */
-  private TableInfo getTableInfo(Configuration configuration) throws IOException {
+  public static TableInfo getTableInfo(Configuration configuration) throws IOException {
     String tableInfoStr = configuration.get(TABLE_INFO);
     if (tableInfoStr == null) {
       return null;
@@ -278,6 +286,7 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
     List<String> invalidSegments = new ArrayList<>();
     List<UpdateVO> invalidTimestampsList = new ArrayList<>();
     List<String> validSegments = Arrays.asList(getSegmentsToAccess(job));
+    List<String> streamSegments = null;
     // get all valid segments and set them into the configuration
     SegmentUpdateStatusManager updateStatusManager = new SegmentUpdateStatusManager(identifier);
     if (validSegments.size() == 0) {
@@ -285,8 +294,9 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
       SegmentStatusManager.ValidAndInvalidSegmentsInfo segments =
           segmentStatusManager.getValidAndInvalidSegments();
       validSegments = segments.getValidSegments();
+      streamSegments = segments.getStreamSegments();
       if (validSegments.size() == 0) {
-        return new ArrayList<>(0);
+        return getSplitsOfStreaming(job, identifier, streamSegments);
       }
       // remove entry in the segment index if there are invalid segments
       invalidSegments.addAll(segments.getInvalidSegments());
@@ -327,7 +337,7 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
       throw new IOException("Missing/Corrupt schema file for table.");
     }
     PartitionInfo partitionInfo = carbonTable.getPartitionInfo(carbonTable.getFactTableName());
-    CarbonInputFormatUtil.processFilterExpression(filter, carbonTable);
+    CarbonInputFormatUtil.processFilterExpression(filter, carbonTable, null, null);
 
     // prune partitions for filter query on partition table
     BitSet matchedPartitions = null;
@@ -356,9 +366,93 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
             .setInvalidTimestampRange(invalidTimestampsList);
       }
     }
+
+    // add all splits of streaming
+    List<InputSplit> splitsOfStreaming = getSplitsOfStreaming(job, identifier, streamSegments);
+    if (!splitsOfStreaming.isEmpty()) {
+      splits.addAll(splitsOfStreaming);
+    }
+    return splits;
+  }
+
+  /**
+   * use file list in .carbonindex file to get the split of streaming.
+   */
+  private List<InputSplit> getSplitsOfStreaming(JobContext job, AbsoluteTableIdentifier identifier,
+      List<String> streamSegments) throws IOException {
+    List<InputSplit> splits = new ArrayList<InputSplit>();
+    if (streamSegments != null && !streamSegments.isEmpty()) {
+
+      CarbonTablePath tablePath = CarbonStorePath.getCarbonTablePath(identifier);
+      long minSize = Math.max(getFormatMinSplitSize(), getMinSplitSize(job));
+      long maxSize = getMaxSplitSize(job);
+      for (String segmentId : streamSegments) {
+        String segmentDir = tablePath.getSegmentDir("0", segmentId);
+        FileFactory.FileType fileType = FileFactory.getFileType(segmentDir);
+        if (FileFactory.isFileExist(segmentDir, fileType)) {
+          String indexName = CarbonTablePath.getCarbonStreamIndexFileName();
+          String indexPath = segmentDir + File.separator + indexName;
+          CarbonFile index = FileFactory.getCarbonFile(indexPath, fileType);
+          // index file exists
+          if (index.exists()) {
+            // data file exists
+            CarbonIndexFileReader indexReader = new CarbonIndexFileReader();
+            try {
+              // map block index
+              indexReader.openThriftReader(indexPath);
+              while (indexReader.hasNext()) {
+                BlockIndex blockIndex = indexReader.readBlockIndexInfo();
+                String filePath = segmentDir + File.separator + blockIndex.getFile_name();
+                Path path = new Path(filePath);
+                long length = blockIndex.getFile_size();
+                if (length != 0) {
+                  BlockLocation[] blkLocations;
+                  FileSystem fs = FileFactory.getFileSystem(path);
+                  FileStatus file = fs.getFileStatus(path);
+                  blkLocations = fs.getFileBlockLocations(path, 0, length);
+                  long blockSize = file.getBlockSize();
+                  long splitSize = computeSplitSize(blockSize, minSize, maxSize);
+                  long bytesRemaining = length;
+                  while (((double) bytesRemaining) / splitSize > 1.1) {
+                    int blkIndex = getBlockIndex(blkLocations, length - bytesRemaining);
+                    splits.add(makeSplit(segmentId, path, length - bytesRemaining, splitSize,
+                        blkLocations[blkIndex].getHosts(),
+                        blkLocations[blkIndex].getCachedHosts(), FileFormat.rowformat));
+                    bytesRemaining -= splitSize;
+                  }
+                  if (bytesRemaining != 0) {
+                    int blkIndex = getBlockIndex(blkLocations, length - bytesRemaining);
+                    splits.add(makeSplit(segmentId, path, length - bytesRemaining, bytesRemaining,
+                        blkLocations[blkIndex].getHosts(),
+                        blkLocations[blkIndex].getCachedHosts(), FileFormat.rowformat));
+                  }
+                } else {
+                  //Create empty hosts array for zero length files
+                  splits.add(makeSplit(segmentId, path, 0, length, new String[0],
+                      FileFormat.rowformat));
+                }
+              }
+            } finally {
+              indexReader.closeThriftReader();
+            }
+          }
+        }
+      }
+    }
     return splits;
   }
 
+  protected FileSplit makeSplit(String segmentId, Path file, long start, long length,
+      String[] hosts, FileFormat fileFormat) {
+    return new CarbonInputSplit(segmentId, file, start, length, hosts, fileFormat);
+  }
+
+
+  protected FileSplit makeSplit(String segmentId, Path file, long start, long length,
+      String[] hosts, String[] inMemoryHosts, FileFormat fileFormat) {
+    return new CarbonInputSplit(segmentId, file, start, length, hosts, inMemoryHosts, fileFormat);
+  }
+
   /**
    * Read data in one segment. For alter table partition statement
    * @param job
@@ -387,7 +481,7 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
         throw new IOException("Missing/Corrupt schema file for table.");
       }
 
-      CarbonInputFormatUtil.processFilterExpression(filter, carbonTable);
+      CarbonInputFormatUtil.processFilterExpression(filter, carbonTable, null, null);
 
       TableProvider tableProvider = new SingleTableProvider(carbonTable);
       // prune partitions for filter query on partition table
@@ -633,7 +727,13 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
 
     // set the filter to the query model in order to filter blocklet before scan
     Expression filter = getFilterPredicates(configuration);
-    CarbonInputFormatUtil.processFilterExpression(filter, carbonTable);
+    boolean[] isFilterDimensions = new boolean[carbonTable.getDimensionOrdinalMax()];
+    boolean[] isFilterMeasures =
+        new boolean[carbonTable.getNumberOfMeasures(carbonTable.getFactTableName())];
+    CarbonInputFormatUtil.processFilterExpression(filter, carbonTable, isFilterDimensions,
+        isFilterMeasures);
+    queryModel.setIsFilterDimensions(isFilterDimensions);
+    queryModel.setIsFilterMeasures(isFilterMeasures);
     FilterResolverIntf filterIntf = CarbonInputFormatUtil
         .resolveFilter(filter, carbonTable.getAbsoluteTableIdentifier(), tableProvider);
     queryModel.setFilterExpressionResolverTree(filterIntf);


[33/49] 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/fgdatamap
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,


[19/49] carbondata git commit: [CARBONDATA-1656][Streaming] Reject alter table command for streaming table

Posted by ra...@apache.org.
[CARBONDATA-1656][Streaming] Reject alter table command for streaming table

This closes #1448


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

Branch: refs/heads/fgdatamap
Commit: 87892522bfa73a4876e5cfe68bbf9d460a9a1f52
Parents: 4c41f86
Author: Jacky Li <ja...@qq.com>
Authored: Wed Nov 8 10:37:04 2017 +0800
Committer: QiangCai <qi...@qq.com>
Committed: Wed Nov 8 16:04:04 2017 +0800

----------------------------------------------------------------------
 .../schema/AlterTableAddColumnCommand.scala     | 115 ------------
 .../AlterTableDataTypeChangeCommand.scala       | 116 -------------
 .../schema/AlterTableDropColumnCommand.scala    | 148 ----------------
 .../schema/AlterTableRenameTableCommand.scala   | 174 -------------------
 .../CarbonAlterTableAddColumnCommand.scala      | 115 ++++++++++++
 .../CarbonAlterTableDataTypeChangeCommand.scala | 116 +++++++++++++
 .../CarbonAlterTableDropColumnCommand.scala     | 148 ++++++++++++++++
 .../schema/CarbonAlterTableRenameCommand.scala  | 174 +++++++++++++++++++
 .../sql/execution/strategy/DDLStrategy.scala    |  10 +-
 .../strategy/StreamingTableStrategy.scala       |  39 ++++-
 .../sql/parser/CarbonSpark2SqlParser.scala      |   8 +-
 .../TestStreamingTableOperation.scala           |  15 ++
 .../restructure/AlterTableRevertTestCase.scala  |   2 +-
 13 files changed, 610 insertions(+), 570 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/87892522/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableAddColumnCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableAddColumnCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableAddColumnCommand.scala
deleted file mode 100644
index 6e6a4b1..0000000
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableAddColumnCommand.scala
+++ /dev/null
@@ -1,115 +0,0 @@
-/*
- * 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.schema
-
-import scala.collection.JavaConverters._
-
-import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
-import org.apache.spark.sql.execution.command.{AlterTableAddColumnsModel, AlterTableColumnSchemaGenerator, RunnableCommand}
-import org.apache.spark.sql.hive.{CarbonRelation, CarbonSessionState}
-import org.apache.spark.util.AlterTableUtil
-
-import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
-import org.apache.carbondata.core.locks.{ICarbonLock, LockUsage}
-import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable
-import org.apache.carbondata.core.util.path.CarbonStorePath
-import org.apache.carbondata.format.TableInfo
-import org.apache.carbondata.spark.rdd.{AlterTableAddColumnRDD, AlterTableDropColumnRDD}
-
-private[sql] case class AlterTableAddColumnCommand(
-    alterTableAddColumnsModel: AlterTableAddColumnsModel)
-  extends RunnableCommand {
-
-  override def run(sparkSession: SparkSession): Seq[Row] = {
-    val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
-    val tableName = alterTableAddColumnsModel.tableName
-    val dbName = alterTableAddColumnsModel.databaseName
-      .getOrElse(sparkSession.catalog.currentDatabase)
-    LOGGER.audit(s"Alter table add columns request has been received for $dbName.$tableName")
-    val locksToBeAcquired = List(LockUsage.METADATA_LOCK, LockUsage.COMPACTION_LOCK)
-    var locks = List.empty[ICarbonLock]
-    var timeStamp = 0L
-    var newCols = Seq[org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema]()
-    var carbonTable: CarbonTable = null
-    try {
-      locks = AlterTableUtil
-        .validateTableAndAcquireLock(dbName, tableName, locksToBeAcquired)(sparkSession)
-      // Consider a concurrent scenario where 2 alter operations are executed in parallel. 1st
-      // operation is success and updates the schema file. 2nd operation will get the lock after
-      // completion of 1st operation but as look up relation is called before it will have the
-      // older carbon table and this can lead to inconsistent state in the system. Therefor look
-      // up relation should be called after acquiring the lock
-      val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
-      carbonTable = metastore
-        .lookupRelation(Some(dbName), tableName)(sparkSession).asInstanceOf[CarbonRelation]
-        .tableMeta.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)
-      newCols = new AlterTableColumnSchemaGenerator(alterTableAddColumnsModel,
-        dbName,
-        wrapperTableInfo,
-        carbonTablePath,
-        carbonTable.getCarbonTableIdentifier,
-        carbonTable.getStorePath, sparkSession.sparkContext).process
-      // generate dictionary files for the newly added columns
-      new AlterTableAddColumnRDD(sparkSession.sparkContext,
-        newCols,
-        carbonTable.getCarbonTableIdentifier,
-        carbonTable.getStorePath).collect()
-      timeStamp = System.currentTimeMillis
-      val schemaEvolutionEntry = new org.apache.carbondata.core.metadata.schema.SchemaEvolutionEntry
-      schemaEvolutionEntry.setTimeStamp(timeStamp)
-      schemaEvolutionEntry.setAdded(newCols.toList.asJava)
-      val thriftTable = schemaConverter
-        .fromWrapperToExternalTableInfo(wrapperTableInfo, dbName, tableName)
-      AlterTableUtil
-        .updateSchemaInfo(carbonTable,
-          schemaConverter.fromWrapperToExternalSchemaEvolutionEntry(schemaEvolutionEntry),
-          thriftTable)(sparkSession,
-          sparkSession.sessionState.asInstanceOf[CarbonSessionState])
-      LOGGER.info(s"Alter table for add columns is successful for table $dbName.$tableName")
-      LOGGER.audit(s"Alter table for add columns is successful for table $dbName.$tableName")
-    } catch {
-      case e: Exception =>
-        LOGGER.error(e, "Alter table add columns failed")
-        if (newCols.nonEmpty) {
-          LOGGER.info("Cleaning up the dictionary files as alter table add operation failed")
-          new AlterTableDropColumnRDD(sparkSession.sparkContext,
-            newCols,
-            carbonTable.getCarbonTableIdentifier,
-            carbonTable.getStorePath).collect()
-          AlterTableUtil.revertAddColumnChanges(dbName, tableName, timeStamp)(sparkSession)
-        }
-        sys.error(s"Alter table add operation failed: ${e.getMessage}")
-    } finally {
-      // release lock after command execution completion
-      AlterTableUtil.releaseLocks(locks)
-    }
-    Seq.empty
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/87892522/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableDataTypeChangeCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableDataTypeChangeCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableDataTypeChangeCommand.scala
deleted file mode 100644
index be87bbb..0000000
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableDataTypeChangeCommand.scala
+++ /dev/null
@@ -1,116 +0,0 @@
-/*
- * 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.schema
-
-import scala.collection.JavaConverters._
-
-import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
-import org.apache.spark.sql.execution.command.{AlterTableDataTypeChangeModel, RunnableCommand}
-import org.apache.spark.sql.hive.{CarbonRelation, CarbonSessionState}
-import org.apache.spark.util.AlterTableUtil
-
-import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
-import org.apache.carbondata.core.locks.{ICarbonLock, LockUsage}
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable
-import org.apache.carbondata.core.util.path.CarbonStorePath
-import org.apache.carbondata.format.{ColumnSchema, SchemaEvolutionEntry, TableInfo}
-import org.apache.carbondata.spark.util.{CarbonScalaUtil, DataTypeConverterUtil}
-
-private[sql] case class AlterTableDataTypeChangeCommand(
-    alterTableDataTypeChangeModel: AlterTableDataTypeChangeModel)
-  extends RunnableCommand {
-
-  override def run(sparkSession: SparkSession): Seq[Row] = {
-    val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
-    val tableName = alterTableDataTypeChangeModel.tableName
-    val dbName = alterTableDataTypeChangeModel.databaseName
-      .getOrElse(sparkSession.catalog.currentDatabase)
-    LOGGER.audit(s"Alter table change data type request has been received for $dbName.$tableName")
-    val locksToBeAcquired = List(LockUsage.METADATA_LOCK, LockUsage.COMPACTION_LOCK)
-    var locks = List.empty[ICarbonLock]
-    // get the latest carbon table and check for column existence
-    var carbonTable: CarbonTable = null
-    var timeStamp = 0L
-    try {
-      locks = AlterTableUtil
-        .validateTableAndAcquireLock(dbName, tableName, locksToBeAcquired)(sparkSession)
-      val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
-      carbonTable = metastore
-        .lookupRelation(Some(dbName), tableName)(sparkSession).asInstanceOf[CarbonRelation]
-        .tableMeta.carbonTable
-      val columnName = alterTableDataTypeChangeModel.columnName
-      val carbonColumns = carbonTable.getCreateOrderColumn(tableName).asScala.filter(!_.isInvisible)
-      if (!carbonColumns.exists(_.getColName.equalsIgnoreCase(columnName))) {
-        LOGGER.audit(s"Alter table change data type request has failed. " +
-                     s"Column $columnName does not exist")
-        sys.error(s"Column does not exist: $columnName")
-      }
-      val carbonColumn = carbonColumns.filter(_.getColName.equalsIgnoreCase(columnName))
-      if (carbonColumn.size == 1) {
-        CarbonScalaUtil
-          .validateColumnDataType(alterTableDataTypeChangeModel.dataTypeInfo, carbonColumn.head)
-      } else {
-        LOGGER.audit(s"Alter table change data type request has failed. " +
-                     s"Column $columnName is invalid")
-        sys.error(s"Invalid Column: $columnName")
-      }
-      // read the latest schema file
-      val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getStorePath,
-        carbonTable.getCarbonTableIdentifier)
-      val tableInfo: TableInfo = metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
-      // maintain the added column for schema evolution history
-      var addColumnSchema: ColumnSchema = null
-      var deletedColumnSchema: ColumnSchema = null
-      val columnSchemaList = tableInfo.fact_table.table_columns.asScala.filter(!_.isInvisible)
-      columnSchemaList.foreach { columnSchema =>
-        if (columnSchema.column_name.equalsIgnoreCase(columnName)) {
-          deletedColumnSchema = columnSchema.deepCopy
-          columnSchema.setData_type(DataTypeConverterUtil
-            .convertToThriftDataType(alterTableDataTypeChangeModel.dataTypeInfo.dataType))
-          columnSchema.setPrecision(alterTableDataTypeChangeModel.dataTypeInfo.precision)
-          columnSchema.setScale(alterTableDataTypeChangeModel.dataTypeInfo.scale)
-          addColumnSchema = columnSchema
-        }
-      }
-      timeStamp = System.currentTimeMillis
-      val schemaEvolutionEntry = new SchemaEvolutionEntry(timeStamp)
-      schemaEvolutionEntry.setAdded(List(addColumnSchema).asJava)
-      schemaEvolutionEntry.setRemoved(List(deletedColumnSchema).asJava)
-      tableInfo.getFact_table.getSchema_evolution.getSchema_evolution_history.get(0)
-        .setTime_stamp(System.currentTimeMillis)
-      AlterTableUtil
-        .updateSchemaInfo(carbonTable,
-          schemaEvolutionEntry,
-          tableInfo)(sparkSession,
-          sparkSession.sessionState.asInstanceOf[CarbonSessionState])
-      LOGGER.info(s"Alter table for data type change is successful for table $dbName.$tableName")
-      LOGGER.audit(s"Alter table for data type change is successful for table $dbName.$tableName")
-    } catch {
-      case e: Exception => LOGGER
-        .error("Alter table change datatype failed : " + e.getMessage)
-        if (carbonTable != null) {
-          AlterTableUtil.revertDataTypeChanges(dbName, tableName, timeStamp)(sparkSession)
-        }
-        sys.error(s"Alter table data type change operation failed: ${e.getMessage}")
-    } finally {
-      // release lock after command execution completion
-      AlterTableUtil.releaseLocks(locks)
-    }
-    Seq.empty
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/87892522/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableDropColumnCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableDropColumnCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableDropColumnCommand.scala
deleted file mode 100644
index 2f1e3d9..0000000
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableDropColumnCommand.scala
+++ /dev/null
@@ -1,148 +0,0 @@
-/*
- * 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.schema
-
-import scala.collection.JavaConverters._
-import scala.collection.mutable.ListBuffer
-
-import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
-import org.apache.spark.sql.execution.command.{AlterTableDropColumnModel, RunnableCommand}
-import org.apache.spark.sql.hive.{CarbonRelation, CarbonSessionState}
-import org.apache.spark.util.AlterTableUtil
-
-import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
-import org.apache.carbondata.core.locks.{ICarbonLock, LockUsage}
-import org.apache.carbondata.core.metadata.encoder.Encoding
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable
-import org.apache.carbondata.core.util.path.CarbonStorePath
-import org.apache.carbondata.format.SchemaEvolutionEntry
-import org.apache.carbondata.spark.rdd.AlterTableDropColumnRDD
-
-private[sql] case class AlterTableDropColumnCommand(
-    alterTableDropColumnModel: AlterTableDropColumnModel)
-  extends RunnableCommand {
-
-  override def run(sparkSession: SparkSession): Seq[Row] = {
-    val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
-    val tableName = alterTableDropColumnModel.tableName
-    val dbName = alterTableDropColumnModel.databaseName
-      .getOrElse(sparkSession.catalog.currentDatabase)
-    LOGGER.audit(s"Alter table drop columns request has been received for $dbName.$tableName")
-    var locks = List.empty[ICarbonLock]
-    var timeStamp = 0L
-    val locksToBeAcquired = List(LockUsage.METADATA_LOCK, LockUsage.COMPACTION_LOCK)
-    // get the latest carbon table and check for column existence
-    var carbonTable: CarbonTable = null
-    try {
-      locks = AlterTableUtil
-        .validateTableAndAcquireLock(dbName, tableName, locksToBeAcquired)(sparkSession)
-      val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
-      carbonTable = metastore
-        .lookupRelation(Some(dbName), tableName)(sparkSession).asInstanceOf[CarbonRelation]
-        .tableMeta.carbonTable
-      val partitionInfo = carbonTable.getPartitionInfo(tableName)
-      if (partitionInfo != null) {
-        val partitionColumnSchemaList = partitionInfo.getColumnSchemaList.asScala
-          .map(_.getColumnName)
-        // check each column existence in the table
-        val partitionColumns = alterTableDropColumnModel.columns.filter {
-          tableColumn => partitionColumnSchemaList.contains(tableColumn)
-        }
-        if (partitionColumns.nonEmpty) {
-          throw new UnsupportedOperationException("Partition columns cannot be dropped: " +
-                                                  s"$partitionColumns")
-        }
-      }
-      val tableColumns = carbonTable.getCreateOrderColumn(tableName).asScala
-      var dictionaryColumns = Seq[org.apache.carbondata.core.metadata.schema.table.column
-      .ColumnSchema]()
-      var keyColumnCountToBeDeleted = 0
-      // TODO: if deleted column list includes bucketted column throw an error
-      alterTableDropColumnModel.columns.foreach { column =>
-        var columnExist = false
-        tableColumns.foreach { tableColumn =>
-          // column should not be already deleted and should exist in the table
-          if (!tableColumn.isInvisible && column.equalsIgnoreCase(tableColumn.getColName)) {
-            if (tableColumn.isDimension) {
-              keyColumnCountToBeDeleted += 1
-              if (tableColumn.hasEncoding(Encoding.DICTIONARY)) {
-                dictionaryColumns ++= Seq(tableColumn.getColumnSchema)
-              }
-            }
-            columnExist = true
-          }
-        }
-        if (!columnExist) {
-          sys.error(s"Column $column does not exists in the table $dbName.$tableName")
-        }
-      }
-      // take the total key column count. key column to be deleted should not
-      // be >= key columns in schema
-      val totalKeyColumnInSchema = tableColumns.count {
-        tableColumn => !tableColumn.isInvisible && tableColumn.isDimension
-      }
-      if (keyColumnCountToBeDeleted >= totalKeyColumnInSchema) {
-        sys.error(s"Alter drop operation failed. AtLeast one key column should exist after drop.")
-      }
-      // read the latest schema file
-      val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getStorePath,
-        carbonTable.getCarbonTableIdentifier)
-      val tableInfo: org.apache.carbondata.format.TableInfo =
-        metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
-      // maintain the deleted columns for schema evolution history
-      var deletedColumnSchema = ListBuffer[org.apache.carbondata.format.ColumnSchema]()
-      val columnSchemaList = tableInfo.fact_table.table_columns.asScala
-      alterTableDropColumnModel.columns.foreach { column =>
-        columnSchemaList.foreach { columnSchema =>
-          if (!columnSchema.invisible && column.equalsIgnoreCase(columnSchema.column_name)) {
-            deletedColumnSchema += columnSchema.deepCopy
-            columnSchema.invisible = true
-          }
-        }
-      }
-      // add deleted columns to schema evolution history and update the schema
-      timeStamp = System.currentTimeMillis
-      val schemaEvolutionEntry = new SchemaEvolutionEntry(timeStamp)
-      schemaEvolutionEntry.setRemoved(deletedColumnSchema.toList.asJava)
-      AlterTableUtil
-        .updateSchemaInfo(carbonTable,
-          schemaEvolutionEntry,
-          tableInfo)(sparkSession,
-          sparkSession.sessionState.asInstanceOf[CarbonSessionState])
-      // TODO: 1. add check for deletion of index tables
-      // delete dictionary files for dictionary column and clear dictionary cache from memory
-      new AlterTableDropColumnRDD(sparkSession.sparkContext,
-        dictionaryColumns,
-        carbonTable.getCarbonTableIdentifier,
-        carbonTable.getStorePath).collect()
-      LOGGER.info(s"Alter table for drop columns is successful for table $dbName.$tableName")
-      LOGGER.audit(s"Alter table for drop columns is successful for table $dbName.$tableName")
-    } catch {
-      case e: Exception => LOGGER
-        .error("Alter table drop columns failed : " + e.getMessage)
-        if (carbonTable != null) {
-          AlterTableUtil.revertDropColumnChanges(dbName, tableName, timeStamp)(sparkSession)
-        }
-        sys.error(s"Alter table drop column operation failed: ${e.getMessage}")
-    } finally {
-      // release lock after command execution completion
-      AlterTableUtil.releaseLocks(locks)
-    }
-    Seq.empty
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/87892522/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableRenameTableCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableRenameTableCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableRenameTableCommand.scala
deleted file mode 100644
index af361d5..0000000
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableRenameTableCommand.scala
+++ /dev/null
@@ -1,174 +0,0 @@
-/*
- * 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.schema
-
-import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
-import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.execution.command.{AlterTableRenameModel, RunnableCommand}
-import org.apache.spark.sql.hive.{CarbonRelation, CarbonSessionState}
-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.datastore.impl.FileFactory
-import org.apache.carbondata.core.locks.{ICarbonLock, LockUsage}
-import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, 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.format.SchemaEvolutionEntry
-import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
-
-private[sql] case class AlterTableRenameTableCommand(
-    alterTableRenameModel: AlterTableRenameModel)
-  extends RunnableCommand {
-
-  override def run(sparkSession: SparkSession): Seq[Row] = {
-    val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
-    val oldTableIdentifier = alterTableRenameModel.oldTableIdentifier
-    val newTableIdentifier = alterTableRenameModel.newTableIdentifier
-    val oldDatabaseName = oldTableIdentifier.database
-      .getOrElse(sparkSession.catalog.currentDatabase)
-    val newDatabaseName = newTableIdentifier.database
-      .getOrElse(sparkSession.catalog.currentDatabase)
-    if (!oldDatabaseName.equalsIgnoreCase(newDatabaseName)) {
-      throw new MalformedCarbonCommandException("Database name should be same for both tables")
-    }
-    val tableExists = sparkSession.catalog.tableExists(oldDatabaseName, newTableIdentifier.table)
-    if (tableExists) {
-      throw new MalformedCarbonCommandException(s"Table with name $newTableIdentifier " +
-                                                s"already exists")
-    }
-    val oldTableName = oldTableIdentifier.table.toLowerCase
-    val newTableName = newTableIdentifier.table.toLowerCase
-    LOGGER.audit(s"Rename table request has been received for $oldDatabaseName.$oldTableName")
-    LOGGER.info(s"Rename table request has been received for $oldDatabaseName.$oldTableName")
-    val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
-    val relation: CarbonRelation =
-      metastore.lookupRelation(oldTableIdentifier.database, oldTableName)(sparkSession)
-        .asInstanceOf[CarbonRelation]
-    if (relation == null) {
-      LOGGER.audit(s"Rename table request has failed. " +
-                   s"Table $oldDatabaseName.$oldTableName does not exist")
-      sys.error(s"Table $oldDatabaseName.$oldTableName does not exist")
-    }
-    val locksToBeAcquired = List(LockUsage.METADATA_LOCK,
-      LockUsage.COMPACTION_LOCK,
-      LockUsage.DELETE_SEGMENT_LOCK,
-      LockUsage.CLEAN_FILES_LOCK,
-      LockUsage.DROP_TABLE_LOCK)
-    var locks = List.empty[ICarbonLock]
-    var timeStamp = 0L
-    var carbonTable: CarbonTable = null
-    try {
-      locks = AlterTableUtil
-        .validateTableAndAcquireLock(oldDatabaseName, oldTableName, locksToBeAcquired)(
-          sparkSession)
-      val tableMeta = metastore.lookupRelation(Some(oldDatabaseName), oldTableName)(sparkSession)
-        .asInstanceOf[CarbonRelation].tableMeta
-      carbonTable = tableMeta.carbonTable
-      // get the latest carbon table and check for column existence
-      val carbonTablePath = CarbonStorePath.
-        getCarbonTablePath(AbsoluteTableIdentifier.fromTablePath(tableMeta.tablePath))
-      val tableMetadataFile = carbonTablePath.getPath
-      val tableInfo: org.apache.carbondata.format.TableInfo =
-        metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
-      val schemaEvolutionEntry = new SchemaEvolutionEntry(System.currentTimeMillis)
-      schemaEvolutionEntry.setTableName(newTableName)
-      timeStamp = System.currentTimeMillis()
-      schemaEvolutionEntry.setTime_stamp(timeStamp)
-      renameBadRecords(oldTableName, newTableName, oldDatabaseName)
-      val fileType = FileFactory.getFileType(tableMetadataFile)
-      if (FileFactory.isFileExist(tableMetadataFile, fileType)) {
-        val rename = FileFactory.getCarbonFile(carbonTablePath.getPath, fileType)
-          .renameForce(carbonTablePath.getParent.toString + CarbonCommonConstants.FILE_SEPARATOR +
-                       newTableName)
-        if (!rename) {
-          renameBadRecords(newTableName, oldTableName, oldDatabaseName)
-          sys.error(s"Folder rename failed for table $oldDatabaseName.$oldTableName")
-        }
-      }
-      val newTableIdentifier = new CarbonTableIdentifier(oldDatabaseName,
-        newTableName, carbonTable.getCarbonTableIdentifier.getTableId)
-      val newTablePath = metastore.updateTableSchema(newTableIdentifier,
-        carbonTable.getCarbonTableIdentifier,
-        tableInfo,
-        schemaEvolutionEntry,
-        tableMeta.tablePath)(sparkSession)
-      metastore.removeTableFromMetadata(oldDatabaseName, oldTableName)
-      sparkSession.sessionState.asInstanceOf[CarbonSessionState].metadataHive
-        .runSqlHive(
-          s"ALTER TABLE $oldDatabaseName.$oldTableName RENAME TO $oldDatabaseName.$newTableName")
-      sparkSession.sessionState.asInstanceOf[CarbonSessionState].metadataHive
-        .runSqlHive(
-          s"ALTER TABLE $oldDatabaseName.$newTableName SET SERDEPROPERTIES" +
-          s"('tableName'='$newTableName', " +
-          s"'dbName'='$oldDatabaseName', 'tablePath'='$newTablePath')")
-      sparkSession.catalog.refreshTable(TableIdentifier(newTableName,
-        Some(oldDatabaseName)).quotedString)
-      LOGGER.audit(s"Table $oldTableName has been successfully renamed to $newTableName")
-      LOGGER.info(s"Table $oldTableName has been successfully renamed to $newTableName")
-    } catch {
-      case e: Exception =>
-        LOGGER.error(e, "Rename table failed: " + e.getMessage)
-        if (carbonTable != null) {
-          AlterTableUtil
-            .revertRenameTableChanges(oldTableIdentifier,
-              newTableName,
-              carbonTable.getStorePath,
-              carbonTable.getCarbonTableIdentifier.getTableId,
-              timeStamp)(
-              sparkSession)
-          renameBadRecords(newTableName, oldTableName, oldDatabaseName)
-        }
-        sys.error(s"Alter table rename table operation failed: ${e.getMessage}")
-    } finally {
-      // release lock after command execution completion
-      AlterTableUtil.releaseLocks(locks)
-      // case specific to rename table as after table rename old table path will not be found
-      if (carbonTable != null) {
-        AlterTableUtil
-          .releaseLocksManually(locks,
-            locksToBeAcquired,
-            oldDatabaseName,
-            newTableName,
-            carbonTable.getStorePath)
-      }
-    }
-    Seq.empty
-  }
-
-  private def renameBadRecords(
-      oldTableName: String,
-      newTableName: String,
-      dataBaseName: String): Unit = {
-    val oldPath = CarbonUtil
-      .getBadLogPath(dataBaseName + CarbonCommonConstants.FILE_SEPARATOR + oldTableName)
-    val newPath = CarbonUtil
-      .getBadLogPath(dataBaseName + CarbonCommonConstants.FILE_SEPARATOR + newTableName)
-    val fileType = FileFactory.getFileType(oldPath)
-    if (FileFactory.isFileExist(oldPath, fileType)) {
-      val renameSuccess = FileFactory.getCarbonFile(oldPath, fileType)
-        .renameForce(newPath)
-      if (!renameSuccess) {
-        sys.error(s"BadRecords Folder Rename Failed for table $dataBaseName.$oldTableName")
-      }
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/87892522/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableAddColumnCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableAddColumnCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableAddColumnCommand.scala
new file mode 100644
index 0000000..8737464
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableAddColumnCommand.scala
@@ -0,0 +1,115 @@
+/*
+ * 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.schema
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
+import org.apache.spark.sql.execution.command.{AlterTableAddColumnsModel, AlterTableColumnSchemaGenerator, RunnableCommand}
+import org.apache.spark.sql.hive.{CarbonRelation, CarbonSessionState}
+import org.apache.spark.util.AlterTableUtil
+
+import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
+import org.apache.carbondata.core.locks.{ICarbonLock, LockUsage}
+import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.format.TableInfo
+import org.apache.carbondata.spark.rdd.{AlterTableAddColumnRDD, AlterTableDropColumnRDD}
+
+private[sql] case class CarbonAlterTableAddColumnCommand(
+    alterTableAddColumnsModel: AlterTableAddColumnsModel)
+  extends RunnableCommand {
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+    val tableName = alterTableAddColumnsModel.tableName
+    val dbName = alterTableAddColumnsModel.databaseName
+      .getOrElse(sparkSession.catalog.currentDatabase)
+    LOGGER.audit(s"Alter table add columns request has been received for $dbName.$tableName")
+    val locksToBeAcquired = List(LockUsage.METADATA_LOCK, LockUsage.COMPACTION_LOCK)
+    var locks = List.empty[ICarbonLock]
+    var timeStamp = 0L
+    var newCols = Seq[org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema]()
+    var carbonTable: CarbonTable = null
+    try {
+      locks = AlterTableUtil
+        .validateTableAndAcquireLock(dbName, tableName, locksToBeAcquired)(sparkSession)
+      // Consider a concurrent scenario where 2 alter operations are executed in parallel. 1st
+      // operation is success and updates the schema file. 2nd operation will get the lock after
+      // completion of 1st operation but as look up relation is called before it will have the
+      // older carbon table and this can lead to inconsistent state in the system. Therefor look
+      // up relation should be called after acquiring the lock
+      val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
+      carbonTable = metastore
+        .lookupRelation(Some(dbName), tableName)(sparkSession).asInstanceOf[CarbonRelation]
+        .tableMeta.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)
+      newCols = new AlterTableColumnSchemaGenerator(alterTableAddColumnsModel,
+        dbName,
+        wrapperTableInfo,
+        carbonTablePath,
+        carbonTable.getCarbonTableIdentifier,
+        carbonTable.getStorePath, sparkSession.sparkContext).process
+      // generate dictionary files for the newly added columns
+      new AlterTableAddColumnRDD(sparkSession.sparkContext,
+        newCols,
+        carbonTable.getCarbonTableIdentifier,
+        carbonTable.getStorePath).collect()
+      timeStamp = System.currentTimeMillis
+      val schemaEvolutionEntry = new org.apache.carbondata.core.metadata.schema.SchemaEvolutionEntry
+      schemaEvolutionEntry.setTimeStamp(timeStamp)
+      schemaEvolutionEntry.setAdded(newCols.toList.asJava)
+      val thriftTable = schemaConverter
+        .fromWrapperToExternalTableInfo(wrapperTableInfo, dbName, tableName)
+      AlterTableUtil
+        .updateSchemaInfo(carbonTable,
+          schemaConverter.fromWrapperToExternalSchemaEvolutionEntry(schemaEvolutionEntry),
+          thriftTable)(sparkSession,
+          sparkSession.sessionState.asInstanceOf[CarbonSessionState])
+      LOGGER.info(s"Alter table for add columns is successful for table $dbName.$tableName")
+      LOGGER.audit(s"Alter table for add columns is successful for table $dbName.$tableName")
+    } catch {
+      case e: Exception =>
+        LOGGER.error(e, "Alter table add columns failed")
+        if (newCols.nonEmpty) {
+          LOGGER.info("Cleaning up the dictionary files as alter table add operation failed")
+          new AlterTableDropColumnRDD(sparkSession.sparkContext,
+            newCols,
+            carbonTable.getCarbonTableIdentifier,
+            carbonTable.getStorePath).collect()
+          AlterTableUtil.revertAddColumnChanges(dbName, tableName, timeStamp)(sparkSession)
+        }
+        sys.error(s"Alter table add operation failed: ${e.getMessage}")
+    } finally {
+      // release lock after command execution completion
+      AlterTableUtil.releaseLocks(locks)
+    }
+    Seq.empty
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/87892522/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDataTypeChangeCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDataTypeChangeCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDataTypeChangeCommand.scala
new file mode 100644
index 0000000..4e180c8
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDataTypeChangeCommand.scala
@@ -0,0 +1,116 @@
+/*
+ * 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.schema
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
+import org.apache.spark.sql.execution.command.{AlterTableDataTypeChangeModel, RunnableCommand}
+import org.apache.spark.sql.hive.{CarbonRelation, CarbonSessionState}
+import org.apache.spark.util.AlterTableUtil
+
+import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
+import org.apache.carbondata.core.locks.{ICarbonLock, LockUsage}
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.format.{ColumnSchema, SchemaEvolutionEntry, TableInfo}
+import org.apache.carbondata.spark.util.{CarbonScalaUtil, DataTypeConverterUtil}
+
+private[sql] case class CarbonAlterTableDataTypeChangeCommand(
+    alterTableDataTypeChangeModel: AlterTableDataTypeChangeModel)
+  extends RunnableCommand {
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+    val tableName = alterTableDataTypeChangeModel.tableName
+    val dbName = alterTableDataTypeChangeModel.databaseName
+      .getOrElse(sparkSession.catalog.currentDatabase)
+    LOGGER.audit(s"Alter table change data type request has been received for $dbName.$tableName")
+    val locksToBeAcquired = List(LockUsage.METADATA_LOCK, LockUsage.COMPACTION_LOCK)
+    var locks = List.empty[ICarbonLock]
+    // get the latest carbon table and check for column existence
+    var carbonTable: CarbonTable = null
+    var timeStamp = 0L
+    try {
+      locks = AlterTableUtil
+        .validateTableAndAcquireLock(dbName, tableName, locksToBeAcquired)(sparkSession)
+      val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
+      carbonTable = metastore
+        .lookupRelation(Some(dbName), tableName)(sparkSession).asInstanceOf[CarbonRelation]
+        .tableMeta.carbonTable
+      val columnName = alterTableDataTypeChangeModel.columnName
+      val carbonColumns = carbonTable.getCreateOrderColumn(tableName).asScala.filter(!_.isInvisible)
+      if (!carbonColumns.exists(_.getColName.equalsIgnoreCase(columnName))) {
+        LOGGER.audit(s"Alter table change data type request has failed. " +
+                     s"Column $columnName does not exist")
+        sys.error(s"Column does not exist: $columnName")
+      }
+      val carbonColumn = carbonColumns.filter(_.getColName.equalsIgnoreCase(columnName))
+      if (carbonColumn.size == 1) {
+        CarbonScalaUtil
+          .validateColumnDataType(alterTableDataTypeChangeModel.dataTypeInfo, carbonColumn.head)
+      } else {
+        LOGGER.audit(s"Alter table change data type request has failed. " +
+                     s"Column $columnName is invalid")
+        sys.error(s"Invalid Column: $columnName")
+      }
+      // read the latest schema file
+      val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getStorePath,
+        carbonTable.getCarbonTableIdentifier)
+      val tableInfo: TableInfo = metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
+      // maintain the added column for schema evolution history
+      var addColumnSchema: ColumnSchema = null
+      var deletedColumnSchema: ColumnSchema = null
+      val columnSchemaList = tableInfo.fact_table.table_columns.asScala.filter(!_.isInvisible)
+      columnSchemaList.foreach { columnSchema =>
+        if (columnSchema.column_name.equalsIgnoreCase(columnName)) {
+          deletedColumnSchema = columnSchema.deepCopy
+          columnSchema.setData_type(DataTypeConverterUtil
+            .convertToThriftDataType(alterTableDataTypeChangeModel.dataTypeInfo.dataType))
+          columnSchema.setPrecision(alterTableDataTypeChangeModel.dataTypeInfo.precision)
+          columnSchema.setScale(alterTableDataTypeChangeModel.dataTypeInfo.scale)
+          addColumnSchema = columnSchema
+        }
+      }
+      timeStamp = System.currentTimeMillis
+      val schemaEvolutionEntry = new SchemaEvolutionEntry(timeStamp)
+      schemaEvolutionEntry.setAdded(List(addColumnSchema).asJava)
+      schemaEvolutionEntry.setRemoved(List(deletedColumnSchema).asJava)
+      tableInfo.getFact_table.getSchema_evolution.getSchema_evolution_history.get(0)
+        .setTime_stamp(System.currentTimeMillis)
+      AlterTableUtil
+        .updateSchemaInfo(carbonTable,
+          schemaEvolutionEntry,
+          tableInfo)(sparkSession,
+          sparkSession.sessionState.asInstanceOf[CarbonSessionState])
+      LOGGER.info(s"Alter table for data type change is successful for table $dbName.$tableName")
+      LOGGER.audit(s"Alter table for data type change is successful for table $dbName.$tableName")
+    } catch {
+      case e: Exception => LOGGER
+        .error("Alter table change datatype failed : " + e.getMessage)
+        if (carbonTable != null) {
+          AlterTableUtil.revertDataTypeChanges(dbName, tableName, timeStamp)(sparkSession)
+        }
+        sys.error(s"Alter table data type change operation failed: ${e.getMessage}")
+    } finally {
+      // release lock after command execution completion
+      AlterTableUtil.releaseLocks(locks)
+    }
+    Seq.empty
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/87892522/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDropColumnCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDropColumnCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDropColumnCommand.scala
new file mode 100644
index 0000000..3ac23f7
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDropColumnCommand.scala
@@ -0,0 +1,148 @@
+/*
+ * 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.schema
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
+import org.apache.spark.sql.execution.command.{AlterTableDropColumnModel, RunnableCommand}
+import org.apache.spark.sql.hive.{CarbonRelation, CarbonSessionState}
+import org.apache.spark.util.AlterTableUtil
+
+import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
+import org.apache.carbondata.core.locks.{ICarbonLock, LockUsage}
+import org.apache.carbondata.core.metadata.encoder.Encoding
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.format.SchemaEvolutionEntry
+import org.apache.carbondata.spark.rdd.AlterTableDropColumnRDD
+
+private[sql] case class CarbonAlterTableDropColumnCommand(
+    alterTableDropColumnModel: AlterTableDropColumnModel)
+  extends RunnableCommand {
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+    val tableName = alterTableDropColumnModel.tableName
+    val dbName = alterTableDropColumnModel.databaseName
+      .getOrElse(sparkSession.catalog.currentDatabase)
+    LOGGER.audit(s"Alter table drop columns request has been received for $dbName.$tableName")
+    var locks = List.empty[ICarbonLock]
+    var timeStamp = 0L
+    val locksToBeAcquired = List(LockUsage.METADATA_LOCK, LockUsage.COMPACTION_LOCK)
+    // get the latest carbon table and check for column existence
+    var carbonTable: CarbonTable = null
+    try {
+      locks = AlterTableUtil
+        .validateTableAndAcquireLock(dbName, tableName, locksToBeAcquired)(sparkSession)
+      val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
+      carbonTable = metastore
+        .lookupRelation(Some(dbName), tableName)(sparkSession).asInstanceOf[CarbonRelation]
+        .tableMeta.carbonTable
+      val partitionInfo = carbonTable.getPartitionInfo(tableName)
+      if (partitionInfo != null) {
+        val partitionColumnSchemaList = partitionInfo.getColumnSchemaList.asScala
+          .map(_.getColumnName)
+        // check each column existence in the table
+        val partitionColumns = alterTableDropColumnModel.columns.filter {
+          tableColumn => partitionColumnSchemaList.contains(tableColumn)
+        }
+        if (partitionColumns.nonEmpty) {
+          throw new UnsupportedOperationException("Partition columns cannot be dropped: " +
+                                                  s"$partitionColumns")
+        }
+      }
+      val tableColumns = carbonTable.getCreateOrderColumn(tableName).asScala
+      var dictionaryColumns = Seq[org.apache.carbondata.core.metadata.schema.table.column
+      .ColumnSchema]()
+      var keyColumnCountToBeDeleted = 0
+      // TODO: if deleted column list includes bucketted column throw an error
+      alterTableDropColumnModel.columns.foreach { column =>
+        var columnExist = false
+        tableColumns.foreach { tableColumn =>
+          // column should not be already deleted and should exist in the table
+          if (!tableColumn.isInvisible && column.equalsIgnoreCase(tableColumn.getColName)) {
+            if (tableColumn.isDimension) {
+              keyColumnCountToBeDeleted += 1
+              if (tableColumn.hasEncoding(Encoding.DICTIONARY)) {
+                dictionaryColumns ++= Seq(tableColumn.getColumnSchema)
+              }
+            }
+            columnExist = true
+          }
+        }
+        if (!columnExist) {
+          sys.error(s"Column $column does not exists in the table $dbName.$tableName")
+        }
+      }
+      // take the total key column count. key column to be deleted should not
+      // be >= key columns in schema
+      val totalKeyColumnInSchema = tableColumns.count {
+        tableColumn => !tableColumn.isInvisible && tableColumn.isDimension
+      }
+      if (keyColumnCountToBeDeleted >= totalKeyColumnInSchema) {
+        sys.error(s"Alter drop operation failed. AtLeast one key column should exist after drop.")
+      }
+      // read the latest schema file
+      val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getStorePath,
+        carbonTable.getCarbonTableIdentifier)
+      val tableInfo: org.apache.carbondata.format.TableInfo =
+        metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
+      // maintain the deleted columns for schema evolution history
+      var deletedColumnSchema = ListBuffer[org.apache.carbondata.format.ColumnSchema]()
+      val columnSchemaList = tableInfo.fact_table.table_columns.asScala
+      alterTableDropColumnModel.columns.foreach { column =>
+        columnSchemaList.foreach { columnSchema =>
+          if (!columnSchema.invisible && column.equalsIgnoreCase(columnSchema.column_name)) {
+            deletedColumnSchema += columnSchema.deepCopy
+            columnSchema.invisible = true
+          }
+        }
+      }
+      // add deleted columns to schema evolution history and update the schema
+      timeStamp = System.currentTimeMillis
+      val schemaEvolutionEntry = new SchemaEvolutionEntry(timeStamp)
+      schemaEvolutionEntry.setRemoved(deletedColumnSchema.toList.asJava)
+      AlterTableUtil
+        .updateSchemaInfo(carbonTable,
+          schemaEvolutionEntry,
+          tableInfo)(sparkSession,
+          sparkSession.sessionState.asInstanceOf[CarbonSessionState])
+      // TODO: 1. add check for deletion of index tables
+      // delete dictionary files for dictionary column and clear dictionary cache from memory
+      new AlterTableDropColumnRDD(sparkSession.sparkContext,
+        dictionaryColumns,
+        carbonTable.getCarbonTableIdentifier,
+        carbonTable.getStorePath).collect()
+      LOGGER.info(s"Alter table for drop columns is successful for table $dbName.$tableName")
+      LOGGER.audit(s"Alter table for drop columns is successful for table $dbName.$tableName")
+    } catch {
+      case e: Exception => LOGGER
+        .error("Alter table drop columns failed : " + e.getMessage)
+        if (carbonTable != null) {
+          AlterTableUtil.revertDropColumnChanges(dbName, tableName, timeStamp)(sparkSession)
+        }
+        sys.error(s"Alter table drop column operation failed: ${e.getMessage}")
+    } finally {
+      // release lock after command execution completion
+      AlterTableUtil.releaseLocks(locks)
+    }
+    Seq.empty
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/87892522/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
new file mode 100644
index 0000000..88cf212
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala
@@ -0,0 +1,174 @@
+/*
+ * 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.schema
+
+import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.execution.command.{AlterTableRenameModel, RunnableCommand}
+import org.apache.spark.sql.hive.{CarbonRelation, CarbonSessionState}
+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.datastore.impl.FileFactory
+import org.apache.carbondata.core.locks.{ICarbonLock, LockUsage}
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, 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.format.SchemaEvolutionEntry
+import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
+
+private[sql] case class CarbonAlterTableRenameCommand(
+    alterTableRenameModel: AlterTableRenameModel)
+  extends RunnableCommand {
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+    val oldTableIdentifier = alterTableRenameModel.oldTableIdentifier
+    val newTableIdentifier = alterTableRenameModel.newTableIdentifier
+    val oldDatabaseName = oldTableIdentifier.database
+      .getOrElse(sparkSession.catalog.currentDatabase)
+    val newDatabaseName = newTableIdentifier.database
+      .getOrElse(sparkSession.catalog.currentDatabase)
+    if (!oldDatabaseName.equalsIgnoreCase(newDatabaseName)) {
+      throw new MalformedCarbonCommandException("Database name should be same for both tables")
+    }
+    val tableExists = sparkSession.catalog.tableExists(oldDatabaseName, newTableIdentifier.table)
+    if (tableExists) {
+      throw new MalformedCarbonCommandException(s"Table with name $newTableIdentifier " +
+                                                s"already exists")
+    }
+    val oldTableName = oldTableIdentifier.table.toLowerCase
+    val newTableName = newTableIdentifier.table.toLowerCase
+    LOGGER.audit(s"Rename table request has been received for $oldDatabaseName.$oldTableName")
+    LOGGER.info(s"Rename table request has been received for $oldDatabaseName.$oldTableName")
+    val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
+    val relation: CarbonRelation =
+      metastore.lookupRelation(oldTableIdentifier.database, oldTableName)(sparkSession)
+        .asInstanceOf[CarbonRelation]
+    if (relation == null) {
+      LOGGER.audit(s"Rename table request has failed. " +
+                   s"Table $oldDatabaseName.$oldTableName does not exist")
+      sys.error(s"Table $oldDatabaseName.$oldTableName does not exist")
+    }
+    val locksToBeAcquired = List(LockUsage.METADATA_LOCK,
+      LockUsage.COMPACTION_LOCK,
+      LockUsage.DELETE_SEGMENT_LOCK,
+      LockUsage.CLEAN_FILES_LOCK,
+      LockUsage.DROP_TABLE_LOCK)
+    var locks = List.empty[ICarbonLock]
+    var timeStamp = 0L
+    var carbonTable: CarbonTable = null
+    try {
+      locks = AlterTableUtil
+        .validateTableAndAcquireLock(oldDatabaseName, oldTableName, locksToBeAcquired)(
+          sparkSession)
+      val tableMeta = metastore.lookupRelation(Some(oldDatabaseName), oldTableName)(sparkSession)
+        .asInstanceOf[CarbonRelation].tableMeta
+      carbonTable = tableMeta.carbonTable
+      // get the latest carbon table and check for column existence
+      val carbonTablePath = CarbonStorePath.
+        getCarbonTablePath(AbsoluteTableIdentifier.fromTablePath(tableMeta.tablePath))
+      val tableMetadataFile = carbonTablePath.getPath
+      val tableInfo: org.apache.carbondata.format.TableInfo =
+        metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
+      val schemaEvolutionEntry = new SchemaEvolutionEntry(System.currentTimeMillis)
+      schemaEvolutionEntry.setTableName(newTableName)
+      timeStamp = System.currentTimeMillis()
+      schemaEvolutionEntry.setTime_stamp(timeStamp)
+      renameBadRecords(oldTableName, newTableName, oldDatabaseName)
+      val fileType = FileFactory.getFileType(tableMetadataFile)
+      if (FileFactory.isFileExist(tableMetadataFile, fileType)) {
+        val rename = FileFactory.getCarbonFile(carbonTablePath.getPath, fileType)
+          .renameForce(carbonTablePath.getParent.toString + CarbonCommonConstants.FILE_SEPARATOR +
+                       newTableName)
+        if (!rename) {
+          renameBadRecords(newTableName, oldTableName, oldDatabaseName)
+          sys.error(s"Folder rename failed for table $oldDatabaseName.$oldTableName")
+        }
+      }
+      val newTableIdentifier = new CarbonTableIdentifier(oldDatabaseName,
+        newTableName, carbonTable.getCarbonTableIdentifier.getTableId)
+      val newTablePath = metastore.updateTableSchema(newTableIdentifier,
+        carbonTable.getCarbonTableIdentifier,
+        tableInfo,
+        schemaEvolutionEntry,
+        tableMeta.tablePath)(sparkSession)
+      metastore.removeTableFromMetadata(oldDatabaseName, oldTableName)
+      sparkSession.sessionState.asInstanceOf[CarbonSessionState].metadataHive
+        .runSqlHive(
+          s"ALTER TABLE $oldDatabaseName.$oldTableName RENAME TO $oldDatabaseName.$newTableName")
+      sparkSession.sessionState.asInstanceOf[CarbonSessionState].metadataHive
+        .runSqlHive(
+          s"ALTER TABLE $oldDatabaseName.$newTableName SET SERDEPROPERTIES" +
+          s"('tableName'='$newTableName', " +
+          s"'dbName'='$oldDatabaseName', 'tablePath'='$newTablePath')")
+      sparkSession.catalog.refreshTable(TableIdentifier(newTableName,
+        Some(oldDatabaseName)).quotedString)
+      LOGGER.audit(s"Table $oldTableName has been successfully renamed to $newTableName")
+      LOGGER.info(s"Table $oldTableName has been successfully renamed to $newTableName")
+    } catch {
+      case e: Exception =>
+        LOGGER.error(e, "Rename table failed: " + e.getMessage)
+        if (carbonTable != null) {
+          AlterTableUtil
+            .revertRenameTableChanges(oldTableIdentifier,
+              newTableName,
+              carbonTable.getStorePath,
+              carbonTable.getCarbonTableIdentifier.getTableId,
+              timeStamp)(
+              sparkSession)
+          renameBadRecords(newTableName, oldTableName, oldDatabaseName)
+        }
+        sys.error(s"Alter table rename table operation failed: ${e.getMessage}")
+    } finally {
+      // release lock after command execution completion
+      AlterTableUtil.releaseLocks(locks)
+      // case specific to rename table as after table rename old table path will not be found
+      if (carbonTable != null) {
+        AlterTableUtil
+          .releaseLocksManually(locks,
+            locksToBeAcquired,
+            oldDatabaseName,
+            newTableName,
+            carbonTable.getStorePath)
+      }
+    }
+    Seq.empty
+  }
+
+  private def renameBadRecords(
+      oldTableName: String,
+      newTableName: String,
+      dataBaseName: String): Unit = {
+    val oldPath = CarbonUtil
+      .getBadLogPath(dataBaseName + CarbonCommonConstants.FILE_SEPARATOR + oldTableName)
+    val newPath = CarbonUtil
+      .getBadLogPath(dataBaseName + CarbonCommonConstants.FILE_SEPARATOR + newTableName)
+    val fileType = FileFactory.getFileType(oldPath)
+    if (FileFactory.isFileExist(oldPath, fileType)) {
+      val renameSuccess = FileFactory.getCarbonFile(oldPath, fileType)
+        .renameForce(newPath)
+      if (!renameSuccess) {
+        sys.error(s"BadRecords Folder Rename Failed for table $dataBaseName.$oldTableName")
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/87892522/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala
index bdfaa5a..bf13e41 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala
@@ -24,7 +24,7 @@ import org.apache.spark.sql.execution.{SparkPlan, SparkStrategy}
 import org.apache.spark.sql.execution.command._
 import org.apache.spark.sql.execution.command.management.{AlterTableCompactionCommand, CarbonShowLoadsCommand, LoadTableByInsertCommand, LoadTableCommand}
 import org.apache.spark.sql.execution.command.partition.ShowCarbonPartitionsCommand
-import org.apache.spark.sql.execution.command.schema.{AlterTableAddColumnCommand, AlterTableDataTypeChangeCommand, AlterTableDropColumnCommand, AlterTableRenameTableCommand}
+import org.apache.spark.sql.execution.command.schema.{CarbonAlterTableAddColumnCommand, CarbonAlterTableDataTypeChangeCommand, CarbonAlterTableDropColumnCommand, CarbonAlterTableRenameCommand}
 import org.apache.spark.sql.hive.execution.command.{CarbonDropDatabaseCommand, CarbonResetCommand, CarbonSetCommand}
 
 import org.apache.carbondata.core.util.CarbonUtil
@@ -56,7 +56,7 @@ class DDLStrategy(sparkSession: SparkSession) extends SparkStrategy {
             sparkSession)
         if (isCarbonTable) {
           val renameModel = AlterTableRenameModel(tableIdentifier, newTableIdentifier)
-          ExecutedCommandExec(AlterTableRenameTableCommand(renameModel)) :: Nil
+          ExecutedCommandExec(CarbonAlterTableRenameCommand(renameModel)) :: Nil
         } else {
           ExecutedCommandExec(alter) :: Nil
         }
@@ -98,7 +98,7 @@ class DDLStrategy(sparkSession: SparkSession) extends SparkStrategy {
           throw new MalformedCarbonCommandException(
             "Operation not allowed : " + altertablemodel.alterSql)
         }
-      case dataTypeChange@AlterTableDataTypeChangeCommand(alterTableChangeDataTypeModel) =>
+      case dataTypeChange@CarbonAlterTableDataTypeChangeCommand(alterTableChangeDataTypeModel) =>
         val isCarbonTable = CarbonEnv.getInstance(sparkSession).carbonMetastore
           .tableExists(TableIdentifier(alterTableChangeDataTypeModel.tableName,
             alterTableChangeDataTypeModel.databaseName))(sparkSession)
@@ -107,7 +107,7 @@ class DDLStrategy(sparkSession: SparkSession) extends SparkStrategy {
         } else {
           throw new MalformedCarbonCommandException("Unsupported alter operation on hive table")
         }
-      case addColumn@AlterTableAddColumnCommand(alterTableAddColumnsModel) =>
+      case addColumn@CarbonAlterTableAddColumnCommand(alterTableAddColumnsModel) =>
         val isCarbonTable = CarbonEnv.getInstance(sparkSession).carbonMetastore
           .tableExists(TableIdentifier(alterTableAddColumnsModel.tableName,
             alterTableAddColumnsModel.databaseName))(sparkSession)
@@ -116,7 +116,7 @@ class DDLStrategy(sparkSession: SparkSession) extends SparkStrategy {
         } else {
           throw new MalformedCarbonCommandException("Unsupported alter operation on hive table")
         }
-      case dropColumn@AlterTableDropColumnCommand(alterTableDropColumnModel) =>
+      case dropColumn@CarbonAlterTableDropColumnCommand(alterTableDropColumnModel) =>
         val isCarbonTable = CarbonEnv.getInstance(sparkSession).carbonMetastore
           .tableExists(TableIdentifier(alterTableDropColumnModel.tableName,
             alterTableDropColumnModel.databaseName))(sparkSession)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/87892522/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/StreamingTableStrategy.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/StreamingTableStrategy.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/StreamingTableStrategy.scala
index 0f0bc24..9ebf47e 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/StreamingTableStrategy.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/StreamingTableStrategy.scala
@@ -21,8 +21,9 @@ import org.apache.spark.sql.{CarbonEnv, SparkSession}
 import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
 import org.apache.spark.sql.execution.{SparkPlan, SparkStrategy}
-import org.apache.spark.sql.execution.command.ExecutedCommandExec
+import org.apache.spark.sql.execution.command.{AlterTableRenameCommand, ExecutedCommandExec}
 import org.apache.spark.sql.execution.command.mutation.{DeleteExecution, ProjectForDeleteCommand, ProjectForUpdateCommand}
+import org.apache.spark.sql.execution.command.schema.{CarbonAlterTableAddColumnCommand, CarbonAlterTableDataTypeChangeCommand, CarbonAlterTableDropColumnCommand}
 import org.apache.spark.sql.hive.CarbonRelation
 
 import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
@@ -34,12 +35,36 @@ private[sql] class StreamingTableStrategy(sparkSession: SparkSession) extends Sp
 
   override def apply(plan: LogicalPlan): Seq[SparkPlan] = {
     plan match {
-      case update@ProjectForUpdateCommand(_, tableIdentifier) =>
-        rejectIfStreamingTable(DeleteExecution.getTableIdentifier(tableIdentifier), "Data update")
-        ExecutedCommandExec(update) :: Nil
-      case delete@ProjectForDeleteCommand(_, tableIdentifier, _) =>
-        rejectIfStreamingTable(DeleteExecution.getTableIdentifier(tableIdentifier), "Date delete")
-        ExecutedCommandExec(delete) :: Nil
+      case ProjectForUpdateCommand(_, tableIdentifier) =>
+        rejectIfStreamingTable(
+          DeleteExecution.getTableIdentifier(tableIdentifier),
+          "Data update")
+        Nil
+      case ProjectForDeleteCommand(_, tableIdentifier, _) =>
+        rejectIfStreamingTable(
+          DeleteExecution.getTableIdentifier(tableIdentifier),
+          "Date delete")
+        Nil
+      case CarbonAlterTableAddColumnCommand(model) =>
+        rejectIfStreamingTable(
+          new TableIdentifier(model.tableName, model.databaseName),
+          "Alter table add column")
+        Nil
+      case CarbonAlterTableDropColumnCommand(model) =>
+        rejectIfStreamingTable(
+          new TableIdentifier(model.tableName, model.databaseName),
+          "Alter table drop column")
+        Nil
+      case CarbonAlterTableDataTypeChangeCommand(model) =>
+        rejectIfStreamingTable(
+          new TableIdentifier(model.tableName, model.databaseName),
+          "Alter table change datatype")
+        Nil
+      case AlterTableRenameCommand(oldTableIdentifier, _, _) =>
+        rejectIfStreamingTable(
+          oldTableIdentifier,
+          "Alter rename table")
+        Nil
       case _ => Nil
     }
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/87892522/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 9c87b8b..fc2ed41 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
@@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.execution.command._
 import org.apache.spark.sql.execution.command.management.{AlterTableCompactionCommand, CleanFilesCommand, DeleteLoadByIdCommand, DeleteLoadByLoadDateCommand, LoadTableCommand}
 import org.apache.spark.sql.execution.command.partition.{AlterTableDropCarbonPartitionCommand, AlterTableSplitCarbonPartitionCommand}
-import org.apache.spark.sql.execution.command.schema.{AlterTableAddColumnCommand, AlterTableDataTypeChangeCommand, AlterTableDropColumnCommand}
+import org.apache.spark.sql.execution.command.schema.{CarbonAlterTableAddColumnCommand, CarbonAlterTableDataTypeChangeCommand, CarbonAlterTableDropColumnCommand}
 import org.apache.spark.sql.types.StructField
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
@@ -326,7 +326,7 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
             table.toLowerCase,
             columnName.toLowerCase,
             columnNameCopy.toLowerCase)
-        AlterTableDataTypeChangeCommand(alterTableChangeDataTypeModel)
+        CarbonAlterTableDataTypeChangeCommand(alterTableChangeDataTypeModel)
     }
 
   protected lazy val alterTableAddColumns: Parser[LogicalPlan] =
@@ -395,7 +395,7 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
           tableModel.dimCols,
           tableModel.msrCols,
           tableModel.highcardinalitydims.getOrElse(Seq.empty))
-        AlterTableAddColumnCommand(alterTableAddColumnsModel)
+        CarbonAlterTableAddColumnCommand(alterTableAddColumnsModel)
     }
 
   private def checkFieldDefaultValue(fieldName: String, defaultValueColumnName: String): Boolean = {
@@ -419,7 +419,7 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
         val alterTableDropColumnModel = AlterTableDropColumnModel(convertDbNameToLowerCase(dbName),
           table.toLowerCase,
           values.map(_.toLowerCase))
-        AlterTableDropColumnCommand(alterTableDropColumnModel)
+        CarbonAlterTableDropColumnCommand(alterTableDropColumnModel)
     }
 
   def getFields(schema: Seq[StructField]): Seq[Field] = {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/87892522/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala
index b733d4f..d5f9426 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala
@@ -80,6 +80,21 @@ class TestStreamingTableOperation extends QueryTest with BeforeAndAfterAll {
     }
   }
 
+  test("test blocking alter table operation on streaming table") {
+    intercept[MalformedCarbonCommandException] {
+      sql("""ALTER TABLE source ADD COLUMNS (c6 string)""").show()
+    }
+    intercept[MalformedCarbonCommandException] {
+      sql("""ALTER TABLE source DROP COLUMNS (c1)""").show()
+    }
+    intercept[MalformedCarbonCommandException] {
+      sql("""ALTER TABLE source RENAME to t""").show()
+    }
+    intercept[MalformedCarbonCommandException] {
+      sql("""ALTER TABLE source CHANGE c1 c1 int""").show()
+    }
+  }
+
   override def afterAll {
     sql("USE default")
     sql("DROP DATABASE IF EXISTS streaming CASCADE")

http://git-wip-us.apache.org/repos/asf/carbondata/blob/87892522/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableRevertTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableRevertTestCase.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableRevertTestCase.scala
index 29de05b..00170e2 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableRevertTestCase.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableRevertTestCase.scala
@@ -51,7 +51,7 @@ class AlterTableRevertTestCase extends Spark2QueryTest with BeforeAndAfterAll {
   }
 
   test("test to revert table name on failure") {
-    intercept[RuntimeException] {
+    val exception = intercept[RuntimeException] {
       new File(TestQueryExecutor.warehouse + "/reverttest_fail").mkdir()
       sql("alter table reverttest rename to reverttest_fail")
       new File(TestQueryExecutor.warehouse + "/reverttest_fail").delete()


[02/49] carbondata git commit: [CARBONDATA-1658] Fixed Thread leak issue in no sort

Posted by ra...@apache.org.
[CARBONDATA-1658] Fixed Thread leak issue in no sort

Problem: In case of no sort executor service is not shutting down in writer step which is causing thread leak. In case of long run it will throwing OOM error
Solution:: Need to shutdown executor service in all the case success and failure

This closes #1454


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

Branch: refs/heads/fgdatamap
Commit: b49160935a7c3c8fe1899e3e1c49ba7022cb0938
Parents: 9955bed
Author: kumarvishal <ku...@gmail.com>
Authored: Mon Oct 30 20:52:19 2017 +0530
Committer: Jacky Li <ja...@qq.com>
Committed: Tue Oct 31 23:27:46 2017 +0530

----------------------------------------------------------------------
 .../src/test/resources/measureinsertintotest.csv     |  7 +++++++
 .../testsuite/sortcolumns/TestSortColumns.scala      | 15 +++++++++++++++
 .../steps/CarbonRowDataWriterProcessorStepImpl.java  | 10 +++++++++-
 3 files changed, 31 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/b4916093/integration/spark-common-test/src/test/resources/measureinsertintotest.csv
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/resources/measureinsertintotest.csv b/integration/spark-common-test/src/test/resources/measureinsertintotest.csv
new file mode 100644
index 0000000..06985e8
--- /dev/null
+++ b/integration/spark-common-test/src/test/resources/measureinsertintotest.csv
@@ -0,0 +1,7 @@
+id,name,city,age
+1,david,shenzhen,31
+2,eason,shenzhen,27
+3,jarry,wuhan,35
+3,jarry,Bangalore,35
+4,kunal,Delhi,26
+4,vishal,Bangalore,29

http://git-wip-us.apache.org/repos/asf/carbondata/blob/b4916093/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumns.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumns.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumns.scala
index b655025..b5fd8a9 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumns.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumns.scala
@@ -16,7 +16,9 @@
  */
 package org.apache.carbondata.spark.testsuite.sortcolumns
 
+import org.apache.spark.sql.Row
 import org.scalatest.BeforeAndAfterAll
+
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.util.CarbonProperties
 import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
@@ -29,6 +31,11 @@ class TestSortColumns extends QueryTest with BeforeAndAfterAll {
 
     sql("CREATE TABLE origintable1 (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 origintable1 OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""")
+
+    sql("CREATE TABLE tableOne(id int, name string, city string, age int) STORED BY 'org.apache.carbondata.format'")
+    sql("CREATE TABLE tableTwo(id int, age int) STORED BY 'org.apache.carbondata.format'")
+    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/measureinsertintotest.csv' into table tableOne")
+    sql("insert into table tableTwo select id, count(age) from tableOne group by id")
   }
 
   test("create table sort columns dictionary include - int") {
@@ -327,6 +334,12 @@ class TestSortColumns extends QueryTest with BeforeAndAfterAll {
   assert(exceptionCaught.getMessage.equals("SORT_COLUMNS Either having duplicate columns : empno or it contains illegal argumnet."))
   }
 
+  test("Test tableTwo data") {
+    checkAnswer(
+      sql("select id,age from tableTwo order by id"),
+      Seq(Row(1, 1), Row(2, 1), Row(3, 2), Row(4, 2)))
+  }
+
   test("Measure columns in sort_columns") {
     val exceptionCaught = intercept[MalformedCarbonCommandException] {
       sql(
@@ -378,6 +391,8 @@ class TestSortColumns extends QueryTest with BeforeAndAfterAll {
     sql("drop table if exists test_sort_col")
     sql("drop table if exists test_sort_col_hive")
     sql("drop table if exists sorttable1b")
+    sql("DROP TABLE IF EXISTS tableOne")
+    sql("DROP TABLE IF EXISTS tableTwo")
   }
 
   def setLoadingProperties(offheap: String, unsafe: String, useBatch: String): Unit = {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/b4916093/processing/src/main/java/org/apache/carbondata/processing/loading/steps/CarbonRowDataWriterProcessorStepImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/steps/CarbonRowDataWriterProcessorStepImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/steps/CarbonRowDataWriterProcessorStepImpl.java
index 7007160..34b4f3d 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/steps/CarbonRowDataWriterProcessorStepImpl.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/steps/CarbonRowDataWriterProcessorStepImpl.java
@@ -30,6 +30,7 @@ import org.apache.carbondata.core.datastore.row.WriteStepRowUtil;
 import org.apache.carbondata.core.keygenerator.KeyGenException;
 import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.util.CarbonThreadFactory;
 import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory;
 import org.apache.carbondata.processing.loading.AbstractDataLoadProcessorStep;
 import org.apache.carbondata.processing.loading.CarbonDataLoadConfiguration;
@@ -100,6 +101,7 @@ public class CarbonRowDataWriterProcessorStepImpl extends AbstractDataLoadProces
     final Iterator<CarbonRowBatch>[] iterators = child.execute();
     tableIdentifier = configuration.getTableIdentifier().getCarbonTableIdentifier();
     tableName = tableIdentifier.getTableName();
+    ExecutorService executorService = null;
     try {
       readCounter = new long[iterators.length];
       writeCounter = new long[iterators.length];
@@ -122,7 +124,9 @@ public class CarbonRowDataWriterProcessorStepImpl extends AbstractDataLoadProces
       if (iterators.length == 1) {
         doExecute(iterators[0], 0, 0);
       } else {
-        ExecutorService executorService = Executors.newFixedThreadPool(iterators.length);
+        executorService = Executors.newFixedThreadPool(iterators.length,
+            new CarbonThreadFactory("NoSortDataWriterPool:" + configuration.getTableIdentifier()
+                .getCarbonTableIdentifier().getTableName()));
         Future[] futures = new Future[iterators.length];
         for (int i = 0; i < iterators.length; i++) {
           futures[i] = executorService.submit(new DataWriterRunnable(iterators[i], i));
@@ -141,6 +145,10 @@ public class CarbonRowDataWriterProcessorStepImpl extends AbstractDataLoadProces
         throw new BadRecordFoundException(e.getMessage(), e);
       }
       throw new CarbonDataLoadingException("There is an unexpected error: " + e.getMessage(), e);
+    } finally {
+      if (null != executorService && executorService.isShutdown()) {
+        executorService.shutdownNow();
+      }
     }
     return null;
   }


[31/49] carbondata git commit: [CARBONDATA-1690][DATAMAP] Clear datamap when renaming table

Posted by ra...@apache.org.
[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/fgdatamap
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')")


[23/49] carbondata git commit: [CARBONDATA-1572][Streaming] Support streaming ingest and query

Posted by ra...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSource.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSource.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSource.scala
index f4f8b75..4580f22 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSource.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSource.scala
@@ -26,27 +26,30 @@ import org.apache.spark.sql.catalyst.analysis.NoSuchTableException
 import org.apache.spark.sql.catalyst.catalog.CatalogTable
 import org.apache.spark.sql.execution.command.{TableModel, TableNewProcessor}
 import org.apache.spark.sql.execution.strategy.CarbonLateDecodeStrategy
+import org.apache.spark.sql.execution.streaming.Sink
 import org.apache.spark.sql.hive.{CarbonMetaStore, CarbonRelation}
 import org.apache.spark.sql.optimizer.CarbonLateDecodeRule
 import org.apache.spark.sql.parser.CarbonSpark2SqlParser
 import org.apache.spark.sql.sources._
+import org.apache.spark.sql.streaming.OutputMode
 import org.apache.spark.sql.types.StructType
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
 import org.apache.carbondata.core.metadata.schema.SchemaEvolutionEntry
 import org.apache.carbondata.core.metadata.schema.table.TableInfo
-import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
+import org.apache.carbondata.core.util.{CarbonProperties, CarbonSessionInfo, CarbonUtil, ThreadLocalSessionInfo}
 import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
 import org.apache.carbondata.spark.CarbonOption
 import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
+import org.apache.carbondata.streaming.{CarbonStreamException, StreamSinkFactory}
 
 /**
  * Carbon relation provider compliant to data source api.
  * Creates carbon relations
  */
 class CarbonSource extends CreatableRelationProvider with RelationProvider
-  with SchemaRelationProvider with DataSourceRegister {
+  with SchemaRelationProvider with StreamSinkProvider with DataSourceRegister {
 
   override def shortName(): String = "carbondata"
 
@@ -208,6 +211,35 @@ class CarbonSource extends CreatableRelationProvider with RelationProvider
     }
   }
 
+  /**
+   * produce a streaming `Sink` for a specific format
+   * now it will create a default sink(CarbonAppendableStreamSink) for row format
+   */
+  override def createSink(sqlContext: SQLContext,
+      parameters: Map[String, String],
+      partitionColumns: Seq[String],
+      outputMode: OutputMode): Sink = {
+
+    // check "tablePath" option
+    val tablePathOption = parameters.get("tablePath")
+    if (tablePathOption.isDefined) {
+      val sparkSession = sqlContext.sparkSession
+      val identifier: AbsoluteTableIdentifier =
+        AbsoluteTableIdentifier.fromTablePath(tablePathOption.get)
+      val carbonTable =
+        CarbonEnv.getInstance(sparkSession).carbonMetastore.
+          createCarbonRelation(parameters, identifier, sparkSession).tableMeta.carbonTable
+
+      // create sink
+      StreamSinkFactory.createStreamTableSink(
+        sqlContext.sparkSession,
+        carbonTable,
+        parameters)
+    } else {
+      throw new CarbonStreamException("Require tablePath option for the write stream")
+    }
+  }
+
 }
 
 object CarbonSource {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 39dc565..8b1212d 100644
--- a/pom.xml
+++ b/pom.xml
@@ -107,7 +107,7 @@
   <properties>
     <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
     <snappy.version>1.1.2.6</snappy.version>
-    <hadoop.version>2.2.0</hadoop.version>
+    <hadoop.version>2.7.2</hadoop.version>
     <hadoop.deps.scope>compile</hadoop.deps.scope>
     <spark.deps.scope>compile</spark.deps.scope>
     <scala.deps.scope>compile</scala.deps.scope>
@@ -438,6 +438,7 @@
         <module>integration/hive</module>
         <module>integration/presto</module>
         <module>examples/flink</module>
+        <module>streaming</module>
       </modules>
       <build>
         <plugins>
@@ -452,13 +453,6 @@
       </build>
     </profile>
     <profile>
-      <id>hadoop-2.2.0</id>
-      <!-- default -->
-      <properties>
-        <hadoop.version>2.2.0</hadoop.version>
-      </properties>
-    </profile>
-    <profile>
       <id>hadoop-2.7.2</id>
       <properties>
         <hadoop.version>2.7.2</hadoop.version>
@@ -478,6 +472,7 @@
         <module>integration/spark2</module>
         <module>integration/hive</module>
         <module>integration/presto</module>
+        <module>streaming</module>
         <module>examples/spark2</module>
       </modules>
       <build>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/processing/src/main/java/org/apache/carbondata/processing/loading/csvinput/CSVInputFormat.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/csvinput/CSVInputFormat.java b/processing/src/main/java/org/apache/carbondata/processing/loading/csvinput/CSVInputFormat.java
index 61646ec..9a37421 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/csvinput/CSVInputFormat.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/csvinput/CSVInputFormat.java
@@ -64,7 +64,7 @@ public class CSVInputFormat extends FileInputFormat<NullWritable, StringArrayWri
   public static final String QUOTE_DEFAULT = "\"";
   public static final String ESCAPE = "carbon.csvinputformat.escape";
   public static final String ESCAPE_DEFAULT = "\\";
-  public static final String HEADER_PRESENT = "caron.csvinputformat.header.present";
+  public static final String HEADER_PRESENT = "carbon.csvinputformat.header.present";
   public static final boolean HEADER_PRESENT_DEFAULT = false;
   public static final String READ_BUFFER_SIZE = "carbon.csvinputformat.read.buffer.size";
   public static final String READ_BUFFER_SIZE_DEFAULT = "65536";
@@ -171,6 +171,24 @@ public class CSVInputFormat extends FileInputFormat<NullWritable, StringArrayWri
     configuration.set(NUMBER_OF_COLUMNS, numberOfColumns);
   }
 
+  public static CsvParserSettings extractCsvParserSettings(Configuration job) {
+    CsvParserSettings parserSettings = new CsvParserSettings();
+    parserSettings.getFormat().setDelimiter(job.get(DELIMITER, DELIMITER_DEFAULT).charAt(0));
+    parserSettings.getFormat().setComment(job.get(COMMENT, COMMENT_DEFAULT).charAt(0));
+    parserSettings.setLineSeparatorDetectionEnabled(true);
+    parserSettings.setNullValue("");
+    parserSettings.setEmptyValue("");
+    parserSettings.setIgnoreLeadingWhitespaces(false);
+    parserSettings.setIgnoreTrailingWhitespaces(false);
+    parserSettings.setSkipEmptyLines(false);
+    parserSettings.setMaxCharsPerColumn(MAX_CHARS_PER_COLUMN_DEFAULT);
+    String maxColumns = job.get(MAX_COLUMNS, "" + DEFAULT_MAX_NUMBER_OF_COLUMNS_FOR_PARSING);
+    parserSettings.setMaxColumns(Integer.parseInt(maxColumns));
+    parserSettings.getFormat().setQuote(job.get(QUOTE, QUOTE_DEFAULT).charAt(0));
+    parserSettings.getFormat().setQuoteEscape(job.get(ESCAPE, ESCAPE_DEFAULT).charAt(0));
+    return parserSettings;
+  }
+
   /**
    * Treats value as line in file. Key is null.
    */
@@ -232,30 +250,13 @@ public class CSVInputFormat extends FileInputFormat<NullWritable, StringArrayWri
       }
       reader = new InputStreamReader(inputStream,
           Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
-      csvParser = new CsvParser(extractCsvParserSettings(job));
-      csvParser.beginParsing(reader);
-    }
-
-    private CsvParserSettings extractCsvParserSettings(Configuration job) {
-      CsvParserSettings parserSettings = new CsvParserSettings();
-      parserSettings.getFormat().setDelimiter(job.get(DELIMITER, DELIMITER_DEFAULT).charAt(0));
-      parserSettings.getFormat().setComment(job.get(COMMENT, COMMENT_DEFAULT).charAt(0));
-      parserSettings.setLineSeparatorDetectionEnabled(true);
-      parserSettings.setNullValue("");
-      parserSettings.setEmptyValue("");
-      parserSettings.setIgnoreLeadingWhitespaces(false);
-      parserSettings.setIgnoreTrailingWhitespaces(false);
-      parserSettings.setSkipEmptyLines(false);
-      parserSettings.setMaxCharsPerColumn(MAX_CHARS_PER_COLUMN_DEFAULT);
-      String maxColumns = job.get(MAX_COLUMNS);
-      parserSettings.setMaxColumns(Integer.parseInt(maxColumns));
-      parserSettings.getFormat().setQuote(job.get(QUOTE, QUOTE_DEFAULT).charAt(0));
-      parserSettings.getFormat().setQuoteEscape(job.get(ESCAPE, ESCAPE_DEFAULT).charAt(0));
+      CsvParserSettings settings = extractCsvParserSettings(job);
       if (start == 0) {
-        parserSettings.setHeaderExtractionEnabled(job.getBoolean(HEADER_PRESENT,
+        settings.setHeaderExtractionEnabled(job.getBoolean(HEADER_PRESENT,
             HEADER_PRESENT_DEFAULT));
       }
-      return parserSettings;
+      csvParser = new CsvParser(settings);
+      csvParser.beginParsing(reader);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/processing/src/main/java/org/apache/carbondata/processing/store/writer/AbstractFactDataWriter.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/writer/AbstractFactDataWriter.java b/processing/src/main/java/org/apache/carbondata/processing/store/writer/AbstractFactDataWriter.java
index 972e414..1b6ba72 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/store/writer/AbstractFactDataWriter.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/store/writer/AbstractFactDataWriter.java
@@ -392,7 +392,7 @@ public abstract class AbstractFactDataWriter implements CarbonFactDataWriter {
   protected abstract void fillBlockIndexInfoDetails(long numberOfRows, String carbonDataFileName,
       long currentPosition);
 
-  protected List<org.apache.carbondata.format.ColumnSchema> getColumnSchemaListAndCardinality(
+  public static List<org.apache.carbondata.format.ColumnSchema> getColumnSchemaListAndCardinality(
       List<Integer> cardinality, int[] dictionaryColumnCardinality,
       List<ColumnSchema> wrapperColumnSchemaList) {
     List<org.apache.carbondata.format.ColumnSchema> columnSchemaList =

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/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 1c7f9e7..761867c 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
@@ -42,7 +42,9 @@ import org.apache.carbondata.core.metadata.CarbonMetadata;
 import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 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.CarbonColumn;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
 import org.apache.carbondata.core.util.CarbonProperties;
@@ -259,6 +261,23 @@ public final class CarbonDataProcessorUtil {
         .toPrimitive(noDictionaryMapping.toArray(new Boolean[noDictionaryMapping.size()]));
   }
 
+  public static boolean[] getNoDictionaryMapping(CarbonColumn[] carbonColumns) {
+    List<Boolean> noDictionaryMapping = new ArrayList<Boolean>();
+    for (CarbonColumn column : carbonColumns) {
+      // for  complex type need to break the loop
+      if (column.isComplex()) {
+        break;
+      }
+      if (!column.hasEncoding(Encoding.DICTIONARY) && column.isDimension()) {
+        noDictionaryMapping.add(true);
+      } else if (column.isDimension()) {
+        noDictionaryMapping.add(false);
+      }
+    }
+    return ArrayUtils
+        .toPrimitive(noDictionaryMapping.toArray(new Boolean[noDictionaryMapping.size()]));
+  }
+
   /**
    * Preparing the boolean [] to map whether the dimension use inverted index or not.
    */

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
index 0b88684..3cf851f 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
@@ -186,7 +186,7 @@ public final class CarbonLoaderUtil {
     }
   }
 
-  private static void deleteStorePath(String path) {
+  public static void deleteStorePath(String path) {
     try {
       FileType fileType = FileFactory.getFileType(path);
       if (FileFactory.isFileExist(path, fileType)) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/streaming/pom.xml
----------------------------------------------------------------------
diff --git a/streaming/pom.xml b/streaming/pom.xml
new file mode 100644
index 0000000..d9dac75
--- /dev/null
+++ b/streaming/pom.xml
@@ -0,0 +1,127 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <parent>
+    <artifactId>carbondata-parent</artifactId>
+    <groupId>org.apache.carbondata</groupId>
+    <version>1.3.0-SNAPSHOT</version>
+    <relativePath>../pom.xml</relativePath>
+  </parent>
+  <modelVersion>4.0.0</modelVersion>
+
+  <artifactId>carbondata-streaming</artifactId>
+  <packaging>jar</packaging>
+
+  <name>Apache CarbonData :: Streaming</name>
+  <url>http://maven.apache.org</url>
+
+  <properties>
+    <dev.path>${basedir}/../dev</dev.path>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.carbondata</groupId>
+      <artifactId>carbondata-spark-common</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <version>3.8.1</version>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+
+  <build>
+    <testSourceDirectory>src/test/scala</testSourceDirectory>
+    <resources>
+      <resource>
+        <directory>src/resources</directory>
+      </resource>
+      <resource>
+        <directory>.</directory>
+        <includes>
+          <include>CARBON_STREAMING_INTERFACELogResource.properties</include>
+        </includes>
+      </resource>
+    </resources>
+    <plugins>
+      <plugin>
+        <groupId>org.scala-tools</groupId>
+        <artifactId>maven-scala-plugin</artifactId>
+        <version>2.15.2</version>
+        <executions>
+          <execution>
+            <id>compile</id>
+            <goals>
+              <goal>compile</goal>
+            </goals>
+            <phase>compile</phase>
+          </execution>
+          <execution>
+            <id>testCompile</id>
+            <goals>
+              <goal>testCompile</goal>
+            </goals>
+            <phase>test</phase>
+          </execution>
+          <execution>
+            <phase>process-resources</phase>
+            <goals>
+              <goal>compile</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <configuration>
+          <source>1.7</source>
+          <target>1.7</target>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-plugin</artifactId>
+        <version>2.18</version>
+        <!-- Note config is repeated in scalatest config -->
+        <configuration>
+          <reportsDirectory>${project.build.directory}/surefire-reports</reportsDirectory>
+          <argLine>-Xmx3g -XX:MaxPermSize=512m -XX:ReservedCodeCacheSize=512m</argLine>
+          <systemProperties>
+            <java.awt.headless>true</java.awt.headless>
+          </systemProperties>
+          <failIfNoTests>false</failIfNoTests>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.scalatest</groupId>
+        <artifactId>scalatest-maven-plugin</artifactId>
+        <version>1.0</version>
+        <!-- Note config is repeated in surefire config -->
+        <configuration>
+          <reportsDirectory>${project.build.directory}/surefire-reports</reportsDirectory>
+          <junitxml>.</junitxml>
+          <filereports>CarbonTestSuite.txt</filereports>
+          <argLine> ${argLine} -ea -Xmx3g -XX:MaxPermSize=512m -XX:ReservedCodeCacheSize=512m
+          </argLine>
+          <stderr />
+          <environmentVariables>
+          </environmentVariables>
+          <systemProperties>
+            <java.awt.headless>true</java.awt.headless>
+          </systemProperties>
+        </configuration>
+        <executions>
+          <execution>
+            <id>test</id>
+            <goals>
+              <goal>test</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+</project>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/streaming/src/main/java/org/apache/carbondata/streaming/CarbonStreamException.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/carbondata/streaming/CarbonStreamException.java b/streaming/src/main/java/org/apache/carbondata/streaming/CarbonStreamException.java
new file mode 100644
index 0000000..602cef7
--- /dev/null
+++ b/streaming/src/main/java/org/apache/carbondata/streaming/CarbonStreamException.java
@@ -0,0 +1,32 @@
+/*
+ * 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.streaming;
+
+/**
+ * Stream exception
+ */
+public class CarbonStreamException extends Exception {
+
+  public CarbonStreamException(String message) {
+    super(message);
+  }
+
+  public CarbonStreamException(String message, Throwable cause) {
+    super(message, cause);
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/streaming/src/main/java/org/apache/carbondata/streaming/parser/CSVStreamParserImp.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/carbondata/streaming/parser/CSVStreamParserImp.java b/streaming/src/main/java/org/apache/carbondata/streaming/parser/CSVStreamParserImp.java
new file mode 100644
index 0000000..eed3fd5
--- /dev/null
+++ b/streaming/src/main/java/org/apache/carbondata/streaming/parser/CSVStreamParserImp.java
@@ -0,0 +1,45 @@
+/*
+ * 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.streaming.parser;
+
+import org.apache.carbondata.processing.loading.csvinput.CSVInputFormat;
+
+import com.univocity.parsers.csv.CsvParser;
+import com.univocity.parsers.csv.CsvParserSettings;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.spark.sql.catalyst.InternalRow;
+
+/**
+ * CSV Stream Parser, it is also the default parser.
+ */
+public class CSVStreamParserImp implements CarbonStreamParser {
+
+  private CsvParser csvParser;
+
+  @Override public void initialize(Configuration configuration) {
+    CsvParserSettings settings = CSVInputFormat.extractCsvParserSettings(configuration);
+    csvParser = new CsvParser(settings);
+  }
+
+  @Override public Object[] parserRow(InternalRow row) {
+    return csvParser.parseLine(row.getString(0));
+  }
+
+  @Override public void close() {
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/streaming/src/main/java/org/apache/carbondata/streaming/parser/CarbonStreamParser.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/carbondata/streaming/parser/CarbonStreamParser.java b/streaming/src/main/java/org/apache/carbondata/streaming/parser/CarbonStreamParser.java
new file mode 100644
index 0000000..a3b5592
--- /dev/null
+++ b/streaming/src/main/java/org/apache/carbondata/streaming/parser/CarbonStreamParser.java
@@ -0,0 +1,38 @@
+/*
+ * 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.streaming.parser;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.spark.sql.catalyst.InternalRow;
+
+/**
+ * Stream parser interface
+ */
+public interface CarbonStreamParser {
+
+  String CARBON_STREAM_PARSER = "carbon.stream.parser";
+
+  String CARBON_STREAM_PARSER_DEFAULT = "org.apache.carbondata.streaming.parser.CSVStreamParserImp";
+
+  void initialize(Configuration configuration);
+
+  Object[] parserRow(InternalRow value);
+
+  void close();
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/streaming/src/main/java/org/apache/carbondata/streaming/segment/StreamSegment.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/carbondata/streaming/segment/StreamSegment.java b/streaming/src/main/java/org/apache/carbondata/streaming/segment/StreamSegment.java
new file mode 100644
index 0000000..32ba332
--- /dev/null
+++ b/streaming/src/main/java/org/apache/carbondata/streaming/segment/StreamSegment.java
@@ -0,0 +1,373 @@
+/*
+ * 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.streaming.segment;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.carbondata.common.CarbonIterator;
+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.datastore.filesystem.CarbonFile;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFileFilter;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.locks.ICarbonLock;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+import org.apache.carbondata.core.reader.CarbonIndexFileReader;
+import org.apache.carbondata.core.statusmanager.FileFormat;
+import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
+import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
+import org.apache.carbondata.core.util.path.CarbonStorePath;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
+import org.apache.carbondata.core.writer.CarbonIndexFileWriter;
+import org.apache.carbondata.format.BlockIndex;
+import org.apache.carbondata.format.BlockletIndex;
+import org.apache.carbondata.hadoop.streaming.CarbonStreamOutputFormat;
+import org.apache.carbondata.hadoop.streaming.CarbonStreamRecordWriter;
+
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+/**
+ * streaming segment manager
+ */
+public class StreamSegment {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(StreamSegment.class.getName());
+
+  public static final long STREAM_SEGMENT_MAX_SIZE = 1024L * 1024 * 1024;
+
+  /**
+   * get stream segment or create new stream segment if not exists
+   */
+  public static String open(CarbonTable table) throws IOException {
+    CarbonTablePath tablePath =
+        CarbonStorePath.getCarbonTablePath(table.getAbsoluteTableIdentifier());
+    SegmentStatusManager segmentStatusManager =
+        new SegmentStatusManager(table.getAbsoluteTableIdentifier());
+    ICarbonLock carbonLock = segmentStatusManager.getTableStatusLock();
+    try {
+      if (carbonLock.lockWithRetries()) {
+        LOGGER.info(
+            "Acquired lock for table" + table.getDatabaseName() + "." + table.getFactTableName()
+                + " for stream table get or create segment");
+
+        LoadMetadataDetails[] details =
+            SegmentStatusManager.readLoadMetadata(tablePath.getMetadataDirectoryPath());
+        LoadMetadataDetails streamSegment = null;
+        for (LoadMetadataDetails detail : details) {
+          if (FileFormat.rowformat == detail.getFileFormat()) {
+            if (CarbonCommonConstants.STORE_LOADSTATUS_STREAMING.equals(detail.getLoadStatus())) {
+              streamSegment = detail;
+              break;
+            }
+          }
+        }
+        if (null == streamSegment) {
+          int segmentId = SegmentStatusManager.createNewSegmentId(details);
+          LoadMetadataDetails newDetail = new LoadMetadataDetails();
+          newDetail.setPartitionCount("0");
+          newDetail.setLoadName("" + segmentId);
+          newDetail.setFileFormat(FileFormat.rowformat);
+          newDetail.setLoadStartTime(System.currentTimeMillis());
+          newDetail.setLoadStatus(CarbonCommonConstants.STORE_LOADSTATUS_STREAMING);
+
+          LoadMetadataDetails[] newDetails = new LoadMetadataDetails[details.length + 1];
+          int i = 0;
+          for (; i < details.length; i++) {
+            newDetails[i] = details[i];
+          }
+          newDetails[i] = newDetail;
+          SegmentStatusManager
+              .writeLoadDetailsIntoFile(tablePath.getTableStatusFilePath(), newDetails);
+          return newDetail.getLoadName();
+        } else {
+          return streamSegment.getLoadName();
+        }
+      } else {
+        LOGGER.error(
+            "Not able to acquire the lock for stream table get or create segment for table " + table
+                .getDatabaseName() + "." + table.getFactTableName());
+        throw new IOException("Failed to get stream segment");
+      }
+    } finally {
+      if (carbonLock.unlock()) {
+        LOGGER.info("Table unlocked successfully after stream table get or create segment" + table
+            .getDatabaseName() + "." + table.getFactTableName());
+      } else {
+        LOGGER.error(
+            "Unable to unlock table lock for stream table" + table.getDatabaseName() + "." + table
+                .getFactTableName() + " during stream table get or create segment");
+      }
+    }
+  }
+
+  /**
+   * marker old stream segment to finished status and create new stream segment
+   */
+  public static String close(CarbonTable table, String segmentId)
+      throws IOException {
+    CarbonTablePath tablePath =
+        CarbonStorePath.getCarbonTablePath(table.getAbsoluteTableIdentifier());
+    SegmentStatusManager segmentStatusManager =
+        new SegmentStatusManager(table.getAbsoluteTableIdentifier());
+    ICarbonLock carbonLock = segmentStatusManager.getTableStatusLock();
+    try {
+      if (carbonLock.lockWithRetries()) {
+        LOGGER.info(
+            "Acquired lock for table" + table.getDatabaseName() + "." + table.getFactTableName()
+                + " for stream table finish segment");
+
+        LoadMetadataDetails[] details = SegmentStatusManager.readLoadMetadata(tablePath.getPath());
+        for (LoadMetadataDetails detail : details) {
+          if (segmentId.equals(detail.getLoadName())) {
+            detail.setLoadEndTime(System.currentTimeMillis());
+            detail.setLoadStatus(CarbonCommonConstants.STORE_LOADSTATUS_STREAMING_FINISH);
+            break;
+          }
+        }
+
+        int newSegmentId = SegmentStatusManager.createNewSegmentId(details);
+        LoadMetadataDetails newDetail = new LoadMetadataDetails();
+        newDetail.setPartitionCount("0");
+        newDetail.setLoadName("" + newSegmentId);
+        newDetail.setFileFormat(FileFormat.rowformat);
+        newDetail.setLoadStartTime(System.currentTimeMillis());
+        newDetail.setLoadStatus(CarbonCommonConstants.STORE_LOADSTATUS_STREAMING);
+
+        LoadMetadataDetails[] newDetails = new LoadMetadataDetails[details.length + 1];
+        int i = 0;
+        for (; i < details.length; i++) {
+          newDetails[i] = details[i];
+        }
+        newDetails[i] = newDetail;
+        SegmentStatusManager
+            .writeLoadDetailsIntoFile(tablePath.getTableStatusFilePath(), newDetails);
+        return newDetail.getLoadName();
+      } else {
+        LOGGER.error(
+            "Not able to acquire the lock for stream table status updation for table " + table
+                .getDatabaseName() + "." + table.getFactTableName());
+        throw new IOException("Failed to get stream segment");
+      }
+    } finally {
+      if (carbonLock.unlock()) {
+        LOGGER.info(
+            "Table unlocked successfully after table status updation" + table.getDatabaseName()
+                + "." + table.getFactTableName());
+      } else {
+        LOGGER.error("Unable to unlock Table lock for table" + table.getDatabaseName() + "." + table
+            .getFactTableName() + " during table status updation");
+      }
+    }
+  }
+
+  /**
+   * invoke CarbonStreamOutputFormat to append batch data to existing carbondata file
+   */
+  public static void appendBatchData(CarbonIterator<Object[]> inputIterators,
+      TaskAttemptContext job) throws Exception {
+    CarbonStreamRecordWriter writer = null;
+    try {
+      writer = (CarbonStreamRecordWriter)new CarbonStreamOutputFormat().getRecordWriter(job);
+      // at the begin of each task, should recover file if necessary
+      // here can reuse some information of record writer
+      recoverFileIfRequired(
+          writer.getSegmentDir(),
+          writer.getFileName(),
+          CarbonTablePath.getCarbonStreamIndexFileName());
+
+      while (inputIterators.hasNext()) {
+        writer.write(null, inputIterators.next());
+      }
+      inputIterators.close();
+    } finally {
+      if (writer != null) {
+        writer.close(job);
+      }
+    }
+  }
+
+  /**
+   * check the health of stream segment and try to recover segment from job fault
+   * this method will be invoked in following scenarios.
+   * 1. at the begin of the streaming (StreamSinkFactory.getStreamSegmentId)
+   * 2. after job failed (CarbonAppendableStreamSink.writeDataFileJob)
+   */
+  public static void recoverSegmentIfRequired(String segmentDir) throws IOException {
+    FileFactory.FileType fileType = FileFactory.getFileType(segmentDir);
+    if (FileFactory.isFileExist(segmentDir, fileType)) {
+      String indexName = CarbonTablePath.getCarbonStreamIndexFileName();
+      String indexPath = segmentDir + File.separator + indexName;
+      CarbonFile index = FileFactory.getCarbonFile(indexPath, fileType);
+      CarbonFile[] files = listDataFiles(segmentDir, fileType);
+      // TODO better to check backup index at first
+      // index file exists
+      if (index.exists()) {
+        // data file exists
+        if (files.length > 0) {
+          CarbonIndexFileReader indexReader = new CarbonIndexFileReader();
+          try {
+            // map block index
+            indexReader.openThriftReader(indexPath);
+            Map<String, Long> tableSizeMap = new HashMap<>();
+            while (indexReader.hasNext()) {
+              BlockIndex blockIndex = indexReader.readBlockIndexInfo();
+              tableSizeMap.put(blockIndex.getFile_name(), blockIndex.getFile_size());
+            }
+            // recover each file
+            for (CarbonFile file : files) {
+              Long size = tableSizeMap.get(file.getName());
+              if (null == size || size == 0) {
+                file.delete();
+              } else if (size < file.getSize()) {
+                FileFactory.truncateFile(file.getCanonicalPath(), fileType, size);
+              }
+            }
+          } finally {
+            indexReader.closeThriftReader();
+          }
+        }
+      } else {
+        if (files.length > 0) {
+          for (CarbonFile file : files) {
+            file.delete();
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * check the health of stream data file and try to recover data file from task fault
+   *  this method will be invoked in following scenarios.
+   *  1. at the begin of writing data file task
+   */
+  public static void recoverFileIfRequired(
+      String segmentDir,
+      String fileName,
+      String indexName) throws IOException {
+
+    FileFactory.FileType fileType = FileFactory.getFileType(segmentDir);
+    String filePath = segmentDir + File.separator + fileName;
+    CarbonFile file = FileFactory.getCarbonFile(filePath, fileType);
+    String indexPath = segmentDir + File.separator + indexName;
+    CarbonFile index = FileFactory.getCarbonFile(indexPath, fileType);
+    if (file.exists() && index.exists()) {
+      CarbonIndexFileReader indexReader = new CarbonIndexFileReader();
+      try {
+        indexReader.openThriftReader(indexPath);
+        while (indexReader.hasNext()) {
+          BlockIndex blockIndex = indexReader.readBlockIndexInfo();
+          if (blockIndex.getFile_name().equals(fileName)) {
+            if (blockIndex.getFile_size() == 0) {
+              file.delete();
+            } else if (blockIndex.getFile_size() < file.getSize()) {
+              FileFactory.truncateFile(filePath, fileType, blockIndex.getFile_size());
+            }
+          }
+        }
+      } finally {
+        indexReader.closeThriftReader();
+      }
+    }
+  }
+
+
+  /**
+   * list all carbondata files of a segment
+   */
+  public static CarbonFile[] listDataFiles(String segmentDir, FileFactory.FileType fileType) {
+    CarbonFile carbonDir = FileFactory.getCarbonFile(segmentDir, fileType);
+    if (carbonDir.exists()) {
+      return carbonDir.listFiles(new CarbonFileFilter() {
+        @Override public boolean accept(CarbonFile file) {
+          return CarbonTablePath.isCarbonDataFile(file.getName());
+        }
+      });
+    } else {
+      return new CarbonFile[0];
+    }
+  }
+
+  /**
+   * update carbonindex file after after a stream batch.
+   */
+  public static void updateIndexFile(String segmentDir) throws IOException {
+    FileFactory.FileType fileType = FileFactory.getFileType(segmentDir);
+    String filePath = CarbonTablePath.getCarbonStreamIndexFilePath(segmentDir);
+    String tempFilePath = filePath + CarbonCommonConstants.TEMPWRITEFILEEXTENSION;
+    CarbonIndexFileWriter writer = new CarbonIndexFileWriter();
+    try {
+      writer.openThriftWriter(tempFilePath);
+      CarbonFile[] files = listDataFiles(segmentDir, fileType);
+      BlockIndex blockIndex;
+      for (CarbonFile file : files) {
+        blockIndex = new BlockIndex();
+        blockIndex.setFile_name(file.getName());
+        blockIndex.setFile_size(file.getSize());
+        // TODO need to collect these information
+        blockIndex.setNum_rows(-1);
+        blockIndex.setOffset(-1);
+        blockIndex.setBlock_index(new BlockletIndex());
+        writer.writeThrift(blockIndex);
+      }
+      writer.close();
+      CarbonFile tempFile = FileFactory.getCarbonFile(tempFilePath, fileType);
+      if (!tempFile.renameForce(filePath)) {
+        throw new IOException(
+            "temporary file renaming failed, src=" + tempFilePath + ", dest=" + filePath);
+      }
+    } catch (IOException ex) {
+      try {
+        writer.close();
+      } catch (IOException t) {
+        LOGGER.error(t);
+      }
+      throw ex;
+    }
+  }
+
+  /**
+   * calculate the size of the segment by the accumulation of data sizes in index file
+   */
+  public static long size(String segmentDir) throws IOException {
+    long size = 0;
+    FileFactory.FileType fileType = FileFactory.getFileType(segmentDir);
+    if (FileFactory.isFileExist(segmentDir, fileType)) {
+      String indexPath = CarbonTablePath.getCarbonStreamIndexFilePath(segmentDir);
+      CarbonFile index = FileFactory.getCarbonFile(indexPath, fileType);
+      if (index.exists()) {
+        CarbonIndexFileReader indexReader = new CarbonIndexFileReader();
+        try {
+          indexReader.openThriftReader(indexPath);
+          while (indexReader.hasNext()) {
+            BlockIndex blockIndex = indexReader.readBlockIndexInfo();
+            size += blockIndex.getFile_size();
+          }
+        } finally {
+          indexReader.closeThriftReader();
+        }
+      }
+    }
+    return size;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/streaming/src/main/scala/org/apache/carbondata/streaming/StreamSinkFactory.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/carbondata/streaming/StreamSinkFactory.scala b/streaming/src/main/scala/org/apache/carbondata/streaming/StreamSinkFactory.scala
new file mode 100644
index 0000000..3ac19d9
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/carbondata/streaming/StreamSinkFactory.scala
@@ -0,0 +1,160 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.streaming
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd}
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.execution.streaming.{CarbonAppendableStreamSink, Sink}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.dictionary.server.DictionaryServer
+import org.apache.carbondata.core.metadata.encoder.Encoding
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.processing.loading.model.CarbonLoadModel
+import org.apache.carbondata.spark.util.DataLoadingUtil
+import org.apache.carbondata.streaming.segment.StreamSegment
+
+/**
+ * Stream sink factory
+ */
+object StreamSinkFactory {
+
+  def createStreamTableSink(
+      sparkSession: SparkSession,
+      carbonTable: CarbonTable,
+      parameters: Map[String, String]): Sink = {
+    validateParameters(parameters)
+
+    // prepare the stream segment
+    val segmentId = getStreamSegmentId(carbonTable)
+    // build load model
+    val carbonLoadModel = buildCarbonLoadModelForStream(
+      sparkSession,
+      carbonTable,
+      parameters,
+      segmentId)
+    // start server if necessary
+    val server = startDictionaryServer(
+      sparkSession,
+      carbonTable,
+      carbonLoadModel.getDictionaryServerPort)
+    if (server.isDefined) {
+      carbonLoadModel.setUseOnePass(true)
+      carbonLoadModel.setDictionaryServerPort(server.get.getPort)
+    } else {
+      carbonLoadModel.setUseOnePass(false)
+    }
+    // default is carbon appended stream sink
+    new CarbonAppendableStreamSink(
+      sparkSession,
+      carbonTable,
+      segmentId,
+      parameters,
+      carbonLoadModel,
+      server)
+  }
+
+  private def validateParameters(parameters: Map[String, String]): Unit = {
+    // TODO require to validate parameters
+  }
+
+  /**
+   * get current stream segment id
+   * @return
+   */
+  private def getStreamSegmentId(carbonTable: CarbonTable): String = {
+    val segmentId = StreamSegment.open(carbonTable)
+    val carbonTablePath = CarbonStorePath
+      .getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
+    val segmentDir = carbonTablePath.getSegmentDir("0", segmentId)
+    val fileType = FileFactory.getFileType(segmentDir)
+    if (FileFactory.isFileExist(segmentDir, fileType)) {
+      // recover fault
+      StreamSegment.recoverSegmentIfRequired(segmentDir)
+    } else {
+      FileFactory.mkdirs(segmentDir, fileType)
+    }
+    segmentId
+  }
+
+  def startDictionaryServer(
+      sparkSession: SparkSession,
+      carbonTable: CarbonTable,
+      port: Int): Option[DictionaryServer] = {
+    // start dictionary server when use one pass load and dimension with DICTIONARY
+    // encoding is present.
+    val allDimensions = carbonTable.getAllDimensions.asScala.toList
+    val createDictionary = allDimensions.exists {
+      carbonDimension => carbonDimension.hasEncoding(Encoding.DICTIONARY) &&
+                         !carbonDimension.hasEncoding(Encoding.DIRECT_DICTIONARY)
+    }
+    val server: Option[DictionaryServer] = if (createDictionary) {
+      val dictionaryServer = DictionaryServer.getInstance(port, carbonTable)
+      sparkSession.sparkContext.addSparkListener(new SparkListener() {
+        override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) {
+          dictionaryServer.shutdown()
+        }
+      })
+      Some(dictionaryServer)
+    } else {
+      None
+    }
+    server
+  }
+
+  private def buildCarbonLoadModelForStream(
+      sparkSession: SparkSession,
+      carbonTable: CarbonTable,
+      parameters: Map[String, String],
+      segmentId: String): CarbonLoadModel = {
+    val carbonProperty: CarbonProperties = CarbonProperties.getInstance()
+    carbonProperty.addProperty("zookeeper.enable.lock", "false")
+    val optionsFinal = DataLoadingUtil.getDataLoadingOptions(carbonProperty, parameters)
+    optionsFinal.put("sort_scope", "no_sort")
+    if (parameters.get("fileheader").isEmpty) {
+      optionsFinal.put("fileheader", carbonTable.getCreateOrderColumn(carbonTable.getFactTableName)
+        .asScala.map(_.getColName).mkString(","))
+    }
+    val carbonLoadModel = new CarbonLoadModel()
+    DataLoadingUtil.buildCarbonLoadModel(
+      carbonTable,
+      carbonProperty,
+      parameters,
+      optionsFinal,
+      carbonLoadModel
+    )
+    carbonLoadModel.setSegmentId(segmentId)
+    // stream should use one pass
+    val dictionaryServerPort = parameters.getOrElse(
+      CarbonCommonConstants.DICTIONARY_SERVER_PORT,
+      carbonProperty.getProperty(
+        CarbonCommonConstants.DICTIONARY_SERVER_PORT,
+        CarbonCommonConstants.DICTIONARY_SERVER_PORT_DEFAULT))
+    val sparkDriverHost = sparkSession.sqlContext.sparkContext.
+      getConf.get("spark.driver.host")
+    carbonLoadModel.setDictionaryServerHost(sparkDriverHost)
+    carbonLoadModel.setDictionaryServerPort(dictionaryServerPort.toInt)
+    carbonLoadModel
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/streaming/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala b/streaming/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala
new file mode 100644
index 0000000..844423a
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala
@@ -0,0 +1,292 @@
+/*
+ * 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.streaming
+
+import java.util.Date
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext, TaskAttemptID, TaskID, TaskType}
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
+import org.apache.spark.{SparkHadoopWriter, TaskContext}
+import org.apache.spark.internal.io.FileCommitProtocol
+import org.apache.spark.internal.io.FileCommitProtocol.TaskCommitMessage
+import org.apache.spark.sql.{DataFrame, SparkSession}
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.execution.{QueryExecution, SQLExecution}
+import org.apache.spark.util.{SerializableConfiguration, Utils}
+
+import org.apache.carbondata.common.CarbonIterator
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.dictionary.server.DictionaryServer
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.hadoop.streaming.CarbonStreamOutputFormat
+import org.apache.carbondata.processing.loading.model.CarbonLoadModel
+import org.apache.carbondata.streaming.CarbonStreamException
+import org.apache.carbondata.streaming.parser.CarbonStreamParser
+import org.apache.carbondata.streaming.segment.StreamSegment
+
+/**
+ * an implement of stream sink, it persist each batch to disk by appending the batch data to
+ * data files.
+ */
+class CarbonAppendableStreamSink(
+    sparkSession: SparkSession,
+    val carbonTable: CarbonTable,
+    var currentSegmentId: String,
+    parameters: Map[String, String],
+    carbonLoadModel: CarbonLoadModel,
+    server: Option[DictionaryServer]) extends Sink {
+
+  private val carbonTablePath = CarbonStorePath
+    .getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
+  private val fileLogPath = carbonTablePath.getStreamingLogDir
+  private val fileLog = new FileStreamSinkLog(FileStreamSinkLog.VERSION, sparkSession, fileLogPath)
+  // prepare configuration
+  private val hadoopConf = {
+    val conf = sparkSession.sessionState.newHadoopConf()
+    CarbonStreamOutputFormat.setCarbonLoadModel(conf, carbonLoadModel)
+    // put all parameters into hadoopConf
+    parameters.foreach { entry =>
+      conf.set(entry._1, entry._2)
+    }
+    conf
+  }
+
+  override def addBatch(batchId: Long, data: DataFrame): Unit = {
+    if (batchId <= fileLog.getLatest().map(_._1).getOrElse(-1L)) {
+      CarbonAppendableStreamSink.LOGGER.info(s"Skipping already committed batch $batchId")
+    } else {
+      checkOrHandOffSegment()
+
+      // committer will record how this spark job commit its output
+      val committer = FileCommitProtocol.instantiate(
+        className = sparkSession.sessionState.conf.streamingFileCommitProtocolClass,
+        jobId = batchId.toString,
+        outputPath = fileLogPath,
+        isAppend = false)
+
+      committer match {
+        case manifestCommitter: ManifestFileCommitProtocol =>
+          manifestCommitter.setupManifestOptions(fileLog, batchId)
+        case _ => // Do nothing
+      }
+
+      CarbonAppendableStreamSink.writeDataFileJob(
+        sparkSession,
+        carbonTable,
+        parameters,
+        batchId,
+        currentSegmentId,
+        data.queryExecution,
+        committer,
+        hadoopConf,
+        server)
+    }
+  }
+
+  /**
+   * if the directory size of current segment beyond the threshold, hand off new segment
+   */
+  private def checkOrHandOffSegment(): Unit = {
+    val segmentDir = carbonTablePath.getSegmentDir("0", currentSegmentId)
+    val fileType = FileFactory.getFileType(segmentDir)
+    if (StreamSegment.STREAM_SEGMENT_MAX_SIZE <=
+        StreamSegment.size(segmentDir)) {
+      val newSegmentId =
+        StreamSegment.close(carbonTable, currentSegmentId)
+      currentSegmentId = newSegmentId
+      val newSegmentDir = carbonTablePath.getSegmentDir("0", currentSegmentId)
+      FileFactory.mkdirs(newSegmentDir, fileType)
+    }
+
+    // TODO trigger hand off operation
+  }
+}
+
+object CarbonAppendableStreamSink {
+
+  private val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+
+  /**
+   * package the hadoop configuration and it will be passed to executor side from driver side
+   */
+  case class WriteDataFileJobDescription(
+      serializableHadoopConf: SerializableConfiguration,
+      batchId: Long,
+      segmentId: String)
+
+  /**
+   * Run a spark job to append the newly arrived data to the existing row format
+   * file directly.
+   * If there are failure in the task, spark will re-try the task and
+   * carbon will do recovery by HDFS file truncate. (see StreamSegment.tryRecoverFromTaskFault)
+   * If there are job level failure, every files in the stream segment will do truncate
+   * if necessary. (see StreamSegment.tryRecoverFromJobFault)
+   */
+  def writeDataFileJob(
+      sparkSession: SparkSession,
+      carbonTable: CarbonTable,
+      parameters: Map[String, String],
+      batchId: Long,
+      segmentId: String,
+      queryExecution: QueryExecution,
+      committer: FileCommitProtocol,
+      hadoopConf: Configuration,
+      server: Option[DictionaryServer]): Unit = {
+
+    // create job
+    val job = Job.getInstance(hadoopConf)
+    job.setOutputKeyClass(classOf[Void])
+    job.setOutputValueClass(classOf[InternalRow])
+
+    val description = WriteDataFileJobDescription(
+      serializableHadoopConf = new SerializableConfiguration(job.getConfiguration),
+      batchId,
+      segmentId
+    )
+
+    // run write data file job
+    SQLExecution.withNewExecutionId(sparkSession, queryExecution) {
+      var result: Array[TaskCommitMessage] = null
+      try {
+        committer.setupJob(job)
+        // initialize dictionary server
+        if (server.isDefined) {
+          server.get.initializeDictionaryGenerator(carbonTable)
+        }
+
+        // write data file
+        result = sparkSession.sparkContext.runJob(queryExecution.toRdd,
+          (taskContext: TaskContext, iterator: Iterator[InternalRow]) => {
+            writeDataFileTask(
+              description,
+              sparkStageId = taskContext.stageId(),
+              sparkPartitionId = taskContext.partitionId(),
+              sparkAttemptNumber = taskContext.attemptNumber(),
+              committer,
+              iterator
+            )
+          })
+
+        // write dictionary
+        if (server.isDefined) {
+          try {
+            server.get.writeTableDictionary(carbonTable.getCarbonTableIdentifier.getTableId)
+          } catch {
+            case _: Exception =>
+              LOGGER.error(
+                s"Error while writing dictionary file for ${carbonTable.getTableUniqueName}")
+              throw new Exception(
+                "Streaming ingest failed due to error while writing dictionary file")
+          }
+        }
+
+        // update data file info in index file
+        val tablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
+        StreamSegment.updateIndexFile(tablePath.getSegmentDir("0", segmentId))
+
+      } catch {
+        // catch fault of executor side
+        case t: Throwable =>
+          val tablePath =
+            CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
+          val segmentDir = tablePath.getSegmentDir("0", segmentId)
+          StreamSegment.recoverSegmentIfRequired(segmentDir)
+          LOGGER.error(t, s"Aborting job ${ job.getJobID }.")
+          committer.abortJob(job)
+          throw new CarbonStreamException("Job failed to write data file", t)
+      }
+      committer.commitJob(job, result)
+      LOGGER.info(s"Job ${ job.getJobID } committed.")
+    }
+  }
+
+  /**
+   * execute a task for each partition to write a data file
+   */
+  def writeDataFileTask(
+      description: WriteDataFileJobDescription,
+      sparkStageId: Int,
+      sparkPartitionId: Int,
+      sparkAttemptNumber: Int,
+      committer: FileCommitProtocol,
+      iterator: Iterator[InternalRow]
+  ): TaskCommitMessage = {
+
+    val jobId = SparkHadoopWriter.createJobID(new Date, sparkStageId)
+    val taskId = new TaskID(jobId, TaskType.MAP, sparkPartitionId)
+    val taskAttemptId = new TaskAttemptID(taskId, sparkAttemptNumber)
+
+    // Set up the attempt context required to use in the output committer.
+    val taskAttemptContext: TaskAttemptContext = {
+      // Set up the configuration object
+      val hadoopConf = description.serializableHadoopConf.value
+      hadoopConf.set("mapred.job.id", jobId.toString)
+      hadoopConf.set("mapred.tip.id", taskAttemptId.getTaskID.toString)
+      hadoopConf.set("mapred.task.id", taskAttemptId.toString)
+      hadoopConf.setBoolean("mapred.task.is.map", true)
+      hadoopConf.setInt("mapred.task.partition", 0)
+      new TaskAttemptContextImpl(hadoopConf, taskAttemptId)
+    }
+
+    committer.setupTask(taskAttemptContext)
+
+    try {
+      Utils.tryWithSafeFinallyAndFailureCallbacks(block = {
+
+        val parserName = taskAttemptContext.getConfiguration.get(
+          CarbonStreamParser.CARBON_STREAM_PARSER,
+          CarbonStreamParser.CARBON_STREAM_PARSER_DEFAULT)
+
+        val streamParser =
+          Class.forName(parserName).newInstance.asInstanceOf[CarbonStreamParser]
+        streamParser.initialize(taskAttemptContext.getConfiguration)
+
+        StreamSegment.appendBatchData(new InputIterator(iterator, streamParser),
+          taskAttemptContext)
+      })(catchBlock = {
+        committer.abortTask(taskAttemptContext)
+        LOGGER.error(s"Job $jobId aborted.")
+      })
+      committer.commitTask(taskAttemptContext)
+    } catch {
+      case t: Throwable =>
+        throw new CarbonStreamException("Task failed while writing rows", t)
+    }
+  }
+
+  /**
+   * convert spark iterator to carbon iterator, so that java module can use it.
+   */
+  class InputIterator(rddIter: Iterator[InternalRow], streamParser: CarbonStreamParser)
+    extends CarbonIterator[Array[Object]] {
+
+    override def hasNext: Boolean = rddIter.hasNext
+
+    override def next: Array[Object] = {
+      streamParser.parserRow(rddIter.next())
+    }
+
+    override def close(): Unit = {
+      streamParser.close()
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d7393da9/streaming/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonStreamingQueryListener.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonStreamingQueryListener.scala b/streaming/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonStreamingQueryListener.scala
new file mode 100644
index 0000000..80936d1
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonStreamingQueryListener.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.spark.sql.execution.streaming
+
+import java.util
+import java.util.UUID
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.streaming.StreamingQueryListener
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.locks.{CarbonLockFactory, ICarbonLock, LockUsage}
+
+class CarbonStreamingQueryListener(spark: SparkSession) extends StreamingQueryListener {
+
+  private val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+
+  private val cache = new util.HashMap[UUID, ICarbonLock]()
+
+  override def onQueryStarted(event: StreamingQueryListener.QueryStartedEvent): Unit = {
+    val qry = spark.streams.get(event.id).asInstanceOf[StreamExecution]
+    if (qry.sink.isInstanceOf[CarbonAppendableStreamSink]) {
+      LOGGER.info("Carbon streaming query started: " + event.id)
+      val sink = qry.sink.asInstanceOf[CarbonAppendableStreamSink]
+      val carbonTable = sink.carbonTable
+      val lock = CarbonLockFactory.getCarbonLockObj(carbonTable.getCarbonTableIdentifier,
+        LockUsage.STREAMING_LOCK)
+      if (lock.lockWithRetries()) {
+        LOGGER.info("Acquired the lock for stream table: " + carbonTable.getDatabaseName + "." +
+                    carbonTable.getFactTableName)
+        cache.put(event.id, lock)
+      } else {
+        LOGGER.error("Not able to acquire the lock for stream table:" +
+                     carbonTable.getDatabaseName + "." + carbonTable.getFactTableName)
+        throw new InterruptedException(
+          "Not able to acquire the lock for stream table: " + carbonTable.getDatabaseName + "." +
+          carbonTable.getFactTableName)
+      }
+    }
+  }
+
+  override def onQueryProgress(event: StreamingQueryListener.QueryProgressEvent): Unit = {
+  }
+
+  override def onQueryTerminated(event: StreamingQueryListener.QueryTerminatedEvent): Unit = {
+    val lock = cache.remove(event.id)
+    if (null != lock) {
+      LOGGER.info("Carbon streaming query: " + event.id)
+      lock.unlock()
+    }
+  }
+}


[43/49] carbondata git commit: [CARBONDATA-1527] [CARBONDATA-1528] [PreAgg] Restrict alter/update/delete for pre-aggregate table

Posted by ra...@apache.org.
[CARBONDATA-1527] [CARBONDATA-1528] [PreAgg] Restrict alter/update/delete for pre-aggregate table

This closes #1476


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

Branch: refs/heads/fgdatamap
Commit: 2b5faefada0d9078988f28b33249ebc3b2549c80
Parents: cc0e6f1
Author: kunal642 <ku...@gmail.com>
Authored: Mon Oct 23 18:28:15 2017 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Tue Nov 14 00:48:16 2017 +0530

----------------------------------------------------------------------
 .../core/metadata/schema/table/CarbonTable.java |   4 +
 .../schema/table/RelationIdentifier.java        |  12 ++
 .../preaggregate/TestPreAggregateLoad.scala     |   3 +-
 .../iud/DeleteCarbonTableTestCase.scala         |  15 ++
 .../iud/UpdateCarbonTableTestCase.scala         |  15 ++
 .../testsuite/sortcolumns/TestSortColumns.scala |   3 +-
 .../carbondata/events/AlterTableEvents.scala    |  12 +-
 .../org/apache/carbondata/events/Events.scala   |   7 +-
 .../management/DeleteLoadByIdCommand.scala      |   2 +-
 .../DeleteLoadByLoadDateCommand.scala           |   6 +-
 .../CreatePreAggregateTableCommand.scala        |   2 +
 .../preaaggregate/PreAggregateListeners.scala   | 188 ++++++++++++++++++-
 .../schema/CarbonAlterTableRenameCommand.scala  |   2 +-
 .../spark/sql/hive/CarbonSessionState.scala     |  13 +-
 .../AlterTableValidationTestCase.scala          |  21 ++-
 .../vectorreader/ChangeDataTypeTestCases.scala  |  17 ++
 .../vectorreader/DropColumnTestCases.scala      |  16 ++
 .../apache/spark/util/CarbonCommandSuite.scala  |  18 +-
 18 files changed, 339 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/2b5faefa/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 4a6fb8b..ca0952d 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
@@ -707,4 +707,8 @@ public class CarbonTable implements Serializable {
         .getParentRelationIdentifiers().isEmpty();
   }
 
+  public boolean hasPreAggregateTables() {
+    return tableInfo.getDataMapSchemaList() != null && !tableInfo
+        .getDataMapSchemaList().isEmpty();
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2b5faefa/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
index 2a2d937..c9c44bf 100644
--- 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
@@ -26,6 +26,18 @@ import java.io.Serializable;
  */
 public class RelationIdentifier implements Serializable, Writable {
 
+  public void setDatabaseName(String databaseName) {
+    this.databaseName = databaseName;
+  }
+
+  public void setTableName(String tableName) {
+    this.tableName = tableName;
+  }
+
+  public void setTableId(String tableId) {
+    this.tableId = tableId;
+  }
+
   private String databaseName;
 
   private String tableName;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2b5faefa/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateLoad.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateLoad.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateLoad.scala
index 0c65577..1f576c5 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateLoad.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateLoad.scala
@@ -19,8 +19,9 @@ package org.apache.carbondata.integration.spark.testsuite.preaggregate
 
 import org.apache.spark.sql.Row
 import org.apache.spark.sql.test.util.QueryTest
-import org.scalatest.BeforeAndAfterAll
+import org.scalatest.{BeforeAndAfterAll, Ignore}
 
+@Ignore
 class TestPreAggregateLoad extends QueryTest with BeforeAndAfterAll {
   
   val testData = s"$resourcesPath/sample.csv"

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2b5faefa/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/DeleteCarbonTableTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/DeleteCarbonTableTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/DeleteCarbonTableTestCase.scala
index 3c2842c..a2bd6aa 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/DeleteCarbonTableTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/DeleteCarbonTableTestCase.scala
@@ -129,6 +129,21 @@ class DeleteCarbonTableTestCase extends QueryTest with BeforeAndAfterAll {
     sql("DROP TABLE IF EXISTS default.carbon2")
   }
 
+  test("test if delete is unsupported for pre-aggregate tables") {
+    sql("drop table if exists preaggMain")
+    sql("drop table if exists preagg1")
+    sql("create table preaggMain (a string, b string, c string) stored by 'carbondata'")
+    sql("create table preagg1 stored BY 'carbondata' tblproperties('parent'='PreAggMain') as select a,sum(b) from PreAggMain group by a")
+    intercept[RuntimeException] {
+      sql("delete from preaggmain where a = 'abc'").show()
+    }.getMessage.contains("Delete operation is not supported for tables")
+    intercept[RuntimeException] {
+      sql("delete from preagg1 where preaggmain_a = 'abc'").show()
+    }.getMessage.contains("Delete operation is not supported for pre-aggregate table")
+    sql("drop table if exists preaggMain")
+    sql("drop table if exists preagg1")
+  }
+
 
   override def afterAll {
     sql("use default")

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2b5faefa/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/UpdateCarbonTableTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/UpdateCarbonTableTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/UpdateCarbonTableTestCase.scala
index db289d9..4c43ec0 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/UpdateCarbonTableTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/UpdateCarbonTableTestCase.scala
@@ -513,6 +513,21 @@ class UpdateCarbonTableTestCase extends QueryTest with BeforeAndAfterAll {
     sql("DROP TABLE IF EXISTS iud.rand")
   }
 
+  test("test if update is unsupported for pre-aggregate tables") {
+    sql("drop table if exists preaggMain")
+    sql("drop table if exists preagg1")
+    sql("create table preaggMain (a string, b string, c string) stored by 'carbondata'")
+    sql("create table preagg1 stored BY 'carbondata' tblproperties('parent'='PreAggMain') as select a,sum(b) from PreAggMain group by a")
+    intercept[RuntimeException] {
+      sql("update preaggmain set (a)=('a')").show
+    }.getMessage.contains("Update operation is not supported for tables")
+    intercept[RuntimeException] {
+      sql("update preagg1 set (a)=('a')").show
+    }.getMessage.contains("Update operation is not supported for pre-aggregate table")
+    sql("drop table if exists preaggMain")
+    sql("drop table if exists preagg1")
+  }
+
   override def afterAll {
     sql("use default")
     sql("drop database  if exists iud cascade")

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2b5faefa/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumns.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumns.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumns.scala
index b5fd8a9..6c5aa55 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumns.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumns.scala
@@ -35,7 +35,7 @@ class TestSortColumns extends QueryTest with BeforeAndAfterAll {
     sql("CREATE TABLE tableOne(id int, name string, city string, age int) STORED BY 'org.apache.carbondata.format'")
     sql("CREATE TABLE tableTwo(id int, age int) STORED BY 'org.apache.carbondata.format'")
     sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/measureinsertintotest.csv' into table tableOne")
-    sql("insert into table tableTwo select id, count(age) from tableOne group by id")
+
   }
 
   test("create table sort columns dictionary include - int") {
@@ -335,6 +335,7 @@ class TestSortColumns extends QueryTest with BeforeAndAfterAll {
   }
 
   test("Test tableTwo data") {
+    sql("insert into table tableTwo select id, count(age) from tableOne group by id")
     checkAnswer(
       sql("select id,age from tableTwo order by id"),
       Seq(Row(1, 1), Row(2, 1), Row(3, 2), Row(4, 2)))

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2b5faefa/integration/spark-common/src/main/scala/org/apache/carbondata/events/AlterTableEvents.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/events/AlterTableEvents.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/events/AlterTableEvents.scala
index ec79acc..2f7cf63 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/events/AlterTableEvents.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/events/AlterTableEvents.scala
@@ -18,7 +18,7 @@ package org.apache.carbondata.events
 
 import org.apache.spark.sql.SparkSession
 import org.apache.spark.sql.SQLContext
-import org.apache.spark.sql.execution.command.{AlterTableDropColumnModel, AlterTableRenameModel}
+import org.apache.spark.sql.execution.command.{AlterTableDataTypeChangeModel, AlterTableDropColumnModel, AlterTableRenameModel}
 
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.processing.loading.model.CarbonLoadModel
@@ -35,6 +35,16 @@ case class AlterTableDropColumnPreEvent(carbonTable: CarbonTable,
 
 
 /**
+ * Class for handling clean up in case of any failure and abort the operation
+ *
+ * @param carbonTable
+ * @param alterTableDataTypeChangeModel
+ */
+case class AlterTableDataTypeChangePreEvent(carbonTable: CarbonTable,
+        alterTableDataTypeChangeModel: AlterTableDataTypeChangeModel)
+  extends Event with AlterTableDataTypeChangeEventInfo
+
+/**
  *
  * @param carbonTable
  * @param alterTableDropColumnModel

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2b5faefa/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 4f8d57e..9796dea 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
@@ -18,7 +18,7 @@
 package org.apache.carbondata.events
 
 import org.apache.spark.sql.SparkSession
-import org.apache.spark.sql.execution.command.{AlterTableDropColumnModel, AlterTableRenameModel}
+import org.apache.spark.sql.execution.command.{AlterTableAddColumnsModel, AlterTableDataTypeChangeModel, AlterTableDropColumnModel, AlterTableRenameModel}
 
 import org.apache.carbondata.core.metadata.CarbonTableIdentifier
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
@@ -69,6 +69,11 @@ trait AlterTableDropColumnEventInfo {
   val alterTableDropColumnModel: AlterTableDropColumnModel
 }
 
+trait AlterTableDataTypeChangeEventInfo {
+  val carbonTable: CarbonTable
+  val alterTableDataTypeChangeModel: AlterTableDataTypeChangeModel
+}
+
 /**
  * event for alter_table_rename
  */

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2b5faefa/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/DeleteLoadByIdCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/DeleteLoadByIdCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/DeleteLoadByIdCommand.scala
index 9ea4018..6a0465c 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/DeleteLoadByIdCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/DeleteLoadByIdCommand.scala
@@ -57,7 +57,7 @@ case class DeleteLoadByIdCommand(
       DeleteSegmentByIdPostEvent(carbonTable,
         loadIds,
         sparkSession)
-    OperationListenerBus.getInstance.fireEvent(deleteSegmentByIdPreEvent, operationContext)
+    OperationListenerBus.getInstance.fireEvent(deleteSegmentPostEvent, operationContext)
     Seq.empty
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2b5faefa/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/DeleteLoadByLoadDateCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/DeleteLoadByLoadDateCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/DeleteLoadByLoadDateCommand.scala
index 58d8236..83f41bb 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/DeleteLoadByLoadDateCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/DeleteLoadByLoadDateCommand.scala
@@ -51,14 +51,12 @@ case class DeleteLoadByLoadDateCommand(
       loadDate,
       GetDB.getDatabaseName(databaseNameOp, sparkSession),
       tableName,
-      carbonTable
-    )
-
+      carbonTable)
     val deleteSegmentPostEvent: DeleteSegmentByDatePostEvent =
       DeleteSegmentByDatePostEvent(carbonTable,
         loadDate,
         sparkSession)
-    OperationListenerBus.getInstance.fireEvent(deleteSegmentByDatePreEvent, operationContext)
+    OperationListenerBus.getInstance.fireEvent(deleteSegmentPostEvent, operationContext)
 
     Seq.empty
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2b5faefa/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 b952285..e12cbb9 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
@@ -137,3 +137,5 @@ case class CreatePreAggregateTableCommand(
     Seq.empty
   }
 }
+
+

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2b5faefa/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateListeners.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateListeners.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateListeners.scala
index b507856..2ce97fe 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateListeners.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateListeners.scala
@@ -24,7 +24,7 @@ import org.apache.spark.sql.execution.command.CarbonDropTableCommand
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.metadata.schema.table.DataMapSchema
-import org.apache.carbondata.events.{DropTablePostEvent, Event, LoadTablePostExecutionEvent, OperationContext, OperationEventListener}
+import org.apache.carbondata.events._
 
 object DropPreAggregateTablePostListener extends OperationEventListener {
 
@@ -79,3 +79,189 @@ object LoadPostAggregateListener extends OperationEventListener {
     }
   }
 }
+
+object PreAggregateDataTypeChangePreListener extends OperationEventListener {
+  /**
+   * Called on a specified event occurrence
+   *
+   * @param event
+   * @param operationContext
+   */
+  override def onEvent(event: Event, operationContext: OperationContext): Unit = {
+    val dataTypeChangePreListener = event.asInstanceOf[AlterTableDataTypeChangePreEvent]
+    val carbonTable = dataTypeChangePreListener.carbonTable
+    val alterTableDataTypeChangeModel = dataTypeChangePreListener.alterTableDataTypeChangeModel
+    val columnToBeAltered: String = alterTableDataTypeChangeModel.columnName
+    val dataMapSchemas = carbonTable.getTableInfo.getDataMapSchemaList
+    if (dataMapSchemas != null && !dataMapSchemas.isEmpty) {
+      dataMapSchemas.asScala.foreach { dataMapSchema =>
+          val childColumns = dataMapSchema.getChildSchema.getListOfColumns
+          if (childColumns.asScala.exists(_.getColumnName.equalsIgnoreCase(columnToBeAltered))) {
+            throw new UnsupportedOperationException(s"Column $columnToBeAltered exists in a " +
+                                                    s"pre-aggregate table ${ dataMapSchema.toString
+                                                    }. Cannot change datatype")
+          }
+      }
+
+      if (carbonTable.isPreAggregateTable) {
+        throw new UnsupportedOperationException(s"Cannot change data type for columns in " +
+                                                s"pre-aggreagate table ${
+                                                  carbonTable.getDatabaseName
+                                                }.${ carbonTable.getFactTableName }")
+      }
+    }
+  }
+}
+
+object PreAggregateDeleteSegmentByDatePreListener extends OperationEventListener {
+  /**
+   * Called on a specified event occurrence
+   *
+   * @param event
+   * @param operationContext
+   */
+  override def onEvent(event: Event, operationContext: OperationContext): Unit = {
+    val deleteSegmentByDatePreEvent = event.asInstanceOf[DeleteSegmentByDatePreEvent]
+    val carbonTable = deleteSegmentByDatePreEvent.carbonTable
+    if (carbonTable != null) {
+      if (carbonTable.hasPreAggregateTables) {
+        throw new UnsupportedOperationException(
+          "Delete segment operation is not supported on tables which have a pre-aggregate table. " +
+          "Drop pre-aggregation table to continue")
+      }
+      if (carbonTable.isPreAggregateTable) {
+        throw new UnsupportedOperationException(
+          "Delete segment operation is not supported on pre-aggregate table")
+      }
+    }
+  }
+}
+
+object PreAggregateDeleteSegmentByIdPreListener extends OperationEventListener {
+  /**
+   * Called on a specified event occurrence
+   *
+   * @param event
+   * @param operationContext
+   */
+  override def onEvent(event: Event, operationContext: OperationContext): Unit = {
+    val tableEvent = event.asInstanceOf[DeleteSegmentByIdPreEvent]
+    val carbonTable = tableEvent.carbonTable
+    if (carbonTable != null) {
+      if (carbonTable.hasPreAggregateTables) {
+        throw new UnsupportedOperationException(
+          "Delete segment operation is not supported on tables which have a pre-aggregate table")
+      }
+      if (carbonTable.isPreAggregateTable) {
+        throw new UnsupportedOperationException(
+          "Delete segment operation is not supported on pre-aggregate table")
+      }
+    }
+  }
+
+}
+
+object PreAggregateDropColumnPreListener extends OperationEventListener {
+  /**
+   * Called on a specified event occurrence
+   *
+   * @param event
+   * @param operationContext
+   */
+  override def onEvent(event: Event, operationContext: OperationContext): Unit = {
+    val dataTypeChangePreListener = event.asInstanceOf[AlterTableDropColumnPreEvent]
+    val carbonTable = dataTypeChangePreListener.carbonTable
+    val alterTableDropColumnModel = dataTypeChangePreListener.alterTableDropColumnModel
+    val columnsToBeDropped = alterTableDropColumnModel.columns
+    val dataMapSchemas = carbonTable.getTableInfo.getDataMapSchemaList
+    if (dataMapSchemas != null && !dataMapSchemas.isEmpty) {
+      dataMapSchemas.asScala.foreach { dataMapSchema =>
+          val parentColumnNames = dataMapSchema.getChildSchema.getListOfColumns.asScala
+            .flatMap(_.getParentColumnTableRelations.asScala.map(_.getColumnName))
+          val columnExistsInChild = parentColumnNames.collectFirst {
+            case parentColumnName if columnsToBeDropped.contains(parentColumnName) =>
+              parentColumnName
+          }
+          if (columnExistsInChild.isDefined) {
+            throw new UnsupportedOperationException(
+              s"Column ${ columnExistsInChild.head } cannot be dropped because it exists in a " +
+              s"pre-aggregate table ${ dataMapSchema.getRelationIdentifier.toString}")
+          }
+      }
+      if (carbonTable.isPreAggregateTable) {
+        throw new UnsupportedOperationException(s"Cannot drop columns in pre-aggreagate table ${
+          carbonTable.getDatabaseName}.${ carbonTable.getFactTableName }")
+      }
+    }
+  }
+}
+
+object PreAggregateRenameTablePreListener extends OperationEventListener {
+  /**
+   * Called on a specified event occurrence
+   *
+   * @param event
+   * @param operationContext
+   */
+  override def onEvent(event: Event,
+      operationContext: OperationContext): Unit = {
+    val renameTablePostListener = event.asInstanceOf[AlterTableRenamePreEvent]
+    val carbonTable = renameTablePostListener.carbonTable
+    if (carbonTable.isPreAggregateTable) {
+      throw new UnsupportedOperationException(
+        "Rename operation for pre-aggregate table is not supported.")
+    }
+    if (carbonTable.hasPreAggregateTables) {
+      throw new UnsupportedOperationException(
+        "Rename operation is not supported for table with pre-aggregate tables")
+    }
+  }
+}
+
+object UpdatePreAggregatePreListener extends OperationEventListener {
+  /**
+   * Called on a specified event occurrence
+   *
+   * @param event
+   * @param operationContext
+   */
+  override def onEvent(event: Event, operationContext: OperationContext): Unit = {
+    val tableEvent = event.asInstanceOf[UpdateTablePreEvent]
+    val carbonTable = tableEvent.carbonTable
+    if (carbonTable != null) {
+      if (carbonTable.hasPreAggregateTables) {
+        throw new UnsupportedOperationException(
+          "Update operation is not supported for tables which have a pre-aggregate table. Drop " +
+          "pre-aggregate tables to continue.")
+      }
+      if (carbonTable.isPreAggregateTable) {
+        throw new UnsupportedOperationException(
+          "Update operation is not supported for pre-aggregate table")
+      }
+    }
+  }
+}
+
+object DeletePreAggregatePreListener extends OperationEventListener {
+  /**
+   * Called on a specified event occurrence
+   *
+   * @param event
+   * @param operationContext
+   */
+  override def onEvent(event: Event, operationContext: OperationContext): Unit = {
+    val tableEvent = event.asInstanceOf[DeleteFromTablePreEvent]
+    val carbonTable = tableEvent.carbonTable
+    if (carbonTable != null) {
+      if (carbonTable.hasPreAggregateTables) {
+        throw new UnsupportedOperationException(
+          "Delete operation is not supported for tables which have a pre-aggregate table. Drop " +
+          "pre-aggregate tables to continue.")
+      }
+      if (carbonTable.isPreAggregateTable) {
+        throw new UnsupportedOperationException(
+          "Delete operation is not supported for pre-aggregate table")
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2b5faefa/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 e0617d6..b96baff 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
@@ -136,7 +136,7 @@ private[sql] case class CarbonAlterTableRenameCommand(
         alterTableRenameModel,
         newTablePath,
         sparkSession)
-      OperationListenerBus.getInstance().fireEvent(alterTableRenamePreEvent, operationContext)
+      OperationListenerBus.getInstance().fireEvent(alterTableRenamePostEvent, operationContext)
 
       sparkSession.catalog.refreshTable(TableIdentifier(newTableName,
         Some(oldDatabaseName)).quotedString)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2b5faefa/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 d17dd11..f698dd4 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,7 +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, LoadPostAggregateListener}
+import org.apache.spark.sql.execution.command.preaaggregate._
 import org.apache.spark.sql.execution.datasources._
 import org.apache.spark.sql.execution.strategy.{CarbonLateDecodeStrategy, DDLStrategy, StreamingTableStrategy}
 import org.apache.spark.sql.internal.SQLConf
@@ -35,7 +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, LoadTablePostExecutionEvent, OperationListenerBus}
+import org.apache.carbondata.events._
 
 /**
  * This class will have carbon catalog and refresh the relation from cache if the carbontable in
@@ -132,8 +132,15 @@ object CarbonSessionState {
     OperationListenerBus.getInstance()
       .addListener(classOf[DropTablePostEvent], DropPreAggregateTablePostListener)
       .addListener(classOf[LoadTablePostExecutionEvent], LoadPostAggregateListener)
+      .addListener(classOf[DeleteSegmentByIdPreEvent], PreAggregateDeleteSegmentByIdPreListener)
+      .addListener(classOf[DeleteSegmentByDatePreEvent], PreAggregateDeleteSegmentByDatePreListener)
+      .addListener(classOf[UpdateTablePreEvent], UpdatePreAggregatePreListener)
+      .addListener(classOf[DeleteFromTablePreEvent], DeletePreAggregatePreListener)
+      .addListener(classOf[DeleteFromTablePreEvent], DeletePreAggregatePreListener)
+      .addListener(classOf[AlterTableDropColumnPreEvent], PreAggregateDropColumnPreListener)
+      .addListener(classOf[AlterTableRenamePreEvent], PreAggregateRenameTablePreListener)
+      .addListener(classOf[AlterTableDataTypeChangePreEvent], PreAggregateDataTypeChangePreListener)
   }
-
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2b5faefa/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 6f618fe..0274605 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
@@ -21,8 +21,11 @@ import java.io.File
 import java.math.{BigDecimal, RoundingMode}
 import java.sql.Timestamp
 
-import org.apache.spark.sql.Row
+import org.apache.spark.sql.{CarbonEnv, Row}
 import org.apache.spark.sql.common.util.Spark2QueryTest
+import org.apache.spark.sql.execution.command.preaaggregate.PreAggregateUtil
+import org.apache.spark.sql.test.Spark2TestQueryExecutor
+import org.junit.Assert
 import org.scalatest.BeforeAndAfterAll
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
@@ -487,6 +490,22 @@ class AlterTableValidationTestCase extends Spark2QueryTest with BeforeAndAfterAl
     checkExistence(sql("describe formatted specifiedSortColumnsWithAlter"),true,"empno,empname,role,doj")
   }
 
+  test("test to check if new parent table name is reflected in pre-aggregate tables") {
+    sql("drop table if exists preaggMain")
+    sql("drop table if exists preaggmain_new")
+    sql("drop table if exists preagg1")
+    sql("create table preaggMain (a string, b string, c string) stored by 'carbondata'")
+    sql(
+      "create table preagg1 stored BY 'carbondata' tblproperties('parent'='PreAggMain') as select" +
+      " a,sum(b) from PreAggMain group by a")
+    intercept[RuntimeException] {
+      sql("alter table preagg1 rename to preagg2")
+    }.getMessage.contains("Rename operation for pre-aggregate table is not supported.")
+    intercept[RuntimeException] {
+      sql("alter table preaggmain rename to preaggmain_new")
+    }.getMessage.contains("Rename operation is not supported for table with pre-aggregate tables")
+  }
+
   override def afterAll {
     sql("DROP TABLE IF EXISTS restructure")
     sql("drop table if exists table1")

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2b5faefa/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/ChangeDataTypeTestCases.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/ChangeDataTypeTestCases.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/ChangeDataTypeTestCases.scala
index 9f104ed..67ea21e 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/ChangeDataTypeTestCases.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/ChangeDataTypeTestCases.scala
@@ -147,6 +147,23 @@ class ChangeDataTypeTestCases extends Spark2QueryTest with BeforeAndAfterAll {
     test_change_int_to_long()
   }
 
+  test("test data type change for with pre-aggregate table should throw exception") {
+    sql("drop table if exists preaggMain")
+    sql("drop table if exists preagg1")
+    sql("create table preaggMain (a string, b string, c string) stored by 'carbondata'")
+    sql(
+      "create table preagg1 stored BY 'carbondata' tblproperties('parent'='PreAggMain') as select" +
+      " a,sum(b) from PreAggMain group by a")
+    intercept[RuntimeException] {
+      sql("alter table preaggmain drop columns(a)").show
+    }.getMessage.contains("exists in pre-aggregate table")
+    intercept[RuntimeException] {
+      sql("alter table preagg1 drop columns(a)").show
+    }.getMessage.contains("cannot be dropped")
+    sql("drop table if exists preaggMain")
+    sql("drop table if exists preagg1")
+  }
+
   override def afterAll {
     sql("DROP TABLE IF EXISTS changedatatypetest")
     sql("DROP TABLE IF EXISTS hivetable")

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2b5faefa/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/DropColumnTestCases.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/DropColumnTestCases.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/DropColumnTestCases.scala
index 00e4a14..1a1d5d5 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/DropColumnTestCases.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/DropColumnTestCases.scala
@@ -98,6 +98,22 @@ class DropColumnTestCases extends Spark2QueryTest with BeforeAndAfterAll {
     test_drop_and_compaction()
   }
 
+  test("test dropping of column in pre-aggregate should throw exception") {
+    sql("drop table if exists preaggMain")
+    sql("drop table if exists preagg1")
+    sql("create table preaggMain (a string, b string, c string) stored by 'carbondata'")
+    sql(
+      "create table preagg1 stored BY 'carbondata' tblproperties('parent'='PreAggMain') as select" +
+      " a,sum(b) from PreAggMain group by a")
+    sql("alter table preaggmain drop columns(c)").show
+    checkExistence(sql("desc table preaggmain"), false, "c")
+    intercept[RuntimeException] {
+      sql("alter table preaggmain drop columns(a)").show
+    }.getMessage.contains("cannot be dropped")
+    sql("drop table if exists preaggMain")
+    sql("drop table if exists preagg1")
+  }
+
   override def afterAll {
     sql("DROP TABLE IF EXISTS dropcolumntest")
     sql("DROP TABLE IF EXISTS hivetable")

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2b5faefa/integration/spark2/src/test/scala/org/apache/spark/util/CarbonCommandSuite.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/util/CarbonCommandSuite.scala b/integration/spark2/src/test/scala/org/apache/spark/util/CarbonCommandSuite.scala
index decb861..c65bcc4 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/util/CarbonCommandSuite.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/util/CarbonCommandSuite.scala
@@ -22,12 +22,11 @@ import java.sql.Timestamp
 import java.util.Date
 
 import org.apache.spark.sql.common.util.Spark2QueryTest
-import org.apache.spark.sql.test.TestQueryExecutor
 import org.scalatest.BeforeAndAfterAll
 
 import org.apache.carbondata.api.CarbonStore
 import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
+import org.apache.carbondata.core.util.CarbonProperties
 
 class CarbonCommandSuite extends Spark2QueryTest with BeforeAndAfterAll {
 
@@ -142,6 +141,21 @@ class CarbonCommandSuite extends Spark2QueryTest with BeforeAndAfterAll {
     dropTable(table)
   }
 
+  test("test if delete segments by id is unsupported for pre-aggregate tables") {
+    dropTable("preaggMain")
+    dropTable("preagg1")
+    sql("create table preaggMain (a string, b string, c string) stored by 'carbondata'")
+    sql("create table preagg1 stored BY 'carbondata' tblproperties('parent'='PreAggMain') as select a,sum(b) from PreAggMain group by a")
+    intercept[UnsupportedOperationException] {
+      sql("delete from table preaggMain where segment.id in (1,2)")
+    }.getMessage.contains("Delete segment operation is not supported on tables")
+    intercept[UnsupportedOperationException] {
+      sql("delete from table preagg1 where segment.id in (1,2)")
+    }.getMessage.contains("Delete segment operation is not supported on pre-aggregate tables")
+    dropTable("preaggMain")
+    dropTable("preagg1")
+  }
+
   protected def dropTable(tableName: String): Unit ={
     sql(s"DROP TABLE IF EXISTS $tableName")
   }


[34/49] 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/fgdatamap
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)
   }
 
   /**


[18/49] carbondata git commit: [CARBONDATA-1580][Streaming] Create table with streaming property

Posted by ra...@apache.org.
[CARBONDATA-1580][Streaming] Create table with streaming property

This closes #1449


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

Branch: refs/heads/fgdatamap
Commit: 4c41f8662c60c10f35f0eed4758975ea8b926a6e
Parents: 74bd52b
Author: Jacky Li <ja...@qq.com>
Authored: Wed Nov 8 10:51:04 2017 +0800
Committer: QiangCai <qi...@qq.com>
Committed: Wed Nov 8 12:16:44 2017 +0800

----------------------------------------------------------------------
 .../apache/carbondata/spark/CarbonOption.scala  |   2 +
 .../spark/sql/catalyst/CarbonDDLSqlParser.scala |  10 +-
 .../sql/parser/CarbonSpark2SqlParser.scala      |   3 +-
 .../spark/sql/parser/CarbonSparkSqlParser.scala | 114 ++++++++++++-------
 .../TestStreamingTableOperation.scala           |  30 ++++-
 5 files changed, 116 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/4c41f866/integration/spark-common/src/main/scala/org/apache/carbondata/spark/CarbonOption.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/CarbonOption.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/CarbonOption.scala
index 0bc9285..fe07aac 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/CarbonOption.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/CarbonOption.scala
@@ -59,5 +59,7 @@ class CarbonOption(options: Map[String, String]) {
   def isBucketingEnabled: Boolean = options.contains("bucketcolumns") &&
                                     options.contains("bucketnumber")
 
+  def isStreaming: Boolean = options.getOrElse("streaming", "false").toBoolean
+
   def toMap: Map[String, String] = options
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4c41f866/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 aae4f25..ee51954 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
@@ -242,11 +242,15 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
    * @param tableProperties
    * @return
    */
-  def prepareTableModel(ifNotExistPresent: Boolean, dbName: Option[String]
-      , tableName: String, fields: Seq[Field],
+  def prepareTableModel(
+      ifNotExistPresent: Boolean,
+      dbName: Option[String],
+      tableName: String,
+      fields: Seq[Field],
       partitionCols: Seq[PartitionerField],
       tableProperties: mutable.Map[String, String],
-      bucketFields: Option[BucketFields], isAlterFlow: Boolean = false,
+      bucketFields: Option[BucketFields],
+      isAlterFlow: Boolean = false,
       tableComment: Option[String] = None): TableModel = {
 
     fields.zipWithIndex.foreach { case (field, index) =>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4c41f866/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 9f4a8ce..9c87b8b 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
@@ -456,7 +456,8 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
     }
   }
 
-  def getBucketFields(properties: mutable.Map[String, String],
+  def getBucketFields(
+      properties: mutable.Map[String, String],
       fields: Seq[Field],
       options: CarbonOption): Option[BucketFields] = {
     if (!CommonUtil.validateTblProperties(properties,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4c41f866/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 81ce73f..0a918df 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
@@ -19,15 +19,17 @@ package org.apache.spark.sql.parser
 import scala.collection.mutable
 
 import org.apache.spark.sql.{CarbonEnv, 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, Field, PartitionerField, TableModel}
+import org.apache.spark.sql.execution.command.{BucketFields, CarbonCreateTableCommand, PartitionerField, TableModel}
 import org.apache.spark.sql.internal.{SQLConf, VariableSubstitution}
+import org.apache.spark.sql.types.StructField
 
-import org.apache.carbondata.core.util.{CarbonSessionInfo, SessionParams, ThreadLocalSessionInfo}
+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
@@ -99,54 +101,42 @@ class CarbonSqlAstBuilder(conf: SQLConf) extends SparkSqlAstBuilder(conf) {
       if (ctx.bucketSpec != null) {
         operationNotAllowed("CREATE TABLE ... CLUSTERED BY", ctx)
       }
-      val partitionByStructFields = Option(ctx.partitionColumns).toSeq.flatMap(visitColTypeList)
-      val partitionerFields = partitionByStructFields.map { structField =>
-        PartitionerField(structField.name, Some(structField.dataType.toString), null)
-      }
-      val tableComment = Option(ctx.STRING()).map(string)
-      val cols = Option(ctx.columns).toSeq.flatMap(visitColTypeList)
-      val properties = Option(ctx.tablePropertyList).map(visitPropertyKeyValues)
-        .getOrElse(Map.empty)
 
-      // Ensuring whether no duplicate name is used in table definition
-      val colNames = cols.map(_.name)
-      if (colNames.length != colNames.distinct.length) {
-        val duplicateColumns = colNames.groupBy(identity).collect {
-          case (x, ys) if ys.length > 1 => "\"" + x + "\""
-        }
-        operationNotAllowed(s"Duplicated column names found in table definition of $name: " +
-                            duplicateColumns.mkString("[", ",", "]"), ctx)
-      }
+      // validate schema
+      val (colsStructFields, colNames) = validateSchema(ctx, name)
 
       val tableProperties = mutable.Map[String, String]()
+      val properties = Option(ctx.tablePropertyList).map(visitPropertyKeyValues)
+        .getOrElse(Map.empty)
       properties.foreach{property => tableProperties.put(property._1, property._2)}
 
-      // validate partition clause
-      if (partitionerFields.nonEmpty) {
-        if (!CommonUtil.validatePartitionColumns(tableProperties, partitionerFields)) {
-          throw new MalformedCarbonCommandException("Error: Invalid partition definition")
-        }
-        // partition columns should not be part of the schema
-        val badPartCols = partitionerFields.map(_.partitionColumn).toSet.intersect(colNames.toSet)
-        if (badPartCols.nonEmpty) {
-          operationNotAllowed(s"Partition columns should not be specified in the schema: " +
-                              badPartCols.map("\"" + _ + "\"").mkString("[", ",", "]"), ctx)
-        }
-      }
-      val fields = parser.getFields(cols ++ partitionByStructFields)
       val options = new CarbonOption(properties)
-      // validate tblProperties
-      val bucketFields = parser.getBucketFields(tableProperties, fields, options)
+
+      // validate streaming table property
+      validateStreamingProperty(ctx, options)
+
+      // validate partition clause
+      val (partitionByStructFields, partitionFields) =
+        validateParitionFields(ctx, colNames, tableProperties)
+
+      val fields = parser.getFields(colsStructFields ++ partitionByStructFields)
+
+      // validate bucket fields
+      val bucketFields: Option[BucketFields] =
+        parser.getBucketFields(tableProperties, fields, options)
+
+      val tableComment = Option(ctx.STRING()).map(string)
 
       // prepare table model of the collected tokens
-      val tableModel: TableModel = parser.prepareTableModel(ifNotExists,
+      val tableModel: TableModel = parser.prepareTableModel(
+        ifNotExists,
         convertDbNameToLowerCase(name.database),
         name.table.toLowerCase,
         fields,
-        partitionerFields,
+        partitionFields,
         tableProperties,
         bucketFields,
-        false,
+        isAlterFlow = false,
         tableComment)
 
       CarbonCreateTableCommand(tableModel)
@@ -189,8 +179,56 @@ class CarbonSqlAstBuilder(conf: SQLConf) extends SparkSqlAstBuilder(conf) {
 
   private def needToConvertToLowerCase(key: String): Boolean = {
     val noConvertList = Array("LIST_INFO", "RANGE_INFO")
-    !noConvertList.exists(x => x.equalsIgnoreCase(key));
+    !noConvertList.exists(x => x.equalsIgnoreCase(key))
+  }
+
+  private def validateParitionFields(
+      ctx: CreateTableContext,
+      colNames: Seq[String],
+      tableProperties: mutable.Map[String, String]): (Seq[StructField], Seq[PartitionerField]) = {
+    val partitionByStructFields = Option(ctx.partitionColumns).toSeq.flatMap(visitColTypeList)
+    val partitionerFields = partitionByStructFields.map { structField =>
+      PartitionerField(structField.name, Some(structField.dataType.toString), null)
+    }
+    if (partitionerFields.nonEmpty) {
+      if (!CommonUtil.validatePartitionColumns(tableProperties, partitionerFields)) {
+        throw new MalformedCarbonCommandException("Error: Invalid partition definition")
+      }
+      // partition columns should not be part of the schema
+      val badPartCols = partitionerFields.map(_.partitionColumn).toSet.intersect(colNames.toSet)
+      if (badPartCols.nonEmpty) {
+        operationNotAllowed(s"Partition columns should not be specified in the schema: " +
+                            badPartCols.map("\"" + _ + "\"").mkString("[", ",", "]"), ctx)
+      }
+    }
+    (partitionByStructFields, partitionerFields)
   }
 
+  private def validateSchema(
+      ctx: CreateTableContext,
+      name: TableIdentifier): (Seq[StructField], Seq[String]) = {
+    // Validate schema, ensuring whether no duplicate name is used in table definition
+    val cols = Option(ctx.columns).toSeq.flatMap(visitColTypeList)
+    val colNames = cols.map(_.name)
+    if (colNames.length != colNames.distinct.length) {
+      val duplicateColumns = colNames.groupBy(identity).collect {
+        case (x, ys) if ys.length > 1 => "\"" + x + "\""
+      }
+      operationNotAllowed(s"Duplicated column names found in table definition of $name: " +
+                          duplicateColumns.mkString("[", ",", "]"), ctx)
+    }
+    (cols, colNames)
+  }
 
+  private def validateStreamingProperty(
+      ctx: CreateTableContext,
+      carbonOption: CarbonOption): Unit = {
+    try {
+      carbonOption.isStreaming
+    } catch {
+      case _: IllegalArgumentException =>
+        throw new MalformedCarbonCommandException(
+          "Table property 'streaming' should be either 'true' or 'false'")
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4c41f866/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala
index 2c1c6b8..b733d4f 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala
@@ -17,6 +17,7 @@
 
 package org.apache.spark.carbondata
 
+import org.apache.spark.sql.catalyst.parser.ParseException
 import org.apache.spark.sql.test.util.QueryTest
 import org.scalatest.BeforeAndAfterAll
 
@@ -31,7 +32,7 @@ class TestStreamingTableOperation extends QueryTest with BeforeAndAfterAll {
     sql("USE streaming")
     sql(
       """
-        | create table source(
+        | CREATE TABLE source(
         |    c1 string,
         |    c2 int,
         |    c3 string,
@@ -42,6 +43,33 @@ class TestStreamingTableOperation extends QueryTest with BeforeAndAfterAll {
     sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO TABLE source""")
   }
 
+  test("validate streaming property") {
+    sql(
+      """
+        | CREATE TABLE correct(
+        |    c1 string
+        | ) STORED BY 'org.apache.carbondata.format'
+        | TBLPROPERTIES ('streaming' = 'true')
+      """.stripMargin)
+    sql("DROP TABLE correct")
+    sql(
+      """
+        | CREATE TABLE correct(
+        |    c1 string
+        | ) STORED BY 'org.apache.carbondata.format'
+        | TBLPROPERTIES ('streaming' = 'false')
+      """.stripMargin)
+    sql("DROP TABLE correct")
+    intercept[MalformedCarbonCommandException] {
+      sql(
+        """
+          | create table wrong(
+          |    c1 string
+          | ) STORED BY 'org.apache.carbondata.format'
+          | TBLPROPERTIES ('streaming' = 'invalid')
+        """.stripMargin)
+    }
+  }
 
   test("test blocking update and delete operation on streaming table") {
     intercept[MalformedCarbonCommandException] {


[44/49] carbondata git commit: [CARBONDATA-1520] [PreAgg] Support pre-aggregate table load

Posted by ra...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/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 5dbd177..22d15d9 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
@@ -25,11 +25,15 @@ import org.apache.carbondata.core.cache.dictionary.Dictionary;
 import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
 import org.apache.carbondata.core.dictionary.client.DictionaryClient;
 import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
+import org.apache.carbondata.core.metadata.ColumnIdentifier;
 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.RelationIdentifier;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
+import org.apache.carbondata.core.metadata.schema.table.column.ParentColumnTableRelation;
+import org.apache.carbondata.core.util.path.CarbonStorePath;
 import org.apache.carbondata.processing.datatypes.ArrayDataType;
 import org.apache.carbondata.processing.datatypes.GenericDataType;
 import org.apache.carbondata.processing.datatypes.PrimitiveDataType;
@@ -76,12 +80,40 @@ public class FieldEncoderFactory {
             isEmptyBadRecord);
       } else if (dataField.getColumn().hasEncoding(Encoding.DICTIONARY) &&
           !dataField.getColumn().isComplex()) {
-        return new DictionaryFieldConverterImpl(dataField, cache, carbonTableIdentifier, nullFormat,
-            index, client, useOnePass, storePath, localCache, isEmptyBadRecord);
+        DictionaryColumnUniqueIdentifier identifier = null;
+        // if parent column table relation is not null then it's a child table
+        // in case of child table it will use parent table dictionary
+        if (null == dataField.getColumn().getColumnSchema().getParentColumnTableRelations()
+            || dataField.getColumn().getColumnSchema().getParentColumnTableRelations().isEmpty()) {
+          identifier = new DictionaryColumnUniqueIdentifier(carbonTableIdentifier,
+              dataField.getColumn().getColumnIdentifier(), dataField.getColumn().getDataType(),
+              CarbonStorePath.getCarbonTablePath(storePath, carbonTableIdentifier));
+          return new DictionaryFieldConverterImpl(dataField, cache, carbonTableIdentifier,
+              nullFormat, index, client, useOnePass, storePath, localCache, isEmptyBadRecord,
+              identifier);
+        } else {
+          ParentColumnTableRelation parentColumnTableRelation =
+              dataField.getColumn().getColumnSchema().getParentColumnTableRelations().get(0);
+          RelationIdentifier relationIdentifier =
+              parentColumnTableRelation
+                  .getRelationIdentifier();
+          CarbonTableIdentifier parentTableIdentifier =
+              new CarbonTableIdentifier(relationIdentifier.getDatabaseName(),
+                  relationIdentifier.getTableName(), relationIdentifier.getTableId());
+          ColumnIdentifier parentColumnIdentifier =
+              new ColumnIdentifier(parentColumnTableRelation.getColumnId(), null,
+                  dataField.getColumn().getDataType());
+          identifier =
+              new DictionaryColumnUniqueIdentifier(parentTableIdentifier, parentColumnIdentifier,
+                  dataField.getColumn().getDataType(),
+                  CarbonStorePath.getCarbonTablePath(storePath, parentTableIdentifier));
+          return new DictionaryFieldConverterImpl(dataField, cache, parentTableIdentifier,
+              nullFormat, index, null, false, storePath, null, isEmptyBadRecord, identifier);
+        }
       } else if (dataField.getColumn().isComplex()) {
         return new ComplexFieldConverterImpl(
             createComplexType(dataField, cache, carbonTableIdentifier,
-                    client, useOnePass, storePath, localCache), index);
+                client, useOnePass, storePath, localCache), index);
       } else {
         return new NonDictionaryFieldConverterImpl(dataField, nullFormat, index, isEmptyBadRecord);
       }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModel.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModel.java b/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModel.java
index 6a156a6..acd7fed 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModel.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModel.java
@@ -163,6 +163,16 @@ public class CarbonLoadModel implements Serializable {
    */
   private String globalSortPartitions;
 
+  private boolean isAggLoadRequest;
+
+  public boolean isAggLoadRequest() {
+    return isAggLoadRequest;
+  }
+
+  public void setAggLoadRequest(boolean aggLoadRequest) {
+    isAggLoadRequest = aggLoadRequest;
+  }
+
   /**
    * get escape char
    *
@@ -353,6 +363,7 @@ public class CarbonLoadModel implements Serializable {
     copy.sortScope = sortScope;
     copy.batchSortSizeInMb = batchSortSizeInMb;
     copy.badRecordsLocation = badRecordsLocation;
+    copy.isAggLoadRequest = isAggLoadRequest;
     return copy;
   }
 
@@ -399,6 +410,7 @@ public class CarbonLoadModel implements Serializable {
     copy.isEmptyDataBadRecord = isEmptyDataBadRecord;
     copy.sortScope = sortScope;
     copy.batchSortSizeInMb = batchSortSizeInMb;
+    copy.isAggLoadRequest = isAggLoadRequest;
     return copy;
   }
 
@@ -449,6 +461,7 @@ public class CarbonLoadModel implements Serializable {
     copyObj.sortScope = sortScope;
     copyObj.batchSortSizeInMb = batchSortSizeInMb;
     copyObj.badRecordsLocation = badRecordsLocation;
+    copyObj.isAggLoadRequest = isAggLoadRequest;
     return copyObj;
   }
 


[39/49] carbondata git commit: [CARBONDATA-1572][Streaming] Add test case for streaming ingest

Posted by ra...@apache.org.
[CARBONDATA-1572][Streaming] Add test case for streaming ingest

Add test case for streaming ingest using socket stream

This closes #1485


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

Branch: refs/heads/fgdatamap
Commit: 40c31e8049784a53594896cd2e37cd2fb5135fbc
Parents: 80195da
Author: QiangCai <qi...@qq.com>
Authored: Fri Nov 10 19:57:17 2017 +0800
Committer: Jacky Li <ja...@qq.com>
Committed: Mon Nov 13 19:10:21 2017 +0800

----------------------------------------------------------------------
 .../core/constants/CarbonCommonConstants.java   |   5 +
 .../core/statusmanager/LoadMetadataDetails.java |   6 +
 .../store/impl/FileFactoryImplUnitTest.java     |  44 ++
 .../streaming/CarbonStreamOutputFormat.java     |  28 +
 .../streaming/CarbonStreamRecordWriter.java     |  17 +-
 .../streaming/CarbonStreamInputFormatTest.java  |  97 +++
 .../streaming/CarbonStreamOutputFormatTest.java | 119 +++
 .../hadoop/test/util/StoreCreator.java          |  18 +-
 .../src/test/resources/streamSample.csv         |   6 +
 .../org/apache/carbondata/api/CarbonStore.scala |  45 +-
 .../carbondata/spark/rdd/CarbonMergerRDD.scala  |  10 +-
 .../org/apache/spark/sql/CarbonSource.scala     |   5 +
 .../TestStreamingTableOperation.scala           | 740 ++++++++++++++++++-
 .../processing/merger/CarbonDataMergerUtil.java |  27 +-
 .../streaming/segment/StreamSegment.java        |  13 +-
 .../streaming/StreamSinkFactory.scala           |  18 +-
 .../streaming/CarbonAppendableStreamSink.scala  |   8 +-
 17 files changed, 1161 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/40c31e80/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
index 5f63cc1..014478f 100644
--- a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
+++ b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
@@ -810,6 +810,11 @@ public final class CarbonCommonConstants {
   public static final String INVALID_SEGMENT_ID = "-1";
 
   /**
+   * default load time of the segment
+   */
+  public static final long SEGMENT_LOAD_TIME_DEFAULT = -1;
+
+  /**
    * Size of Major Compaction in MBs
    */
   @CarbonProperty

http://git-wip-us.apache.org/repos/asf/carbondata/blob/40c31e80/core/src/main/java/org/apache/carbondata/core/statusmanager/LoadMetadataDetails.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/statusmanager/LoadMetadataDetails.java b/core/src/main/java/org/apache/carbondata/core/statusmanager/LoadMetadataDetails.java
index 7748d17..51a04e4 100644
--- a/core/src/main/java/org/apache/carbondata/core/statusmanager/LoadMetadataDetails.java
+++ b/core/src/main/java/org/apache/carbondata/core/statusmanager/LoadMetadataDetails.java
@@ -84,6 +84,9 @@ public class LoadMetadataDetails implements Serializable {
   }
 
   public long getLoadEndTime() {
+    if (timestamp == null) {
+      return CarbonCommonConstants.SEGMENT_LOAD_TIME_DEFAULT;
+    }
     return convertTimeStampToLong(timestamp);
   }
 
@@ -161,6 +164,9 @@ public class LoadMetadataDetails implements Serializable {
    * @return the startLoadTime
    */
   public long getLoadStartTime() {
+    if (loadStartTime == null) {
+      return CarbonCommonConstants.SEGMENT_LOAD_TIME_DEFAULT;
+    }
     return convertTimeStampToLong(loadStartTime);
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/40c31e80/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/FileFactoryImplUnitTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/FileFactoryImplUnitTest.java b/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/FileFactoryImplUnitTest.java
index 6fd27d3..65590d6 100644
--- a/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/FileFactoryImplUnitTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/FileFactoryImplUnitTest.java
@@ -18,8 +18,10 @@
 package org.apache.carbondata.core.carbon.datastorage.filesystem.store.impl;
 
 import java.io.File;
+import java.io.FileWriter;
 import java.io.IOException;
 
+import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
 
 import mockit.Mock;
@@ -147,5 +149,47 @@ public class FileFactoryImplUnitTest {
     FileFactory.getDataOutputStream(filePath, FileFactory.FileType.VIEWFS);
     assertNotNull(FileFactory.getCarbonFile(filePath, FileFactory.FileType.HDFS));
   }
+
+  @Test public void testTruncateFile() {
+    FileWriter writer = null;
+    String path = null;
+    try {
+      // generate a file
+      path = new File("target/truncatFile").getCanonicalPath();
+      writer = new FileWriter(path);
+      for (int i = 0; i < 4000; i++) {
+        writer.write("test truncate file method");
+      }
+      writer.close();
+      CarbonFile file = FileFactory.getCarbonFile(path);
+      assertTrue(file.getSize() == 100000L);
+
+      // truncate file to 4000 bytes
+      FileFactory.truncateFile(
+          path,
+          FileFactory.getFileType(path),
+          4000);
+      file = FileFactory.getCarbonFile(path);
+      assertEquals(file.getSize(), 4000L);
+    } catch (IOException e) {
+      e.printStackTrace();
+      assertTrue(e.getMessage(), false);
+    } finally {
+      if (writer != null) {
+        try {
+          writer.close();
+        } catch (IOException e) {
+          e.printStackTrace();
+        }
+      }
+      if (path != null) {
+        try {
+          FileFactory.deleteFile(path, FileFactory.getFileType(path));
+        } catch (IOException e) {
+          e.printStackTrace();
+        }
+      }
+    }
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/40c31e80/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamOutputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamOutputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamOutputFormat.java
index 1c21504..47b43c4 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamOutputFormat.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamOutputFormat.java
@@ -51,6 +51,24 @@ public class CarbonStreamOutputFormat extends FileOutputFormat<Void, Object> {
 
   private static final String LOAD_Model = "mapreduce.output.carbon.load.model";
 
+  private static final String SEGMENT_ID = "carbon.segment.id";
+
+  /**
+   * if the byte size of streaming segment reach this value,
+   * the system will create a new stream segment
+   */
+  public static final String HANDOFF_SIZE = "carbon.handoff.size";
+
+  /**
+   * the min handoff size of streaming segment, the unit is byte
+   */
+  public static final long HANDOFF_SIZE_MIN = 1024L * 1024 * 64;
+
+  /**
+   * the default handoff size of streaming segment, the unit is byte
+   */
+  public static final long HANDOFF_SIZE_DEFAULT = 1024L * 1024 * 1024;
+
   @Override public RecordWriter<Void, Object> getRecordWriter(TaskAttemptContext job)
       throws IOException, InterruptedException {
     return new CarbonStreamRecordWriter(job);
@@ -72,4 +90,14 @@ public class CarbonStreamOutputFormat extends FileOutputFormat<Void, Object> {
     }
   }
 
+  public static void setSegmentId(Configuration hadoopConf, String segmentId) throws IOException {
+    if (segmentId != null) {
+      hadoopConf.set(SEGMENT_ID, segmentId);
+    }
+  }
+
+  public static String getSegmentId(Configuration hadoopConf) throws IOException {
+    return hadoopConf.get(SEGMENT_ID);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/40c31e80/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 3c4b55c..7df87e3 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
@@ -75,6 +75,7 @@ public class CarbonStreamRecordWriter extends RecordWriter<Void, Object> {
 
   // parser and converter
   private RowParser rowParser;
+  private BadRecordsLogger badRecordLogger;
   private RowConverter converter;
   private CarbonRow currentRow = new CarbonRow(null);
 
@@ -106,6 +107,8 @@ public class CarbonStreamRecordWriter extends RecordWriter<Void, Object> {
       throw new IOException(
           "CarbonStreamRecordWriter require configuration: mapreduce.output.carbon.load.model");
     }
+    String segmentId = CarbonStreamOutputFormat.getSegmentId(hadoopConf);
+    carbonLoadModel.setSegmentId(segmentId);
     carbonTable = carbonLoadModel.getCarbonDataLoadSchema().getCarbonTable();
     int taskNo = TaskID.forName(hadoopConf.get("mapred.tip.id")).getId();
     carbonLoadModel.setTaskNo("" + taskNo);
@@ -117,7 +120,7 @@ public class CarbonStreamRecordWriter extends RecordWriter<Void, Object> {
 
     CarbonTablePath tablePath =
         CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier());
-    segmentDir = tablePath.getSegmentDir("0", carbonLoadModel.getSegmentId());
+    segmentDir = tablePath.getSegmentDir("0", segmentId);
     fileName = CarbonTablePath.getCarbonDataFileName(0, taskNo, 0, 0, "0");
   }
 
@@ -138,8 +141,7 @@ public class CarbonStreamRecordWriter extends RecordWriter<Void, Object> {
 
     // initialize parser and converter
     rowParser = new RowParserImpl(dataFields, configuration);
-    BadRecordsLogger badRecordLogger =
-        DataConverterProcessorStepImpl.createBadRecordLogger(configuration);
+    badRecordLogger = DataConverterProcessorStepImpl.createBadRecordLogger(configuration);
     converter = new RowConverterImpl(configuration.getDataFields(), configuration, badRecordLogger);
     configuration.setCardinalityFinder(converter);
     converter.initialize();
@@ -285,14 +287,19 @@ public class CarbonStreamRecordWriter extends RecordWriter<Void, Object> {
   @Override public void close(TaskAttemptContext context) throws IOException, InterruptedException {
     try {
       // append remain buffer data
-      if (!hasException) {
+      if (!hasException && !isFirstRow) {
         appendBlockletToDataFile();
         converter.finish();
       }
     } finally {
       // close resource
       CarbonUtil.closeStreams(outputStream);
-      output.close();
+      if (output != null) {
+        output.close();
+      }
+      if (badRecordLogger != null) {
+        badRecordLogger.closeStreams();
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/40c31e80/hadoop/src/test/java/org/apache/carbondata/hadoop/streaming/CarbonStreamInputFormatTest.java
----------------------------------------------------------------------
diff --git a/hadoop/src/test/java/org/apache/carbondata/hadoop/streaming/CarbonStreamInputFormatTest.java b/hadoop/src/test/java/org/apache/carbondata/hadoop/streaming/CarbonStreamInputFormatTest.java
new file mode 100644
index 0000000..9970c50
--- /dev/null
+++ b/hadoop/src/test/java/org/apache/carbondata/hadoop/streaming/CarbonStreamInputFormatTest.java
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.hadoop.streaming;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
+import java.util.UUID;
+
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
+import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
+import org.apache.carbondata.core.statusmanager.FileFormat;
+import org.apache.carbondata.hadoop.CarbonInputSplit;
+import org.apache.carbondata.hadoop.CarbonMultiBlockSplit;
+
+import junit.framework.TestCase;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
+import org.apache.spark.SparkHadoopWriter;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class CarbonStreamInputFormatTest extends TestCase {
+
+  private TaskAttemptID taskAttemptId;
+  private TaskAttemptContext taskAttemptContext;
+  private Configuration hadoopConf;
+  private AbsoluteTableIdentifier identifier;
+  private String storePath;
+
+  @Override protected void setUp() throws Exception {
+    storePath = new File("target/stream_input").getCanonicalPath();
+    String dbName = "default";
+    String tableName = "stream_table_input";
+    identifier = new AbsoluteTableIdentifier(storePath,
+        new CarbonTableIdentifier(dbName, tableName, UUID.randomUUID().toString()));
+
+    JobID jobId = SparkHadoopWriter.createJobID(new Date(), 0);
+    TaskID taskId = new TaskID(jobId, TaskType.MAP, 0);
+    taskAttemptId = new TaskAttemptID(taskId, 0);
+
+    hadoopConf = new Configuration();
+    taskAttemptContext = new TaskAttemptContextImpl(hadoopConf, taskAttemptId);
+  }
+
+  private InputSplit buildInputSplit() throws IOException {
+    CarbonInputSplit carbonInputSplit = new CarbonInputSplit();
+    List<CarbonInputSplit> splitList = new ArrayList<>();
+    splitList.add(carbonInputSplit);
+    return new CarbonMultiBlockSplit(identifier, splitList, new String[] { "localhost" },
+        FileFormat.rowformat);
+  }
+
+  @Test public void testCreateRecordReader() {
+    try {
+      InputSplit inputSplit = buildInputSplit();
+      CarbonStreamInputFormat inputFormat = new CarbonStreamInputFormat();
+      RecordReader recordReader = inputFormat.createRecordReader(inputSplit, taskAttemptContext);
+      Assert.assertNotNull("Failed to create record reader", recordReader);
+    } catch (Exception e) {
+      e.printStackTrace();
+      Assert.assertTrue(e.getMessage(), false);
+    }
+  }
+
+  @Override protected void tearDown() throws Exception {
+    super.tearDown();
+    if (storePath != null) {
+      FileFactory.deleteAllFilesOfDir(new File(storePath));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/40c31e80/hadoop/src/test/java/org/apache/carbondata/hadoop/streaming/CarbonStreamOutputFormatTest.java
----------------------------------------------------------------------
diff --git a/hadoop/src/test/java/org/apache/carbondata/hadoop/streaming/CarbonStreamOutputFormatTest.java b/hadoop/src/test/java/org/apache/carbondata/hadoop/streaming/CarbonStreamOutputFormatTest.java
new file mode 100644
index 0000000..daa2540
--- /dev/null
+++ b/hadoop/src/test/java/org/apache/carbondata/hadoop/streaming/CarbonStreamOutputFormatTest.java
@@ -0,0 +1,119 @@
+/*
+ * 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.hadoop.streaming;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Date;
+import java.util.UUID;
+
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
+import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+import org.apache.carbondata.hadoop.test.util.StoreCreator;
+import org.apache.carbondata.processing.loading.model.CarbonLoadModel;
+
+import junit.framework.TestCase;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
+import org.apache.spark.SparkHadoopWriter;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class CarbonStreamOutputFormatTest extends TestCase {
+
+  private Configuration hadoopConf;
+  private TaskAttemptID taskAttemptId;
+  private CarbonLoadModel carbonLoadModel;
+  private String storePath;
+
+  @Override protected void setUp() throws Exception {
+    super.setUp();
+    JobID jobId = SparkHadoopWriter.createJobID(new Date(), 0);
+    TaskID taskId = new TaskID(jobId, TaskType.MAP, 0);
+    taskAttemptId = new TaskAttemptID(taskId, 0);
+
+    hadoopConf = new Configuration();
+    hadoopConf.set("mapred.job.id", jobId.toString());
+    hadoopConf.set("mapred.tip.id", taskAttemptId.getTaskID().toString());
+    hadoopConf.set("mapred.task.id", taskAttemptId.toString());
+    hadoopConf.setBoolean("mapred.task.is.map", true);
+    hadoopConf.setInt("mapred.task.partition", 0);
+
+    storePath = new File("target/stream_output").getCanonicalPath();
+    String dbName = "default";
+    String tableName = "stream_table_output";
+    AbsoluteTableIdentifier identifier = new AbsoluteTableIdentifier(storePath,
+        new CarbonTableIdentifier(dbName, tableName, UUID.randomUUID().toString()));
+
+    CarbonTable table = StoreCreator.createTable(identifier);
+
+    String factFilePath = new File("../hadoop/src/test/resources/data.csv").getCanonicalPath();
+    carbonLoadModel = StoreCreator.buildCarbonLoadModel(table, factFilePath, identifier);
+  }
+
+  @Test public void testSetCarbonLoadModel() {
+    try {
+      CarbonStreamOutputFormat.setCarbonLoadModel(hadoopConf, carbonLoadModel);
+    } catch (IOException e) {
+      Assert.assertTrue("Failed to config CarbonLoadModel for CarbonStreamOutputFromat", false);
+    }
+  }
+
+  @Test public void testGetCarbonLoadModel() {
+    try {
+      CarbonStreamOutputFormat.setCarbonLoadModel(hadoopConf, carbonLoadModel);
+      CarbonLoadModel model = CarbonStreamOutputFormat.getCarbonLoadModel(hadoopConf);
+
+      Assert.assertNotNull("Failed to get CarbonLoadModel", model);
+      Assert.assertTrue("CarbonLoadModel should be same with previous",
+          carbonLoadModel.getFactTimeStamp() == model.getFactTimeStamp());
+
+    } catch (IOException e) {
+      Assert.assertTrue("Failed to get CarbonLoadModel for CarbonStreamOutputFromat", false);
+    }
+  }
+
+  @Test public void testGetRecordWriter() {
+    CarbonStreamOutputFormat outputFormat = new CarbonStreamOutputFormat();
+    try {
+      CarbonStreamOutputFormat.setCarbonLoadModel(hadoopConf, carbonLoadModel);
+      TaskAttemptContext taskAttemptContext =
+          new TaskAttemptContextImpl(hadoopConf, taskAttemptId);
+      RecordWriter recordWriter = outputFormat.getRecordWriter(taskAttemptContext);
+      Assert.assertNotNull("Failed to get CarbonStreamRecordWriter", recordWriter);
+    } catch (Exception e) {
+      e.printStackTrace();
+      Assert.assertTrue(e.getMessage(), false);
+    }
+  }
+
+  @Override protected void tearDown() throws Exception {
+    super.tearDown();
+    if (storePath != null) {
+      FileFactory.deleteAllFilesOfDir(new File(storePath));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/40c31e80/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/StoreCreator.java
----------------------------------------------------------------------
diff --git a/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/StoreCreator.java b/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/StoreCreator.java
index 29d8d03..dd48cd9 100644
--- a/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/StoreCreator.java
+++ b/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/StoreCreator.java
@@ -49,7 +49,6 @@ import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.metadata.ColumnIdentifier;
 import org.apache.carbondata.core.metadata.converter.SchemaConverter;
 import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl;
-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.SchemaEvolution;
@@ -116,7 +115,8 @@ public class StoreCreator {
     return absoluteTableIdentifier;
   }
 
-  public static CarbonLoadModel buildCarbonLoadModel(CarbonTable table, String factFilePath) {
+  public static CarbonLoadModel buildCarbonLoadModel(CarbonTable table, String factFilePath,
+      AbsoluteTableIdentifier absoluteTableIdentifier) {
     CarbonDataLoadSchema schema = new CarbonDataLoadSchema(table);
     CarbonLoadModel loadModel = new CarbonLoadModel();
     loadModel.setCarbonDataLoadSchema(schema);
@@ -159,28 +159,28 @@ public class StoreCreator {
    * Create store without any restructure
    */
   public static void createCarbonStore() {
-
     try {
-
-      String factFilePath = new File("../hadoop/src/test/resources/data.csv").getCanonicalPath();
+      String factFilePath =
+          new File("../hadoop/src/test/resources/data.csv").getCanonicalPath();
       File storeDir = new File(absoluteTableIdentifier.getStorePath());
       CarbonUtil.deleteFoldersAndFiles(storeDir);
       CarbonProperties.getInstance().addProperty(CarbonCommonConstants.STORE_LOCATION_HDFS,
           absoluteTableIdentifier.getStorePath());
 
-      CarbonTable table = createTable();
+      CarbonTable table = createTable(absoluteTableIdentifier);
       writeDictionary(factFilePath, table);
-      CarbonLoadModel loadModel = buildCarbonLoadModel(table, factFilePath);
+      CarbonLoadModel loadModel =
+          buildCarbonLoadModel(table, factFilePath, absoluteTableIdentifier);
 
       executeGraph(loadModel, absoluteTableIdentifier.getStorePath());
 
     } catch (Exception e) {
       e.printStackTrace();
     }
-
   }
 
-  public static CarbonTable createTable() throws IOException {
+  public static CarbonTable createTable(
+      AbsoluteTableIdentifier absoluteTableIdentifier) throws IOException {
     TableInfo tableInfo = new TableInfo();
     tableInfo.setStorePath(absoluteTableIdentifier.getStorePath());
     tableInfo.setDatabaseName(absoluteTableIdentifier.getCarbonTableIdentifier().getDatabaseName());

http://git-wip-us.apache.org/repos/asf/carbondata/blob/40c31e80/integration/spark-common-test/src/test/resources/streamSample.csv
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/resources/streamSample.csv b/integration/spark-common-test/src/test/resources/streamSample.csv
new file mode 100644
index 0000000..590ea90
--- /dev/null
+++ b/integration/spark-common-test/src/test/resources/streamSample.csv
@@ -0,0 +1,6 @@
+id,name,city,salary,file
+100000001,batch_1,city_1,0.1,school_1:school_11$20
+100000002,batch_2,city_2,0.2,school_2:school_22$30
+100000003,batch_3,city_3,0.3,school_3:school_33$40
+100000004,batch_4,city_4,0.4,school_4:school_44$50
+100000005,batch_5,city_5,0.5,school_5:school_55$60

http://git-wip-us.apache.org/repos/asf/carbondata/blob/40c31e80/integration/spark-common/src/main/scala/org/apache/carbondata/api/CarbonStore.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/api/CarbonStore.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/api/CarbonStore.scala
index a95bc01..054f778 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/api/CarbonStore.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/api/CarbonStore.scala
@@ -64,21 +64,36 @@ object CarbonStore {
       }
 
       loadMetadataDetailsSortedArray
-          .filter(_.getVisibility.equalsIgnoreCase("true"))
-          .map { load =>
-            val mergedTo = if (load.getMergedLoadName != null) {
-         load.getMergedLoadName
-       } else {
-         ""
-       }
-            Row(
-              load.getLoadName,
-              load.getLoadStatus,
-              new java.sql.Timestamp(load.getLoadStartTime),
-              new java.sql.Timestamp(load.getLoadEndTime),
-              mergedTo
-            )
-          }.toSeq
+        .filter(_.getVisibility.equalsIgnoreCase("true"))
+        .map { load =>
+          val mergedTo = if (load.getMergedLoadName != null) {
+            load.getMergedLoadName
+          } else {
+            ""
+          }
+
+          val startTime =
+            if (load.getLoadStartTime == CarbonCommonConstants.SEGMENT_LOAD_TIME_DEFAULT) {
+              null
+            } else {
+              new java.sql.Timestamp(load.getLoadStartTime)
+            }
+
+          val endTime =
+            if (load.getLoadEndTime == CarbonCommonConstants.SEGMENT_LOAD_TIME_DEFAULT) {
+              null
+            } else {
+              new java.sql.Timestamp(load.getLoadEndTime)
+            }
+
+          Row(
+            load.getLoadName,
+            load.getLoadStatus,
+            startTime,
+            endTime,
+            mergedTo
+          )
+        }.toSeq
     } else {
       Seq.empty
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/40c31e80/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
index 1f88f25..c6edc2a 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
@@ -40,7 +40,7 @@ import org.apache.carbondata.core.metadata.blocklet.DataFileFooter
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema
 import org.apache.carbondata.core.mutate.UpdateVO
 import org.apache.carbondata.core.scan.result.iterator.RawResultIterator
-import org.apache.carbondata.core.statusmanager.SegmentUpdateStatusManager
+import org.apache.carbondata.core.statusmanager.{FileFormat, SegmentUpdateStatusManager}
 import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil, DataTypeUtil}
 import org.apache.carbondata.hadoop.{CarbonInputSplit, CarbonMultiBlockSplit}
 import org.apache.carbondata.hadoop.api.CarbonTableInputFormat
@@ -307,7 +307,8 @@ class CarbonMergerRDD[K, V](
 
       // keep on assigning till last one is reached.
       if (null != splits && splits.size > 0) {
-        splitsOfLastSegment = splits.asScala.map(_.asInstanceOf[CarbonInputSplit]).toList.asJava
+        splitsOfLastSegment = splits.asScala.map(_.asInstanceOf[CarbonInputSplit])
+          .filter { split => FileFormat.carbondata.equals(split.getFileFormat) }.toList.asJava
       }
 
       carbonInputSplits ++:= splits.asScala.map(_.asInstanceOf[CarbonInputSplit]).filter(entry => {
@@ -316,9 +317,10 @@ class CarbonMergerRDD[K, V](
           entry.getLocations, entry.getLength, entry.getVersion,
           updateStatusManager.getDeleteDeltaFilePath(entry.getPath.toString)
         )
-        ((!updated) || ((updated) && (!CarbonUtil
+        (((!updated) || ((updated) && (!CarbonUtil
           .isInvalidTableBlock(blockInfo.getSegmentId, blockInfo.getFilePath,
-            updateDetails, updateStatusManager))))
+            updateDetails, updateStatusManager)))) &&
+         FileFormat.carbondata.equals(entry.getFileFormat))
       })
     }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/40c31e80/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSource.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSource.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSource.scala
index 4580f22..de01c8d 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSource.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSource.scala
@@ -230,6 +230,11 @@ class CarbonSource extends CreatableRelationProvider with RelationProvider
         CarbonEnv.getInstance(sparkSession).carbonMetastore.
           createCarbonRelation(parameters, identifier, sparkSession).tableMeta.carbonTable
 
+      if (!carbonTable.isStreamingTable) {
+        throw new CarbonStreamException(s"Table ${carbonTable.getDatabaseName}." +
+                                        s"${carbonTable.getFactTableName} is not a streaming table")
+      }
+
       // create sink
       StreamSinkFactory.createStreamTableSink(
         sqlContext.sparkSession,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/40c31e80/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala
index d5f9426..ffa977b 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala
@@ -17,15 +17,30 @@
 
 package org.apache.spark.carbondata
 
-import org.apache.spark.sql.catalyst.parser.ParseException
+import java.io.{File, PrintWriter}
+import java.net.ServerSocket
+import java.util.concurrent.Executors
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.hive.CarbonRelation
+import org.apache.spark.sql.{CarbonEnv, Row, SaveMode, SparkSession}
+import org.apache.spark.sql.streaming.{ProcessingTime, StreamingQuery}
 import org.apache.spark.sql.test.util.QueryTest
+import org.apache.spark.sql.types.StructType
 import org.scalatest.BeforeAndAfterAll
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
+import org.apache.carbondata.hadoop.streaming.CarbonStreamOutputFormat
 import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 
 class TestStreamingTableOperation extends QueryTest with BeforeAndAfterAll {
+
+  private val spark = sqlContext.sparkSession
+  private val dataFilePath = s"$resourcesPath/streamSample.csv"
+
   override def beforeAll {
     sql("DROP DATABASE IF EXISTS streaming CASCADE")
     sql("CREATE DATABASE streaming")
@@ -41,6 +56,52 @@ class TestStreamingTableOperation extends QueryTest with BeforeAndAfterAll {
         | TBLPROPERTIES ('streaming' = 'true')
       """.stripMargin)
     sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO TABLE source""")
+
+    dropTable()
+
+    // 1. normal table not support streaming ingest
+    createTable(tableName = "batch_table", streaming = false, withBatchLoad = true)
+
+    // 2. streaming table with different input source
+    // socket source
+    createTable(tableName = "stream_table_socket", streaming = true, withBatchLoad = true)
+    // file source
+    createTable(tableName = "stream_table_file", streaming = true, withBatchLoad = true)
+
+    // 3. streaming table with bad records
+    createTable(tableName = "bad_record_force", streaming = true, withBatchLoad = true)
+    createTable(tableName = "bad_record_fail", streaming = true, withBatchLoad = true)
+
+    // 4. streaming frequency check
+    createTable(tableName = "stream_table_1s", streaming = true, withBatchLoad = true)
+    createTable(tableName = "stream_table_10s", streaming = true, withBatchLoad = true)
+
+    // 5. streaming table execute batch loading
+    createTable(tableName = "stream_table_batch", streaming = true, withBatchLoad = false)
+
+    // 6. detail query
+    // full scan
+    createTable(tableName = "stream_table_scan", streaming = true, withBatchLoad = true)
+    createTableWithComplexType(
+      tableName = "stream_table_scan_complex", streaming = true, withBatchLoad = true)
+    // filter scan
+    createTable(tableName = "stream_table_filter", streaming = true, withBatchLoad = true)
+    createTableWithComplexType(
+      tableName = "stream_table_filter_complex", streaming = true, withBatchLoad = true)
+
+    // 7. aggregate query
+    createTable(tableName = "stream_table_agg", streaming = true, withBatchLoad = true)
+    createTableWithComplexType(
+      tableName = "stream_table_agg_complex", streaming = true, withBatchLoad = true)
+
+    // 8. compaction
+    createTable(tableName = "stream_table_compact", streaming = true, withBatchLoad = true)
+
+    // 9. create new stream segment if current stream segment is full
+    createTable(tableName = "stream_table_new", streaming = true, withBatchLoad = true)
+
+    // 10. fault tolerant
+    createTable(tableName = "stream_table_tolerant", streaming = true, withBatchLoad = true)
   }
 
   test("validate streaming property") {
@@ -96,7 +157,682 @@ class TestStreamingTableOperation extends QueryTest with BeforeAndAfterAll {
   }
 
   override def afterAll {
+    dropTable()
     sql("USE default")
     sql("DROP DATABASE IF EXISTS streaming CASCADE")
   }
+
+  def dropTable(): Unit = {
+    sql("drop table if exists streaming.batch_table")
+    sql("drop table if exists streaming.stream_table_socket")
+    sql("drop table if exists streaming.stream_table_file")
+    sql("drop table if exists streaming.bad_record_force")
+    sql("drop table if exists streaming.bad_record_fail")
+    sql("drop table if exists streaming.stream_table_1s")
+    sql("drop table if exists streaming.stream_table_10s")
+    sql("drop table if exists streaming.stream_table_batch")
+    sql("drop table if exists streaming.stream_table_scan")
+    sql("drop table if exists streaming.stream_table_scan_complex")
+    sql("drop table if exists streaming.stream_table_filter")
+    sql("drop table if exists streaming.stream_table_filter_complex")
+    sql("drop table if exists streaming.stream_table_agg")
+    sql("drop table if exists streaming.stream_table_agg_complex")
+    sql("drop table if exists streaming.stream_table_compact")
+    sql("drop table if exists streaming.stream_table_new")
+    sql("drop table if exists streaming.stream_table_tolerant")
+  }
+
+  // normal table not support streaming ingest
+  test("normal table not support streaming ingest") {
+    val identifier = new TableIdentifier("batch_table", Option("streaming"))
+    val carbonTable = CarbonEnv.getInstance(spark).carbonMetastore.lookupRelation(identifier)(spark)
+      .asInstanceOf[CarbonRelation].metaData.carbonTable
+    val tablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
+    var server: ServerSocket = null
+    try {
+      server = new ServerSocket(7071)
+      val thread1 = createWriteSocketThread(server, 2, 10, 1)
+      thread1.start()
+      // use thread pool to catch the exception of sink thread
+      val pool = Executors.newSingleThreadExecutor()
+      val thread2 = createSocketStreamingThread(spark, tablePath)
+      val future = pool.submit(thread2)
+      Thread.sleep(1000)
+      thread1.interrupt()
+      try {
+        future.get()
+        assert(false)
+      } catch {
+        case ex =>
+          assert(ex.getMessage.contains("is not a streaming table"))
+      }
+    } finally {
+      if (server != null) {
+        server.close()
+      }
+    }
+  }
+
+  // input source: socket
+  test("streaming ingest from socket source") {
+    executeStreamingIngest(
+      tableName = "stream_table_socket",
+      batchNums = 2,
+      rowNumsEachBatch = 10,
+      intervalOfSource = 1,
+      intervalOfIngest = 1,
+      continueSeconds = 10,
+      generateBadRecords = false,
+      badRecordAction = "force"
+    )
+
+    checkAnswer(
+      sql("select count(*) from streaming.stream_table_socket"),
+      Seq(Row(25))
+    )
+  }
+
+  // input source: file
+  test("streaming ingest from file source") {
+    val identifier = new TableIdentifier("stream_table_file", Option("streaming"))
+    val carbonTable = CarbonEnv.getInstance(spark).carbonMetastore.lookupRelation(identifier)(spark)
+      .asInstanceOf[CarbonRelation].metaData.carbonTable
+    val tablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
+    val csvDataDir = new File("target/csvdata").getCanonicalPath
+    // streaming ingest 10 rows
+    generateCSVDataFile(spark, idStart = 10, rowNums = 10, csvDataDir)
+    val thread = createFileStreamingThread(spark, tablePath, csvDataDir, intervalSecond = 1)
+    thread.start()
+    Thread.sleep(2000)
+    generateCSVDataFile(spark, idStart = 30, rowNums = 10, csvDataDir)
+    Thread.sleep(10000)
+    thread.interrupt()
+    checkAnswer(
+      sql("select count(*) from streaming.stream_table_file"),
+      Seq(Row(25))
+    )
+  }
+
+  // bad records
+  test("streaming table with bad records action: force") {
+    executeStreamingIngest(
+      tableName = "bad_record_force",
+      batchNums = 2,
+      rowNumsEachBatch = 10,
+      intervalOfSource = 1,
+      intervalOfIngest = 1,
+      continueSeconds = 10,
+      generateBadRecords = true,
+      badRecordAction = "force"
+    )
+    checkAnswer(
+      sql("select count(*) from streaming.stream_table_socket"),
+      Seq(Row(25))
+    )
+
+  }
+
+  test("streaming table with bad records action: fail") {
+    executeStreamingIngest(
+      tableName = "bad_record_fail",
+      batchNums = 2,
+      rowNumsEachBatch = 10,
+      intervalOfSource = 1,
+      intervalOfIngest = 1,
+      continueSeconds = 10,
+      generateBadRecords = true,
+      badRecordAction = "fail"
+    )
+    val result = sql("select count(*) from streaming.bad_record_fail").collect()
+    assert(result(0).getLong(0) < 25)
+  }
+
+  // ingest with different interval
+  test("1 row per 1 second interval") {
+    executeStreamingIngest(
+      tableName = "stream_table_1s",
+      batchNums = 20,
+      rowNumsEachBatch = 1,
+      intervalOfSource = 1,
+      intervalOfIngest = 1,
+      continueSeconds = 20,
+      generateBadRecords = false,
+      badRecordAction = "force"
+    )
+    val result = sql("select count(*) from streaming.stream_table_1s").collect()
+    // 20 seconds can't ingest all data, exists data delay
+    assert(result(0).getLong(0) > 5 + 10)
+  }
+
+  test("10000 row per 10 seconds interval") {
+    executeStreamingIngest(
+      tableName = "stream_table_10s",
+      batchNums = 5,
+      rowNumsEachBatch = 10000,
+      intervalOfSource = 5,
+      intervalOfIngest = 10,
+      continueSeconds = 50,
+      generateBadRecords = false,
+      badRecordAction = "force"
+    )
+    checkAnswer(
+      sql("select count(*) from streaming.stream_table_10s"),
+      Seq(Row(5 + 10000*5)))
+  }
+
+  // batch loading on streaming table
+  test("streaming table execute batch loading") {
+    executeStreamingIngest(
+      tableName = "stream_table_batch",
+      batchNums = 5,
+      rowNumsEachBatch = 100,
+      intervalOfSource = 3,
+      intervalOfIngest = 5,
+      continueSeconds = 20,
+      generateBadRecords = false,
+      badRecordAction = "force"
+    )
+    checkAnswer(
+      sql("select count(*) from streaming.stream_table_batch"),
+      Seq(Row(100*5)))
+
+    executeBatchLoad("stream_table_batch")
+
+    checkAnswer(
+      sql("select count(*) from streaming.stream_table_batch"),
+      Seq(Row(100*5 + 5)))
+  }
+
+  // detail query on batch and stream segment
+  test("non-filter query on stream table with dictionary, sort_columns") {
+    executeStreamingIngest(
+      tableName = "stream_table_scan",
+      batchNums = 5,
+      rowNumsEachBatch = 10,
+      intervalOfSource = 2,
+      intervalOfIngest = 4,
+      continueSeconds = 20,
+      generateBadRecords = false,
+      badRecordAction = "force"
+    )
+
+    val result = sql("select * from streaming.stream_table_scan order by id").collect()
+    assert(result != null)
+    assert(result.length == 55)
+    // check one row of streaming data
+    assert(result(0).getInt(0) == 1)
+    assert(result(0).getString(1) == "name_1")
+    // check one row of batch loading
+    assert(result(50).getInt(0) == 100000001)
+    assert(result(50).getString(1) == "batch_1")
+  }
+
+  test("non-filter query on stream table with dictionary, sort_columns and complex column") {
+    executeStreamingIngest(
+      tableName = "stream_table_scan_complex",
+      batchNums = 5,
+      rowNumsEachBatch = 10,
+      intervalOfSource = 2,
+      intervalOfIngest = 4,
+      continueSeconds = 20,
+      generateBadRecords = false,
+      badRecordAction = "force"
+    )
+
+    val result = sql("select * from streaming.stream_table_scan_complex order by id").collect()
+    assert(result != null)
+    assert(result.length == 55)
+    // check one row of streaming data
+    assert(result(0).getInt(0) == 1)
+    assert(result(0).getString(1) == "name_1")
+    assert(result(0).getStruct(4).getInt(1) == 1)
+    // check one row of batch loading
+    assert(result(50).getInt(0) == 100000001)
+    assert(result(50).getString(1) == "batch_1")
+    assert(result(50).getStruct(4).getInt(1) == 20)
+  }
+
+  test("filter query on stream table with dictionary, sort_columns") {
+    executeStreamingIngest(
+      tableName = "stream_table_filter",
+      batchNums = 5,
+      rowNumsEachBatch = 10,
+      intervalOfSource = 2,
+      intervalOfIngest = 4,
+      continueSeconds = 20,
+      generateBadRecords = true,
+      badRecordAction = "force"
+    )
+
+    checkAnswer(
+      sql("select * from stream_table_filter where id = 1"),
+      Seq(Row(1, "name_1", "city_1", 10000.0)))
+
+    checkAnswer(
+      sql("select * from stream_table_filter where name = 'name_2'"),
+      Seq(Row(2, "name_2", "", 20000.0)))
+
+    checkAnswer(
+      sql("select * from stream_table_filter where city = 'city_1'"),
+      Seq(Row(1, "name_1", "city_1", 10000.0),
+        Row(100000001, "batch_1", "city_1", 0.1)))
+
+    checkAnswer(
+      sql("select * from stream_table_filter where id > 49 and id < 100000002"),
+      Seq(Row(50, "name_50", "city_50", 500000.0),
+        Row(100000001, "batch_1", "city_1", 0.1)))
+
+    checkAnswer(
+      sql("select * from stream_table_filter where id is null"),
+      Seq(Row(null, "name_6", "city_6", 60000.0)))
+
+    checkAnswer(
+      sql("select * from stream_table_filter where city = ''"),
+      Seq(Row(2, "name_2", "", 20000.0)))
+
+  }
+
+  test("filter query on stream table with dictionary, sort_columns and complex column") {
+    executeStreamingIngest(
+      tableName = "stream_table_filter_complex",
+      batchNums = 5,
+      rowNumsEachBatch = 10,
+      intervalOfSource = 2,
+      intervalOfIngest = 4,
+      continueSeconds = 20,
+      generateBadRecords = true,
+      badRecordAction = "force"
+    )
+
+    checkAnswer(
+      sql("select * from stream_table_filter_complex where id = 1"),
+      Seq(Row(1, "name_1", "city_1", 10000.0, Row(wrap(Array("school_1", "school_11")), 1))))
+
+    checkAnswer(
+      sql("select * from stream_table_filter_complex where name = 'name_2'"),
+      Seq(Row(2, "name_2", "", 20000.0, Row(wrap(Array("school_2", "school_22")), 2))))
+
+    checkAnswer(
+      sql("select * from stream_table_filter_complex where file.age = 3"),
+      Seq(Row(3, "name_3", "city_3", 30000.0, Row(wrap(Array("school_3", "school_33")), 3))))
+
+    checkAnswer(
+      sql("select * from stream_table_filter_complex where city = 'city_1'"),
+      Seq(Row(1, "name_1", "city_1", 10000.0, Row(wrap(Array("school_1", "school_11")), 1)),
+        Row(100000001, "batch_1", "city_1", 0.1, Row(wrap(Array("school_1", "school_11")), 20))))
+
+    checkAnswer(
+      sql("select * from stream_table_filter_complex where id > 49 and id < 100000002"),
+      Seq(Row(50, "name_50", "city_50", 500000.0, Row(wrap(Array("school_50", "school_5050")), 50)),
+        Row(100000001, "batch_1", "city_1", 0.1, Row(wrap(Array("school_1", "school_11")), 20))))
+
+    checkAnswer(
+      sql("select * from stream_table_filter_complex where id is null"),
+      Seq(Row(null, "name_6", "city_6", 60000.0, Row(wrap(Array("school_6", "school_66")), 6))))
+
+    checkAnswer(
+      sql("select * from stream_table_filter_complex where city = ''"),
+      Seq(Row(2, "name_2", "", 20000.0, Row(wrap(Array("school_2", "school_22")), 2))))
+
+  }
+
+  // aggregation
+  test("aggregation query") {
+    executeStreamingIngest(
+      tableName = "stream_table_agg",
+      batchNums = 5,
+      rowNumsEachBatch = 10,
+      intervalOfSource = 2,
+      intervalOfIngest = 4,
+      continueSeconds = 20,
+      generateBadRecords = true,
+      badRecordAction = "force"
+    )
+
+    checkAnswer(
+      sql("select count(*), max(id), min(name), cast(avg(id) as integer), sum(id) " +
+          "from stream_table_agg where id >= 2 and id <= 100000004"),
+      Seq(Row(52, 100000004, "batch_1", 7692332, 400001278)))
+
+    checkAnswer(
+      sql("select city, count(id), sum(id), cast(avg(id) as integer), " +
+          "max(salary), min(salary) " +
+          "from stream_table_agg " +
+          "where name in ('batch_1', 'batch_2', 'batch_3', 'name_1', 'name_2', 'name_3') " +
+          "and city <> '' " +
+          "group by city " +
+          "order by city"),
+      Seq(Row("city_1", 2, 100000002, 50000001, 10000.0, 0.1),
+        Row("city_2", 1, 100000002, 100000002, 0.2, 0.2),
+        Row("city_3", 2, 100000006, 50000003, 30000.0, 0.3)))
+  }
+
+  test("aggregation query with complex") {
+    executeStreamingIngest(
+      tableName = "stream_table_agg_complex",
+      batchNums = 5,
+      rowNumsEachBatch = 10,
+      intervalOfSource = 2,
+      intervalOfIngest = 4,
+      continueSeconds = 20,
+      generateBadRecords = true,
+      badRecordAction = "force"
+    )
+
+    checkAnswer(
+      sql("select count(*), max(id), min(name), cast(avg(file.age) as integer), sum(file.age) " +
+          "from stream_table_agg_complex where id >= 2 and id <= 100000004"),
+      Seq(Row(52, 100000004, "batch_1", 27, 1408)))
+
+    checkAnswer(
+      sql("select city, count(id), sum(id), cast(avg(file.age) as integer), " +
+          "max(salary), min(salary) " +
+          "from stream_table_agg_complex " +
+          "where name in ('batch_1', 'batch_2', 'batch_3', 'name_1', 'name_2', 'name_3') " +
+          "and city <> '' " +
+          "group by city " +
+          "order by city"),
+      Seq(Row("city_1", 2, 100000002, 10, 10000.0, 0.1),
+        Row("city_2", 1, 100000002, 30, 0.2, 0.2),
+        Row("city_3", 2, 100000006, 21, 30000.0, 0.3)))
+  }
+
+  // compaction
+  test("test compaction on stream table") {
+    executeStreamingIngest(
+      tableName = "stream_table_compact",
+      batchNums = 5,
+      rowNumsEachBatch = 10,
+      intervalOfSource = 2,
+      intervalOfIngest = 4,
+      continueSeconds = 20,
+      generateBadRecords = false,
+      badRecordAction = "force"
+    )
+    for (_ <- 0 to 3) {
+      executeBatchLoad("stream_table_compact")
+    }
+
+    sql("alter table streaming.stream_table_compact compact 'minor'")
+
+    val result = sql("show segments for table streaming.stream_table_compact").collect()
+    result.foreach { row =>
+      if (row.getString(0).equals("1")) {
+        assert(row.getString(1).equals(CarbonCommonConstants.STORE_LOADSTATUS_STREAMING))
+      }
+    }
+  }
+
+  // stream segment max size
+  test("create new stream segment if current stream segment is full") {
+    executeStreamingIngest(
+      tableName = "stream_table_new",
+      batchNums = 6,
+      rowNumsEachBatch = 10000,
+      intervalOfSource = 5,
+      intervalOfIngest = 10,
+      continueSeconds = 40,
+      generateBadRecords = false,
+      badRecordAction = "force",
+      handoffSize = 1024L * 200
+    )
+    sql("show segments for table streaming.stream_table_new").show(100, false)
+
+    assert(sql("show segments for table streaming.stream_table_new").count() == 4)
+
+    checkAnswer(
+      sql("select count(*) from streaming.stream_table_new"),
+      Seq(Row(5 + 10000 * 6))
+    )
+  }
+
+  def createWriteSocketThread(
+      serverSocket: ServerSocket,
+      writeNums: Int,
+      rowNums: Int,
+      intervalSecond: Int,
+      badRecords: Boolean = false): Thread = {
+    new Thread() {
+      override def run(): Unit = {
+        // wait for client to connection request and accept
+        val clientSocket = serverSocket.accept()
+        val socketWriter = new PrintWriter(clientSocket.getOutputStream())
+        var index = 0
+        for (_ <- 1 to writeNums) {
+          // write 5 records per iteration
+          for (_ <- 1 to rowNums) {
+            index = index + 1
+            if (badRecords) {
+              if (index == 2) {
+                // null value
+                socketWriter.println(index.toString + ",name_" + index
+                                     + ",," + (10000.00 * index).toString +
+                                     ",school_" + index + ":school_" + index + index + "$" + index)
+              } else if (index == 6) {
+                // illegal number
+                socketWriter.println(index.toString + "abc,name_" + index
+                                     + ",city_" + index + "," + (10000.00 * index).toString +
+                                     ",school_" + index + ":school_" + index + index + "$" + index)
+              } else {
+                socketWriter.println(index.toString + ",name_" + index
+                                     + ",city_" + index + "," + (10000.00 * index).toString +
+                                     ",school_" + index + ":school_" + index + index + "$" + index)
+              }
+            } else {
+              socketWriter.println(index.toString + ",name_" + index
+                                   + ",city_" + index + "," + (10000.00 * index).toString +
+                                   ",school_" + index + ":school_" + index + index + "$" + index)
+            }
+          }
+          socketWriter.flush()
+          Thread.sleep(1000 * intervalSecond)
+        }
+        socketWriter.close()
+      }
+    }
+  }
+
+  def createSocketStreamingThread(
+      spark: SparkSession,
+      tablePath: CarbonTablePath,
+      badRecordAction: String = "force",
+      intervalSecond: Int = 2,
+      handoffSize: Long = CarbonStreamOutputFormat.HANDOFF_SIZE_DEFAULT): Thread = {
+    new Thread() {
+      override def run(): Unit = {
+        var qry: StreamingQuery = null
+        try {
+          val readSocketDF = spark.readStream
+            .format("socket")
+            .option("host", "localhost")
+            .option("port", 7071)
+            .load()
+
+          // Write data from socket stream to carbondata file
+          qry = readSocketDF.writeStream
+            .format("carbondata")
+            .trigger(ProcessingTime(s"$intervalSecond seconds"))
+            .option("checkpointLocation", tablePath.getStreamingCheckpointDir)
+            .option("tablePath", tablePath.getPath)
+            .option("bad_records_action", badRecordAction)
+            .option(CarbonStreamOutputFormat.HANDOFF_SIZE, handoffSize)
+            .start()
+          qry.awaitTermination()
+        } catch {
+          case ex =>
+            throw new Exception(ex.getMessage)
+        } finally {
+          if (null != qry) {
+            qry.stop()
+          }
+        }
+      }
+    }
+  }
+
+  def executeStreamingIngest(
+      tableName: String,
+      batchNums: Int,
+      rowNumsEachBatch: Int,
+      intervalOfSource: Int,
+      intervalOfIngest: Int,
+      continueSeconds: Int,
+      generateBadRecords: Boolean,
+      badRecordAction: String,
+      handoffSize: Long = CarbonStreamOutputFormat.HANDOFF_SIZE_DEFAULT
+  ): Unit = {
+    val identifier = new TableIdentifier(tableName, Option("streaming"))
+    val carbonTable = CarbonEnv.getInstance(spark).carbonMetastore.lookupRelation(identifier)(spark)
+      .asInstanceOf[CarbonRelation].metaData.carbonTable
+    val tablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
+    var server: ServerSocket = null
+    try {
+      server = new ServerSocket(7071)
+
+      val thread1 = createWriteSocketThread(
+        serverSocket = server,
+        writeNums = batchNums,
+        rowNums = rowNumsEachBatch,
+        intervalSecond = intervalOfSource,
+        badRecords = generateBadRecords)
+      val thread2 = createSocketStreamingThread(
+        spark = spark,
+        tablePath = tablePath,
+        badRecordAction = badRecordAction,
+        intervalSecond = intervalOfIngest,
+        handoffSize = handoffSize)
+      thread1.start()
+      thread2.start()
+      Thread.sleep(continueSeconds * 1000)
+      thread1.interrupt()
+      thread2.interrupt()
+    } finally {
+      if (null != server) {
+        server.close()
+      }
+    }
+  }
+
+  def generateCSVDataFile(
+      spark: SparkSession,
+      idStart: Int,
+      rowNums: Int,
+      csvDirPath: String): Unit = {
+    // Create csv data frame file
+    val csvRDD = spark.sparkContext.parallelize(idStart until idStart + rowNums)
+      .map { id =>
+        (id,
+          "name_" + id,
+          "city_" + id,
+          10000.00 * id,
+          "school_" + id + ":school_" + id + id + "$" + id)
+      }
+    val csvDataDF = spark.createDataFrame(csvRDD).toDF("id", "name", "city", "salary", "file")
+
+    csvDataDF.write
+      .option("header", "false")
+      .mode(SaveMode.Overwrite)
+      .csv(csvDirPath)
+  }
+
+  def createFileStreamingThread(
+      spark: SparkSession,
+      tablePath: CarbonTablePath,
+      csvDataDir: String,
+      intervalSecond: Int): Thread = {
+    new Thread() {
+      override def run(): Unit = {
+        val inputSchema = new StructType()
+          .add("id", "integer")
+          .add("name", "string")
+          .add("city", "string")
+          .add("salary", "float")
+          .add("file", "string")
+        var qry: StreamingQuery = null
+        try {
+          val readSocketDF = spark.readStream
+            .format("csv")
+            .option("sep", ",")
+            .schema(inputSchema)
+            .option("path", csvDataDir)
+            .option("header", "false")
+            .load()
+
+          // Write data from socket stream to carbondata file
+          qry = readSocketDF.writeStream
+            .format("carbondata")
+            .trigger(ProcessingTime(s"${ intervalSecond } seconds"))
+            .option("checkpointLocation", tablePath.getStreamingCheckpointDir)
+            .option("tablePath", tablePath.getPath)
+            .start()
+
+          qry.awaitTermination()
+        } catch {
+          case _: InterruptedException =>
+            println("Done reading and writing streaming data")
+        } finally {
+          if (qry != null) {
+            qry.stop()
+          }
+        }
+      }
+    }
+  }
+
+  def createTable(tableName: String, streaming: Boolean, withBatchLoad: Boolean): Unit = {
+    sql(
+      s"""
+         | CREATE TABLE streaming.$tableName(
+         | id INT,
+         | name STRING,
+         | city STRING,
+         | salary FLOAT
+         | )
+         | STORED BY 'carbondata'
+         | TBLPROPERTIES(${if (streaming) "'streaming'='true', " else "" }
+         | 'sort_columns'='name', 'dictionary_include'='city')
+         | """.stripMargin)
+
+    if (withBatchLoad) {
+      // batch loading 5 rows
+      executeBatchLoad(tableName)
+    }
+  }
+
+  def createTableWithComplexType(
+      tableName: String,
+      streaming: Boolean,
+      withBatchLoad: Boolean): Unit = {
+    sql(
+      s"""
+         | CREATE TABLE streaming.$tableName(
+         | id INT,
+         | name STRING,
+         | city STRING,
+         | salary FLOAT,
+         | file struct<school:array<string>, age:int>
+         | )
+         | STORED BY 'carbondata'
+         | TBLPROPERTIES(${if (streaming) "'streaming'='true', " else "" }
+         | 'sort_columns'='name', 'dictionary_include'='city')
+         | """.stripMargin)
+
+    if (withBatchLoad) {
+      // batch loading 5 rows
+      executeBatchLoad(tableName)
+    }
+  }
+
+  def executeBatchLoad(tableName: String): Unit = {
+    sql(
+      s"""
+         | LOAD DATA LOCAL INPATH '$dataFilePath'
+         | INTO TABLE streaming.$tableName
+         | OPTIONS('HEADER'='true')
+         """.stripMargin)
+  }
+
+  def wrap(array: Array[String]) = {
+    new mutable.WrappedArray.ofRef(array)
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/40c31e80/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java b/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
index 53add22..45fcbb0 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
@@ -478,6 +478,12 @@ public final class CarbonDataMergerUtil {
       Date segDate1 = null;
       SimpleDateFormat sdf = new SimpleDateFormat(CarbonCommonConstants.CARBON_TIMESTAMP);
       for (LoadMetadataDetails segment : listOfSegmentsBelowThresholdSize) {
+        // compaction should skip streaming segments
+        if (CarbonCommonConstants.STORE_LOADSTATUS_STREAMING.equals(segment.getLoadStatus()) ||
+            CarbonCommonConstants.STORE_LOADSTATUS_STREAMING_FINISH.equals(
+                segment.getLoadStatus())) {
+          continue;
+        }
 
         if (first) {
           segDate1 = initializeFirstSegment(loadsOfSameDate, segment, sdf);
@@ -506,7 +512,15 @@ public final class CarbonDataMergerUtil {
         }
       }
     } else {
-      return listOfSegmentsBelowThresholdSize;
+      for (LoadMetadataDetails segment : listOfSegmentsBelowThresholdSize) {
+        // compaction should skip streaming segments
+        if (CarbonCommonConstants.STORE_LOADSTATUS_STREAMING.equals(segment.getLoadStatus()) ||
+            CarbonCommonConstants.STORE_LOADSTATUS_STREAMING_FINISH.equals(
+                segment.getLoadStatus())) {
+          continue;
+        }
+        loadsOfSameDate.add(segment);
+      }
     }
 
     return loadsOfSameDate;
@@ -583,6 +597,11 @@ public final class CarbonDataMergerUtil {
 
     // check size of each segment , sum it up across partitions
     for (LoadMetadataDetails segment : listOfSegmentsAfterPreserve) {
+      // compaction should skip streaming segments
+      if (CarbonCommonConstants.STORE_LOADSTATUS_STREAMING.equals(segment.getLoadStatus()) ||
+          CarbonCommonConstants.STORE_LOADSTATUS_STREAMING_FINISH.equals(segment.getLoadStatus())) {
+        continue;
+      }
 
       String segId = segment.getLoadName();
       // variable to store one  segment size across partition.
@@ -688,7 +707,11 @@ public final class CarbonDataMergerUtil {
 
     // check size of each segment , sum it up across partitions
     for (LoadMetadataDetails segment : listOfSegmentsAfterPreserve) {
-
+      // compaction should skip streaming segments
+      if (CarbonCommonConstants.STORE_LOADSTATUS_STREAMING.equals(segment.getLoadStatus()) ||
+          CarbonCommonConstants.STORE_LOADSTATUS_STREAMING_FINISH.equals(segment.getLoadStatus())) {
+        continue;
+      }
       String segName = segment.getLoadName();
 
       // if a segment is already merged 2 levels then it s name will become .2

http://git-wip-us.apache.org/repos/asf/carbondata/blob/40c31e80/streaming/src/main/java/org/apache/carbondata/streaming/segment/StreamSegment.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/carbondata/streaming/segment/StreamSegment.java b/streaming/src/main/java/org/apache/carbondata/streaming/segment/StreamSegment.java
index 32ba332..d65f24d 100644
--- a/streaming/src/main/java/org/apache/carbondata/streaming/segment/StreamSegment.java
+++ b/streaming/src/main/java/org/apache/carbondata/streaming/segment/StreamSegment.java
@@ -53,8 +53,6 @@ public class StreamSegment {
   private static final LogService LOGGER =
       LogServiceFactory.getLogService(StreamSegment.class.getName());
 
-  public static final long STREAM_SEGMENT_MAX_SIZE = 1024L * 1024 * 1024;
-
   /**
    * get stream segment or create new stream segment if not exists
    */
@@ -136,7 +134,8 @@ public class StreamSegment {
             "Acquired lock for table" + table.getDatabaseName() + "." + table.getFactTableName()
                 + " for stream table finish segment");
 
-        LoadMetadataDetails[] details = SegmentStatusManager.readLoadMetadata(tablePath.getPath());
+        LoadMetadataDetails[] details =
+            SegmentStatusManager.readLoadMetadata(tablePath.getMetadataDirectoryPath());
         for (LoadMetadataDetails detail : details) {
           if (segmentId.equals(detail.getLoadName())) {
             detail.setLoadEndTime(System.currentTimeMillis());
@@ -187,7 +186,7 @@ public class StreamSegment {
       TaskAttemptContext job) throws Exception {
     CarbonStreamRecordWriter writer = null;
     try {
-      writer = (CarbonStreamRecordWriter)new CarbonStreamOutputFormat().getRecordWriter(job);
+      writer = (CarbonStreamRecordWriter) new CarbonStreamOutputFormat().getRecordWriter(job);
       // at the begin of each task, should recover file if necessary
       // here can reuse some information of record writer
       recoverFileIfRequired(
@@ -199,6 +198,12 @@ public class StreamSegment {
         writer.write(null, inputIterators.next());
       }
       inputIterators.close();
+    } catch (Exception ex) {
+      if (writer != null) {
+        LOGGER.error(ex, "Failed to append batch data to stream segment: " +
+            writer.getSegmentDir());
+      }
+      throw ex;
     } finally {
       if (writer != null) {
         writer.close(job);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/40c31e80/streaming/src/main/scala/org/apache/carbondata/streaming/StreamSinkFactory.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/carbondata/streaming/StreamSinkFactory.scala b/streaming/src/main/scala/org/apache/carbondata/streaming/StreamSinkFactory.scala
index 3ac19d9..31ed1f6 100644
--- a/streaming/src/main/scala/org/apache/carbondata/streaming/StreamSinkFactory.scala
+++ b/streaming/src/main/scala/org/apache/carbondata/streaming/StreamSinkFactory.scala
@@ -23,7 +23,6 @@ import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd}
 import org.apache.spark.sql.SparkSession
 import org.apache.spark.sql.execution.streaming.{CarbonAppendableStreamSink, Sink}
 
-import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.dictionary.server.DictionaryServer
@@ -31,6 +30,7 @@ import org.apache.carbondata.core.metadata.encoder.Encoding
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.util.CarbonProperties
 import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.hadoop.streaming.CarbonStreamOutputFormat
 import org.apache.carbondata.processing.loading.model.CarbonLoadModel
 import org.apache.carbondata.spark.util.DataLoadingUtil
 import org.apache.carbondata.streaming.segment.StreamSegment
@@ -76,7 +76,21 @@ object StreamSinkFactory {
   }
 
   private def validateParameters(parameters: Map[String, String]): Unit = {
-    // TODO require to validate parameters
+    val segmentSize = parameters.get(CarbonStreamOutputFormat.HANDOFF_SIZE)
+    if (segmentSize.isDefined) {
+      try {
+        val value = java.lang.Long.parseLong(segmentSize.get)
+        if (value < CarbonStreamOutputFormat.HANDOFF_SIZE_MIN) {
+          new CarbonStreamException(CarbonStreamOutputFormat.HANDOFF_SIZE +
+                                    "should be bigger than or equal " +
+                                    CarbonStreamOutputFormat.HANDOFF_SIZE_MIN)
+        }
+      } catch {
+        case ex: NumberFormatException =>
+          new CarbonStreamException(CarbonStreamOutputFormat.HANDOFF_SIZE +
+                                    s" $segmentSize is an illegal number")
+      }
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/40c31e80/streaming/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala b/streaming/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala
index 844423a..b3f0964 100644
--- a/streaming/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala
+++ b/streaming/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala
@@ -68,6 +68,10 @@ class CarbonAppendableStreamSink(
     }
     conf
   }
+  // segment max size(byte)
+  private val segmentMaxSize = hadoopConf.getLong(
+    CarbonStreamOutputFormat.HANDOFF_SIZE,
+    CarbonStreamOutputFormat.HANDOFF_SIZE_DEFAULT)
 
   override def addBatch(batchId: Long, data: DataFrame): Unit = {
     if (batchId <= fileLog.getLatest().map(_._1).getOrElse(-1L)) {
@@ -107,8 +111,7 @@ class CarbonAppendableStreamSink(
   private def checkOrHandOffSegment(): Unit = {
     val segmentDir = carbonTablePath.getSegmentDir("0", currentSegmentId)
     val fileType = FileFactory.getFileType(segmentDir)
-    if (StreamSegment.STREAM_SEGMENT_MAX_SIZE <=
-        StreamSegment.size(segmentDir)) {
+    if (segmentMaxSize <= StreamSegment.size(segmentDir)) {
       val newSegmentId =
         StreamSegment.close(carbonTable, currentSegmentId)
       currentSegmentId = newSegmentId
@@ -238,6 +241,7 @@ object CarbonAppendableStreamSink {
     val taskAttemptContext: TaskAttemptContext = {
       // Set up the configuration object
       val hadoopConf = description.serializableHadoopConf.value
+      CarbonStreamOutputFormat.setSegmentId(hadoopConf, description.segmentId)
       hadoopConf.set("mapred.job.id", jobId.toString)
       hadoopConf.set("mapred.tip.id", taskAttemptId.getTaskID.toString)
       hadoopConf.set("mapred.task.id", taskAttemptId.toString)


[41/49] carbondata git commit: [CARBONDATA-1576] Added create datamap parser and saved to schema file

Posted by ra...@apache.org.
[CARBONDATA-1576] Added create datamap parser and saved to schema file

User can create datamap using the following syntax.

 CREATE DATAMAP agg_sales
  ON TABLE sales
USING org.apache.carbondata.datamap.AggregateDataMapHandler
DMPROPERTIES (
 'KEY’=’VALUE’
) AS
SELECT order_time, count(user_id) FROM sales GROUP BY order_time
In the above syntax DMPROPERTIES and AS QUERY are optional.

This closes #1481


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

Branch: refs/heads/fgdatamap
Commit: 1c16afad8decb42f28cb656e5f11664dcadf503b
Parents: 2b5faef
Author: ravipesala <ra...@gmail.com>
Authored: Sun Nov 12 17:08:09 2017 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Tue Nov 14 00:48:16 2017 +0530

----------------------------------------------------------------------
 .../ThriftWrapperSchemaConverterImpl.java       |  44 +++---
 .../metadata/schema/table/DataMapSchema.java    |  14 +-
 .../core/metadata/schema/table/TableInfo.java   |   2 +-
 .../core/metadata/schema/table/TableSchema.java |  12 +-
 format/src/main/thrift/schema.thrift            |  10 +-
 .../preaggregate/TestPreAggCreateCommand.scala  | 112 +++++++--------
 .../preaggregate/TestPreAggregateDrop.scala     |  28 ++--
 .../preaggregate/TestPreAggregateLoad.scala     |  19 +--
 .../testsuite/datamap/TestDataMapCommand.scala  |  95 +++++++++++++
 .../iud/DeleteCarbonTableTestCase.scala         |   8 +-
 .../iud/UpdateCarbonTableTestCase.scala         |   8 +-
 .../spark/sql/catalyst/CarbonDDLSqlParser.scala |   5 +-
 .../datamap/CarbonCreateDataMapCommand.scala    |  68 +++++++++
 .../CreatePreAggregateTableCommand.scala        | 141 ++++++++-----------
 .../preaaggregate/PreAggregateListeners.scala   |   8 +-
 .../preaaggregate/PreAggregateUtil.scala        |   8 +-
 .../spark/sql/hive/CarbonFileMetastore.scala    |   5 +-
 .../sql/parser/CarbonSpark2SqlParser.scala      |  24 +++-
 .../spark/sql/parser/CarbonSparkSqlParser.scala |  31 +---
 .../AlterTableValidationTestCase.scala          |   6 +-
 .../vectorreader/ChangeDataTypeTestCases.scala  |   8 +-
 .../vectorreader/DropColumnTestCases.scala      |   6 +-
 .../apache/spark/util/CarbonCommandSuite.scala  |   6 +-
 23 files changed, 411 insertions(+), 257 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/1c16afad/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 5ffc612..b914e06 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
@@ -322,17 +322,22 @@ public class ThriftWrapperSchemaConverterImpl implements SchemaConverter {
     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);
+      if (wrapperChildSchema.getRelationIdentifier() != null) {
+        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()));
+      thriftChildSchema.setDataMapName(wrapperChildSchema.getDataMapName());
+      if (wrapperChildSchema.getChildSchema() != null) {
+        thriftChildSchema.setChildTableSchema(
+            fromWrapperToExternalTableSchema(wrapperChildSchema.getChildSchema()));
+      }
       thriftChildSchemas.add(thriftChildSchema);
     }
     return thriftChildSchemas;
@@ -623,16 +628,19 @@ public class ThriftWrapperSchemaConverterImpl implements SchemaConverter {
 
   @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());
+    DataMapSchema childSchema =
+        new DataMapSchema(thriftDataMapSchema.getDataMapName(), thriftDataMapSchema.getClassName());
     childSchema.setProperties(thriftDataMapSchema.getProperties());
-    childSchema.setChildSchema(
-        fromExternalToWrapperTableSchema(thriftDataMapSchema.getChildTableSchema(),
-            relationIdentifier.getTableName()));
-    childSchema.setRelationIdentifier(relationIdentifier);
+    if (thriftDataMapSchema.getRelationIdentifire() != null) {
+      RelationIdentifier relationIdentifier =
+          new RelationIdentifier(thriftDataMapSchema.getRelationIdentifire().getDatabaseName(),
+              thriftDataMapSchema.getRelationIdentifire().getTableName(),
+              thriftDataMapSchema.getRelationIdentifire().getTableId());
+      childSchema.setRelationIdentifier(relationIdentifier);
+      childSchema.setChildSchema(
+          fromExternalToWrapperTableSchema(thriftDataMapSchema.getChildTableSchema(),
+              relationIdentifier.getTableName()));
+    }
     return childSchema;
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1c16afad/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
index 9c71e37..e0632d9 100644
--- 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
@@ -30,6 +30,8 @@ public class DataMapSchema implements Serializable, Writable {
 
   private static final long serialVersionUID = 6577149126264181553L;
 
+  private String dataMapName;
+
   private String className;
 
   private RelationIdentifier relationIdentifier;
@@ -43,7 +45,11 @@ public class DataMapSchema implements Serializable, Writable {
    */
   private Map<String, String> properties;
 
-  public DataMapSchema(String className) {
+  public DataMapSchema() {
+  }
+
+  public DataMapSchema(String dataMapName, String className) {
+    this.dataMapName = dataMapName;
     this.className = className;
   }
 
@@ -75,7 +81,12 @@ public class DataMapSchema implements Serializable, Writable {
     this.properties = properties;
   }
 
+  public String getDataMapName() {
+    return dataMapName;
+  }
+
   @Override public void write(DataOutput out) throws IOException {
+    out.writeUTF(dataMapName);
     out.writeUTF(className);
     boolean isRelationIdentifierExists = null != relationIdentifier;
     out.writeBoolean(isRelationIdentifierExists);
@@ -99,6 +110,7 @@ public class DataMapSchema implements Serializable, Writable {
   }
 
   @Override public void readFields(DataInput in) throws IOException {
+    this.dataMapName = in.readUTF();
     this.className = in.readUTF();
     boolean isRelationIdnentifierExists = in.readBoolean();
     if (isRelationIdnentifierExists) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1c16afad/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 44d8126..1d9e2ec 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
@@ -293,7 +293,7 @@ public class TableInfo implements Serializable, Writable {
     if (isChildSchemaExists) {
       short numberOfChildTable = in.readShort();
       for (int i = 0; i < numberOfChildTable; i++) {
-        DataMapSchema childSchema = new DataMapSchema(null);
+        DataMapSchema childSchema = new DataMapSchema();
         childSchema.readFields(in);
         dataMapSchemaList.add(childSchema);
       }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1c16afad/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 5b013d1..714e0d8 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
@@ -255,21 +255,15 @@ 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) {
+  public DataMapSchema buildChildSchema(String dataMapName, 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 dataMapSchema = new DataMapSchema(dataMapName, className);
     dataMapSchema.setChildSchema(this);
     dataMapSchema.setProperties(properties);
     dataMapSchema.setRelationIdentifier(relationIdentifier);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1c16afad/format/src/main/thrift/schema.thrift
----------------------------------------------------------------------
diff --git a/format/src/main/thrift/schema.thrift b/format/src/main/thrift/schema.thrift
index 4e9bb21..fc73cfb 100644
--- a/format/src/main/thrift/schema.thrift
+++ b/format/src/main/thrift/schema.thrift
@@ -187,15 +187,17 @@ struct ParentColumnTableRelation {
 }
 
 struct DataMapSchema  {
+    // DataMap name
+    1: required string dataMapName;
     // class name
-    1: required string className;
+    2: required string className;
     // relation indentifier
-    2: optional RelationIdentifier relationIdentifire;
+    3: 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;
+    4: optional TableSchema childTableSchema;
     // to maintain properties like select query, query type like groupby, join
-    4: optional map<string, string> properties;
+    5: optional map<string, string> properties;
 }
 
 struct TableInfo{

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1c16afad/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
index 6120e88..ce2d1d2 100644
--- 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
@@ -16,100 +16,100 @@ class TestPreAggCreateCommand extends QueryTest with BeforeAndAfterAll {
 
 
   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")
+    sql("create datamap preagg1 on table PreAggMain using 'preaggregate' as select a,sum(b) from PreAggMain group by a")
+    checkExistence(sql("DESCRIBE FORMATTED PreAggMain_preagg1"), true, "preaggmain_a")
+    checkExistence(sql("DESCRIBE FORMATTED PreAggMain_preagg1"), true, "preaggmain_b_sum")
+    sql("drop table PreAggMain_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")
+    sql("create datamap preagg2 on table PreAggMain using 'preaggregate' as select a as a1,sum(b) from PreAggMain group by a")
+    checkExistence(sql("DESCRIBE FORMATTED PreAggMain_preagg2"), true, "preaggmain_a")
+    checkExistence(sql("DESCRIBE FORMATTED PreAggMain_preagg2"), true, "preaggmain_b_sum")
+    sql("drop table PreAggMain_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")
+    sql("create datamap preagg3 on table PreAggMain using 'preaggregate' as select a,sum(b) as sum from PreAggMain group by a")
+    checkExistence(sql("DESCRIBE FORMATTED PreAggMain_preagg3"), true, "preaggmain_a")
+    checkExistence(sql("DESCRIBE FORMATTED PreAggMain_preagg3"), true, "preaggmain_b_sum")
+    sql("drop table PreAggMain_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")
+    sql("create datamap preagg4 on table PreAggMain using 'preaggregate' as select a as a1,sum(b) as sum from PreAggMain group by a")
+    checkExistence(sql("DESCRIBE FORMATTED PreAggMain_preagg4"), true, "preaggmain_a")
+    checkExistence(sql("DESCRIBE FORMATTED PreAggMain_preagg4"), true, "preaggmain_b_sum")
+    sql("drop table PreAggMain_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")
+    sql("create datamap preagg11 on table PreAggMain1 using 'preaggregate'as select a,sum(b) from PreAggMain1 group by a")
+    checkExistence(sql("DESCRIBE FORMATTED PreAggMain1_preagg11"), true, "preaggmain1_a")
+    checkExistence(sql("DESCRIBE FORMATTED PreAggMain1_preagg11"), true, "preaggmain1_b_sum")
+    checkExistence(sql("DESCRIBE FORMATTED PreAggMain1_preagg11"), true, "DICTIONARY")
+    sql("drop table PreAggMain1_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")
+    sql("create datamap preagg12 on table PreAggMain1 using 'preaggregate' as select a as a1,sum(b) from PreAggMain1 group by a")
+    checkExistence(sql("DESCRIBE FORMATTED PreAggMain1_preagg12"), true, "preaggmain1_a")
+    checkExistence(sql("DESCRIBE FORMATTED PreAggMain1_preagg12"), true, "preaggmain1_b_sum")
+    checkExistence(sql("DESCRIBE FORMATTED PreAggMain1_preagg12"), true, "DICTIONARY")
+    sql("drop table PreAggMain1_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")
+    sql("create datamap preagg13 on table PreAggMain1 using 'preaggregate' as select a,sum(b) as sum from PreAggMain1 group by a")
+    checkExistence(sql("DESCRIBE FORMATTED PreAggMain1_preagg13"), true, "preaggmain1_a")
+    checkExistence(sql("DESCRIBE FORMATTED PreAggMain1_preagg13"), true, "preaggmain1_b_sum")
+    checkExistence(sql("DESCRIBE FORMATTED PreAggMain1_preagg13"), true, "DICTIONARY")
+    sql("drop table PreAggMain1_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")
+    sql("create datamap preagg14 on table PreAggMain1 using 'preaggregate' as select a as a1,sum(b) as sum from PreAggMain1 group by a")
+    checkExistence(sql("DESCRIBE FORMATTED PreAggMain1_preagg14"), true, "preaggmain1_a")
+    checkExistence(sql("DESCRIBE FORMATTED PreAggMain1_preagg14"), true, "preaggmain1_b_sum")
+    checkExistence(sql("DESCRIBE FORMATTED PreAggMain1_preagg14"), true, "DICTIONARY")
+    sql("drop table PreAggMain1_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")
+    sql("create datamap preagg15 on table PreAggMain2 using 'preaggregate' as select a,avg(b) from PreAggMain2 group by a")
+    checkExistence(sql("DESCRIBE FORMATTED PreAggMain2_preagg15"), true, "preaggmain2_a")
+    checkExistence(sql("DESCRIBE FORMATTED PreAggMain2_preagg15"), true, "preaggmain2_b_sum")
+    checkExistence(sql("DESCRIBE FORMATTED PreAggMain2_preagg15"), true, "preaggmain2_b_count")
+    sql("drop table PreAggMain2_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")
+    sql("create datamap preagg16 on table PreAggMain2 using 'preaggregate' as select a as a1,max(b) from PreAggMain2 group by a")
+    checkExistence(sql("DESCRIBE FORMATTED PreAggMain2_preagg16"), true, "preaggmain2_a")
+    checkExistence(sql("DESCRIBE FORMATTED PreAggMain2_preagg16"), true, "preaggmain2_b_max")
+    sql("drop table PreAggMain2_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")
+    sql("create datamap preagg17 on table PreAggMain2 using 'preaggregate' as select a,min(b) from PreAggMain2 group by a")
+    checkExistence(sql("DESCRIBE FORMATTED PreAggMain2_preagg17"), true, "preaggmain2_a")
+    checkExistence(sql("DESCRIBE FORMATTED PreAggMain2_preagg17"), true, "preaggmain2_b_min")
+    sql("drop table PreAggMain2_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")
+    sql("create datamap preagg18 on table PreAggMain2 using 'preaggregate' as select a as a1,count(b) from PreAggMain2 group by a")
+    checkExistence(sql("DESCRIBE FORMATTED PreAggMain2_preagg18"), true, "preaggmain2_a")
+    checkExistence(sql("DESCRIBE FORMATTED PreAggMain2_preagg18"), true, "preaggmain2_b_count")
+    sql("drop table PreAggMain2_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")
+        "create datamap preagg19 on table PreAggMain2 using 'preaggregate' as select a as a1,count(distinct b) from PreAggMain2 group by a")
       assert(false)
     } catch {
       case _: Exception =>
@@ -120,7 +120,7 @@ class TestPreAggCreateCommand extends QueryTest with BeforeAndAfterAll {
   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")
+        "create datamap preagg20 on table PreAggMain2 using 'preaggregate' as select a as a1,sum(distinct b) from PreAggMain2 group by a")
       assert(false)
     } catch {
       case _: Exception =>
@@ -131,7 +131,7 @@ class TestPreAggCreateCommand extends QueryTest with BeforeAndAfterAll {
   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")
+        "create datamap preagg21 on table PreAggMain2 using 'preaggregate' as select a as a1,sum(b) from PreAggMain2 where a='vishal' group by a")
       assert(false)
     } catch {
       case _: Exception =>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1c16afad/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
index 4dad3e1..fee73a7 100644
--- 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
@@ -24,44 +24,44 @@ 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("drop table if exists maintable_preagg1")
+    sql("drop table if exists maintable_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" +
+      "create datamap preagg1 on table maintable using 'preaggregate' as select" +
       " a,sum(b) from maintable group by a")
-    sql("drop table if exists preagg1")
-    checkExistence(sql("show tables"), false, "preagg1")
+    sql("drop table if exists maintable_preagg1")
+    checkExistence(sql("show tables"), false, "maintable_preagg1")
   }
 
   test("dropping 1 aggregate table should not drop others") {
     sql(
-      "create table preagg1 stored BY 'carbondata' tblproperties('parent'='maintable') as select" +
+      "create datamap preagg1 on table maintable using 'preaggregate' as select" +
       " a,sum(b) from maintable group by a")
     sql(
-      "create table preagg2 stored BY 'carbondata' tblproperties('parent'='maintable') as select" +
+      "create datamap preagg2 on table maintable using 'preaggregate'  as select" +
       " a,sum(c) from maintable group by a")
-    sql("drop table if exists preagg2")
+    sql("drop table if exists maintable_preagg2")
     val showTables = sql("show tables")
-    checkExistence(showTables, false, "preagg2")
-    checkExistence(showTables, true, "preagg1")
+    checkExistence(showTables, false, "maintable_preagg2")
+    checkExistence(showTables, true, "maintable_preagg1")
   }
   
   test("drop main table and check if preaggreagte is deleted") {
     sql(
-      "create table preagg2 stored BY 'carbondata' tblproperties('parent'='maintable') as select" +
+      "create datamap preagg2 on table maintable using 'preaggregate' as select" +
       " a,sum(c) from maintable group by a")
     sql("drop table if exists maintable")
-    checkExistence(sql("show tables"), false, "preagg1", "maintable", "preagg2")
+    checkExistence(sql("show tables"), false, "maintable_preagg1", "maintable", "maintable_preagg2")
   }
 
   override def afterAll() {
     sql("drop table if exists maintable")
-    sql("drop table if exists preagg1")
-    sql("drop table if exists preagg2")
+    sql("drop table if exists maintable_preagg1")
+    sql("drop table if exists maintable_preagg2")
   }
   
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1c16afad/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateLoad.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateLoad.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateLoad.scala
index 1f576c5..5ac3534 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateLoad.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateLoad.scala
@@ -23,33 +23,28 @@ import org.scalatest.{BeforeAndAfterAll, Ignore}
 
 @Ignore
 class TestPreAggregateLoad extends QueryTest with BeforeAndAfterAll {
-  
+
   val testData = s"$resourcesPath/sample.csv"
-  
+
   override def beforeAll(): Unit = {
     sql("DROP TABLE IF EXISTS maintable")
   }
 
   private def createAllAggregateTables(parentTableName: String): Unit = {
     sql(
-      s"""create table ${ parentTableName }_preagg_sum stored BY 'carbondata' tblproperties
-         |('parent'='$parentTableName') as select id,sum(age) from $parentTableName group by id"""
+      s"""create datamap preagg_sum on table $parentTableName using 'preaggregate' as select id,sum(age) from $parentTableName group by id"""
         .stripMargin)
     sql(
-      s"""create table ${ parentTableName }_preagg_avg stored BY 'carbondata' tblproperties
-         |('parent'='$parentTableName') as select id,avg(age) from $parentTableName group by id"""
+      s"""create datamap preagg_avg on table $parentTableName using 'preaggregate' as select id,avg(age) from $parentTableName group by id"""
         .stripMargin)
     sql(
-      s"""create table ${ parentTableName }_preagg_count stored BY 'carbondata' tblproperties
-         |('parent'='$parentTableName') as select id,count(age) from $parentTableName group by id"""
+      s"""create datamap preagg_count on table $parentTableName using 'preaggregate' as select id,count(age) from $parentTableName group by id"""
         .stripMargin)
     sql(
-      s"""create table ${ parentTableName }_preagg_min stored BY 'carbondata' tblproperties
-         |('parent'='$parentTableName') as select id,min(age) from $parentTableName group by id"""
+      s"""create datamap preagg_min on table $parentTableName using 'preaggregate' as select id,min(age) from $parentTableName group by id"""
         .stripMargin)
     sql(
-      s"""create table ${ parentTableName }_preagg_max stored BY 'carbondata' tblproperties
-         |('parent'='$parentTableName') as select id,max(age) from $parentTableName group by id"""
+      s"""create datamap preagg_max on table $parentTableName using 'preaggregate' as select id,max(age) from $parentTableName group by id"""
         .stripMargin)
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1c16afad/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/TestDataMapCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/TestDataMapCommand.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/TestDataMapCommand.scala
new file mode 100644
index 0000000..2512def
--- /dev/null
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/TestDataMapCommand.scala
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.spark.testsuite.datamap
+
+import org.apache.spark.sql.test.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+import org.apache.carbondata.core.metadata.CarbonMetadata
+
+class TestDataMapCommand extends QueryTest with BeforeAndAfterAll {
+
+  override def beforeAll {
+    sql("drop table if exists datamaptest")
+    sql("create table datamaptest (a string, b string, c string) stored by 'carbondata'")
+  }
+
+
+  test("test datamap create") {
+    sql("create datamap datamap1 on table datamaptest using 'new.class'")
+    val table = CarbonMetadata.getInstance().getCarbonTable("default_datamaptest")
+    assert(table != null)
+    val dataMapSchemaList = table.getTableInfo.getDataMapSchemaList
+    assert(dataMapSchemaList.size() == 1)
+    assert(dataMapSchemaList.get(0).getDataMapName.equals("datamap1"))
+    assert(dataMapSchemaList.get(0).getClassName.equals("new.class"))
+  }
+
+  test("test datamap create with dmproperties") {
+    sql("create datamap datamap2 on table datamaptest using 'new.class' dmproperties('key'='value')")
+    val table = CarbonMetadata.getInstance().getCarbonTable("default_datamaptest")
+    assert(table != null)
+    val dataMapSchemaList = table.getTableInfo.getDataMapSchemaList
+    assert(dataMapSchemaList.size() == 2)
+    assert(dataMapSchemaList.get(1).getDataMapName.equals("datamap2"))
+    assert(dataMapSchemaList.get(1).getClassName.equals("new.class"))
+    assert(dataMapSchemaList.get(1).getProperties.get("key").equals("value"))
+  }
+
+  test("test datamap create with existing name") {
+    intercept[Exception] {
+      sql(
+        "create datamap datamap2 on table datamaptest using 'new.class' dmproperties('key'='value')")
+    }
+    val table = CarbonMetadata.getInstance().getCarbonTable("default_datamaptest")
+    assert(table != null)
+    val dataMapSchemaList = table.getTableInfo.getDataMapSchemaList
+    assert(dataMapSchemaList.size() == 2)
+  }
+
+  test("test datamap create with preagg") {
+    sql("drop table if exists datamap3")
+    sql(
+      "create datamap datamap3 on table datamaptest using 'preaggregate' dmproperties('key'='value') as select count(a) from datamaptest")
+    val table = CarbonMetadata.getInstance().getCarbonTable("default_datamaptest")
+    assert(table != null)
+    val dataMapSchemaList = table.getTableInfo.getDataMapSchemaList
+    assert(dataMapSchemaList.size() == 3)
+    assert(dataMapSchemaList.get(2).getDataMapName.equals("datamap3"))
+    assert(dataMapSchemaList.get(2).getProperties.get("key").equals("value"))
+    assert(dataMapSchemaList.get(2).getChildSchema.getTableName.equals("datamaptest_datamap3"))
+  }
+
+  test("test datamap create with preagg with duplicate name") {
+    intercept[Exception] {
+      sql("drop table if exists datamap2")
+      sql(
+        "create datamap datamap2 on table datamaptest using 'preaggregate' dmproperties('key'='value') as select count(a) from datamaptest")
+
+    }
+    val table = CarbonMetadata.getInstance().getCarbonTable("default_datamaptest")
+    assert(table != null)
+    val dataMapSchemaList = table.getTableInfo.getDataMapSchemaList
+    assert(dataMapSchemaList.size() == 3)
+  }
+
+
+  override def afterAll {
+    sql("drop table if exists datamaptest")
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1c16afad/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/DeleteCarbonTableTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/DeleteCarbonTableTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/DeleteCarbonTableTestCase.scala
index a2bd6aa..f7e93af 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/DeleteCarbonTableTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/DeleteCarbonTableTestCase.scala
@@ -131,17 +131,17 @@ class DeleteCarbonTableTestCase extends QueryTest with BeforeAndAfterAll {
 
   test("test if delete is unsupported for pre-aggregate tables") {
     sql("drop table if exists preaggMain")
-    sql("drop table if exists preagg1")
+    sql("drop table if exists preaggmain_preagg1")
     sql("create table preaggMain (a string, b string, c string) stored by 'carbondata'")
-    sql("create table preagg1 stored BY 'carbondata' tblproperties('parent'='PreAggMain') as select a,sum(b) from PreAggMain group by a")
+    sql("create datamap preagg1 on table PreAggMain USING 'preaggregate' as select a,sum(b) from PreAggMain group by a")
     intercept[RuntimeException] {
       sql("delete from preaggmain where a = 'abc'").show()
     }.getMessage.contains("Delete operation is not supported for tables")
     intercept[RuntimeException] {
-      sql("delete from preagg1 where preaggmain_a = 'abc'").show()
+      sql("delete from preaggmain_preagg1 where preaggmain_a = 'abc'").show()
     }.getMessage.contains("Delete operation is not supported for pre-aggregate table")
     sql("drop table if exists preaggMain")
-    sql("drop table if exists preagg1")
+    sql("drop table if exists preaggmain_preagg1")
   }
 
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1c16afad/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/UpdateCarbonTableTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/UpdateCarbonTableTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/UpdateCarbonTableTestCase.scala
index 4c43ec0..fd5f144 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/UpdateCarbonTableTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/UpdateCarbonTableTestCase.scala
@@ -515,17 +515,17 @@ class UpdateCarbonTableTestCase extends QueryTest with BeforeAndAfterAll {
 
   test("test if update is unsupported for pre-aggregate tables") {
     sql("drop table if exists preaggMain")
-    sql("drop table if exists preagg1")
+    sql("drop table if exists preaggMain_preagg1")
     sql("create table preaggMain (a string, b string, c string) stored by 'carbondata'")
-    sql("create table preagg1 stored BY 'carbondata' tblproperties('parent'='PreAggMain') as select a,sum(b) from PreAggMain group by a")
+    sql("create datamap preagg1 on table PreAggMain using 'preaggregate' as select a,sum(b) from PreAggMain group by a")
     intercept[RuntimeException] {
       sql("update preaggmain set (a)=('a')").show
     }.getMessage.contains("Update operation is not supported for tables")
     intercept[RuntimeException] {
-      sql("update preagg1 set (a)=('a')").show
+      sql("update preaggMain_preagg1 set (a)=('a')").show
     }.getMessage.contains("Update operation is not supported for pre-aggregate table")
     sql("drop table if exists preaggMain")
-    sql("drop table if exists preagg1")
+    sql("drop table if exists preaggMain_preagg1")
   }
 
   override def afterAll {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1c16afad/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 bb80bce..42447da 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
@@ -170,6 +170,9 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
   protected val CHANGE = carbonKeyWord("CHANGE")
   protected val TBLPROPERTIES = carbonKeyWord("TBLPROPERTIES")
   protected val ID = carbonKeyWord("ID")
+  protected val DATAMAP = carbonKeyWord("DATAMAP")
+  protected val ON = carbonKeyWord("ON")
+  protected val DMPROPERTIES = carbonKeyWord("DMPROPERTIES")
 
   protected val doubleQuotedString = "\"([^\"]+)\"".r
   protected val singleQuotedString = "'([^']+)'".r
@@ -750,7 +753,7 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
    * @param dbName
    * @return Option of String
    */
-  protected def convertDbNameToLowerCase(dbName: Option[String]): Option[String] = {
+  def convertDbNameToLowerCase(dbName: Option[String]): Option[String] = {
     dbName match {
       case Some(databaseName) => Some(convertDbNameToLowerCase(databaseName))
       case None => dbName

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1c16afad/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonCreateDataMapCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonCreateDataMapCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonCreateDataMapCommand.scala
new file mode 100644
index 0000000..8b6e3b9
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonCreateDataMapCommand.scala
@@ -0,0 +1,68 @@
+/*
+ * 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.datamap
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.execution.command._
+import org.apache.spark.sql.execution.command.preaaggregate.{CreatePreAggregateTableCommand, PreAggregateUtil}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.metadata.schema.table.DataMapSchema
+
+/**
+ * Below command class will be used to create datamap on table
+ * and updating the parent table about the datamap information
+ *
+ * @param queryString
+ */
+case class CarbonCreateDataMapCommand(
+    dataMapName: String,
+    tableIdentifier: TableIdentifier,
+    dmClassName: String,
+    dmproperties: Map[String, String],
+    queryString: Option[String])
+  extends RunnableCommand with SchemaProcessCommand {
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    processSchema(sparkSession)
+  }
+
+  override def processSchema(sparkSession: SparkSession): Seq[Row] = {
+    val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+    if (dmClassName.equals("org.apache.carbondata.datamap.AggregateDataMapHandler") ||
+        dmClassName.equalsIgnoreCase("preaggregate")) {
+      CreatePreAggregateTableCommand(dataMapName,
+        tableIdentifier,
+        dmClassName,
+        dmproperties,
+        queryString.get).run(sparkSession)
+    } else {
+      val dataMapSchema = new DataMapSchema(dataMapName, dmClassName)
+      dataMapSchema.setProperties(new java.util.HashMap[String, String](dmproperties.asJava))
+      val dbName = GetDB.getDatabaseName(tableIdentifier.database, sparkSession)
+      // upadting the parent table about dataschema
+      PreAggregateUtil.updateMainTable(dbName, tableIdentifier.table, dataMapSchema, sparkSession)
+    }
+    LOGGER.audit(s"DataMap ${dataMapName} successfully added to Table ${tableIdentifier.table}")
+    Seq.empty
+  }
+}
+
+

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1c16afad/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 e12cbb9..ebf6273 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
@@ -16,16 +16,14 @@
  */
 package org.apache.spark.sql.execution.command.preaaggregate
 
+import scala.collection.mutable
+import scala.collection.JavaConverters._
+
 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.TableInfo
-import org.apache.carbondata.core.util.CarbonUtil
+import org.apache.spark.sql.hive.CarbonRelation
+import org.apache.spark.sql.parser.CarbonSpark2SqlParser
 
 /**
  * Below command class will be used to create pre-aggregate table
@@ -34,17 +32,14 @@ import org.apache.carbondata.core.util.CarbonUtil
  * 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])
+    dataMapName: String,
+    parentTableIdentifier: TableIdentifier,
+    dmClassName: String,
+    dmproperties: Map[String, String],
+    queryString: String)
   extends RunnableCommand with SchemaProcessCommand {
 
   override def run(sparkSession: SparkSession): Seq[Row] = {
@@ -52,87 +47,61 @@ case class CreatePreAggregateTableCommand(
   }
 
   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]")
+    val df = sparkSession.sql(queryString)
+    val fieldRelationMap = PreAggregateUtil
+      .validateActualSelectPlanAndGetAttrubites(df.logicalPlan, queryString)
+    val fields = fieldRelationMap.keySet.toSeq
+    val tableProperties = mutable.Map[String, String]()
+    dmproperties.foreach(t => tableProperties.put(t._1, t._2))
+    // Create the aggregation table name with parent table name prefix
+    val tableIdentifier = TableIdentifier(
+        parentTableIdentifier.table +"_" + dataMapName, parentTableIdentifier.database)
+    // prepare table model of the collected tokens
+    val tableModel: TableModel = new CarbonSpark2SqlParser().prepareTableModel(false,
+      new CarbonSpark2SqlParser().convertDbNameToLowerCase(tableIdentifier.database),
+      tableIdentifier.table.toLowerCase,
+      fields,
+      Seq(),
+      tableProperties,
+      None,
+      false,
+      None)
+
     // getting the parent table
-    val parentTable = PreAggregateUtil.getParentCarbonTable(dataFrame.logicalPlan)
+    val parentTable = PreAggregateUtil.getParentCarbonTable(df.logicalPlan)
     // getting the table name
     val parentTableName = parentTable.getFactTableName
     // getting the db name of parent table
     val parentDbName = parentTable.getDatabaseName
+
+    assert(parentTableName.equalsIgnoreCase(parentTableIdentifier.table))
     // 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)
-          val loadAvailable = PreAggregateUtil
-            .checkMainTableLoad(parentTable)
-          if (loadAvailable) {
-            sparkSession.sql(s"insert into ${ cm.databaseName }.${ cm.tableName } $queryString")
-          }
-        } 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
-        }
+    tableModel.parentTable = Some(parentTable)
+    tableModel.dataMapRelation = Some(fieldRelationMap)
+    CarbonCreateTableCommand(tableModel).run(sparkSession)
+    try {
+      val relation = CarbonEnv.getInstance(sparkSession).carbonMetastore.
+        lookupRelation(tableIdentifier)(sparkSession).asInstanceOf[CarbonRelation]
+      val tableInfo = relation.tableMeta.carbonTable.getTableInfo
+      // child schema object which will be updated on parent table about the
+      val childSchema = tableInfo.getFactTable.buildChildSchema(
+        dataMapName, "", tableInfo.getDatabaseName, queryString, "AGGREGATION")
+      dmproperties.foreach(f => childSchema.getProperties.put(f._1, f._2))
+      // updating the parent table about child table
+      PreAggregateUtil.updateMainTable(parentDbName, parentTableName, childSchema, sparkSession)
+      val loadAvailable = PreAggregateUtil.checkMainTableLoad(parentTable)
+      if (loadAvailable) {
+        sparkSession.sql(
+          s"insert into ${ tableModel.databaseName }.${ tableModel.tableName } $queryString")
       }
+    } catch {
+      case e: Exception =>
+        sparkSession.sql(
+          s"""DROP TABLE IF EXISTS ${ tableModel.databaseName }.${ tableModel.tableName }""")
+        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/1c16afad/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateListeners.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateListeners.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateListeners.scala
index 2ce97fe..8271e57 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateListeners.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateListeners.scala
@@ -41,9 +41,11 @@ object DropPreAggregateTablePostListener extends OperationEventListener {
         !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)
+        if (childSchema.getRelationIdentifier != null) {
+          CarbonDropTableCommand(ifExistsSet = true,
+            Some(childSchema.getRelationIdentifier.getDatabaseName),
+            childSchema.getRelationIdentifier.getTableName).run(sparkSession)
+        }
       }
     }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1c16afad/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 b35b525..b926705 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
@@ -17,6 +17,7 @@
 package org.apache.spark.sql.execution.command.preaaggregate
 
 import scala.collection.mutable.ListBuffer
+import scala.collection.JavaConverters._
 
 import org.apache.spark.SparkConf
 import org.apache.spark.sql.{CarbonDatasourceHadoopRelation, CarbonEnv, SparkSession}
@@ -303,6 +304,10 @@ object PreAggregateUtil {
           tableName,
           carbonTable.getStorePath)
       numberOfCurrentChild = wrapperTableInfo.getDataMapSchemaList.size
+      if (wrapperTableInfo.getDataMapSchemaList.asScala.
+        exists(f => f.getDataMapName.equalsIgnoreCase(childSchema.getDataMapName))) {
+        throw new Exception("Duplicate datamap")
+      }
       wrapperTableInfo.getDataMapSchemaList.add(childSchema)
       val thriftTable = schemaConverter
         .fromWrapperToExternalTableInfo(wrapperTableInfo, dbName, tableName)
@@ -446,7 +451,6 @@ object PreAggregateUtil {
   }
 
   def checkMainTableLoad(carbonTable: CarbonTable): Boolean = {
-    SegmentStatusManager.readLoadMetadata(
-      carbonTable.getMetaDataFilepath).nonEmpty
+    SegmentStatusManager.readLoadMetadata(carbonTable.getMetaDataFilepath).nonEmpty
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1c16afad/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 ac75fa7..a5e1ec4 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
@@ -465,8 +465,9 @@ class CarbonFileMetastore extends CarbonMetaStore {
         }
         val childSchemaIterator = childSchemas.iterator()
         while (childSchemaIterator.hasNext) {
-          val childSchema = childSchemaIterator.next()
-          if (childSchema.getChildSchema.equals(childCarbonTable.getTableInfo.getFactTable)) {
+          val childSchema = childSchemaIterator.next().getChildSchema
+          if (childSchema != null &&
+              childSchema.equals(childCarbonTable.getTableInfo.getFactTable)) {
             childSchemaIterator.remove()
           }
         }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1c16afad/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 bf21bc8..46a2515 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
@@ -21,11 +21,12 @@ import scala.collection.mutable
 import scala.language.implicitConversions
 
 import org.apache.spark.sql.{DeleteRecords, ShowLoadsCommand, UpdateTable}
-import org.apache.spark.sql.catalyst.CarbonDDLSqlParser
+import org.apache.spark.sql.catalyst.{CarbonDDLSqlParser, TableIdentifier}
 import org.apache.spark.sql.catalyst.CarbonTableIdentifierImplicit._
 import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.execution.command._
+import org.apache.spark.sql.execution.command.datamap.CarbonCreateDataMapCommand
 import org.apache.spark.sql.execution.command.management.{AlterTableCompactionCommand, CleanFilesCommand, DeleteLoadByIdCommand, DeleteLoadByLoadDateCommand, LoadTableCommand}
 import org.apache.spark.sql.execution.command.partition.{AlterTableDropCarbonPartitionCommand, AlterTableSplitCarbonPartitionCommand}
 import org.apache.spark.sql.execution.command.schema.{CarbonAlterTableAddColumnCommand, CarbonAlterTableDataTypeChangeCommand, CarbonAlterTableDropColumnCommand}
@@ -66,7 +67,8 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
   protected lazy val start: Parser[LogicalPlan] = explainPlan | startCommand
 
   protected lazy val startCommand: Parser[LogicalPlan] =
-    loadManagement|showLoads|alterTable|restructure|updateTable|deleteRecords|alterPartition
+    loadManagement|showLoads|alterTable|restructure|updateTable|deleteRecords|
+    alterPartition|datamapManagement
 
   protected lazy val loadManagement: Parser[LogicalPlan] =
     deleteLoadsByID | deleteLoadsByLoadDate | cleanFiles | loadDataNew
@@ -77,6 +79,9 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
   protected lazy val alterPartition: Parser[LogicalPlan] =
     alterAddPartition | alterSplitPartition | alterDropPartition
 
+  protected lazy val datamapManagement: Parser[LogicalPlan] =
+    createDataMap
+
   protected lazy val alterAddPartition: Parser[LogicalPlan] =
     ALTER ~> TABLE ~> (ident <~ ".").? ~ ident ~ (ADD ~> PARTITION ~>
       "(" ~> repsep(stringLit, ",") <~ ")") <~ opt(";") ^^ {
@@ -121,6 +126,21 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
         AlterTableCompactionCommand(altertablemodel)
     }
 
+  /**
+   * The syntax of datamap creation is as follows.
+   * CREATE DATAMAP datamapName ON TABLE tableName USING 'DataMapClassName'
+   * DMPROPERTIES('KEY'='VALUE') AS SELECT COUNT(COL1) FROM tableName
+   */
+  protected lazy val createDataMap: Parser[LogicalPlan] =
+    CREATE ~> DATAMAP ~> ident ~ (ON ~ TABLE) ~  (ident <~ ".").? ~ ident ~
+    (USING ~> stringLit) ~ (DMPROPERTIES ~> "(" ~> repsep(loadOptions, ",") <~ ")").? ~
+    (AS ~> restInput).? <~ opt(";")  ^^ {
+      case dmname ~ ontable ~ dbName ~ tableName ~ className ~ dmprops ~ query =>
+        val map = dmprops.getOrElse(List[(String, String)]()).toMap[String, String]
+        CarbonCreateDataMapCommand(dmname,
+          TableIdentifier(tableName, dbName), className, map, query)
+    }
+
   protected lazy val deleteRecords: Parser[LogicalPlan] =
     (DELETE ~> FROM ~> table) ~ restInput.? <~ opt(";") ^^ {
       case table ~ rest =>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1c16afad/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 256477e..5c51156 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
@@ -25,8 +25,7 @@ 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._
-import org.apache.spark.sql.execution.command.preaaggregate.{CreatePreAggregateTableCommand, PreAggregateUtil}
+import org.apache.spark.sql.execution.command.{BucketFields, CarbonCreateTableCommand, PartitionerField, TableModel}
 import org.apache.spark.sql.internal.{SQLConf, VariableSubstitution}
 import org.apache.spark.sql.types.StructField
 
@@ -40,7 +39,7 @@ import org.apache.carbondata.spark.util.CommonUtil
  */
 class CarbonSparkSqlParser(conf: SQLConf, sparkSession: SparkSession) extends AbstractSqlParser {
 
-  val astBuilder = new CarbonSqlAstBuilder(conf, sparkSession: SparkSession)
+  val astBuilder = new CarbonSqlAstBuilder(conf)
 
   private val substitutor = new VariableSubstitution(conf)
 
@@ -70,8 +69,7 @@ class CarbonSparkSqlParser(conf: SQLConf, sparkSession: SparkSession) extends Ab
   }
 }
 
-class CarbonSqlAstBuilder(conf: SQLConf, sparkSession: SparkSession)
-  extends SparkSqlAstBuilder(conf) {
+class CarbonSqlAstBuilder(conf: SQLConf) extends SparkSqlAstBuilder(conf) {
 
   val parser = new CarbonSpark2SqlParser
 
@@ -119,18 +117,8 @@ class CarbonSqlAstBuilder(conf: SQLConf, sparkSession: SparkSession)
       val (partitionByStructFields, partitionFields) =
         validateParitionFields(ctx, colNames, tableProperties)
 
-      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
-      }
+      val fields = parser.getFields(colsStructFields ++ partitionByStructFields)
+
       // validate bucket fields
       val bucketFields: Option[BucketFields] =
         parser.getBucketFields(tableProperties, fields, options)
@@ -149,14 +137,7 @@ class CarbonSqlAstBuilder(conf: SQLConf, sparkSession: SparkSession)
         isAlterFlow = false,
         tableComment)
 
-      if(!isAggTable) {
-        CarbonCreateTableCommand(tableModel)
-      } else {
-        CreatePreAggregateTableCommand(tableModel,
-          dfAndFieldRelationTuple.get._1,
-          queryString = source(ctx.query).toString,
-          fieldRelationMap = dfAndFieldRelationTuple.get._2)
-      }
+      CarbonCreateTableCommand(tableModel)
     } else {
       super.visitCreateTable(ctx)
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1c16afad/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 0274605..b798379 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
@@ -493,13 +493,13 @@ class AlterTableValidationTestCase extends Spark2QueryTest with BeforeAndAfterAl
   test("test to check if new parent table name is reflected in pre-aggregate tables") {
     sql("drop table if exists preaggMain")
     sql("drop table if exists preaggmain_new")
-    sql("drop table if exists preagg1")
+    sql("drop table if exists preaggMain_preagg1")
     sql("create table preaggMain (a string, b string, c string) stored by 'carbondata'")
     sql(
-      "create table preagg1 stored BY 'carbondata' tblproperties('parent'='PreAggMain') as select" +
+      "create datamap preagg1 on table PreAggMain using 'preaggregate' as select" +
       " a,sum(b) from PreAggMain group by a")
     intercept[RuntimeException] {
-      sql("alter table preagg1 rename to preagg2")
+      sql("alter table PreAggMain_preagg1 rename to preagg2")
     }.getMessage.contains("Rename operation for pre-aggregate table is not supported.")
     intercept[RuntimeException] {
       sql("alter table preaggmain rename to preaggmain_new")

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1c16afad/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/ChangeDataTypeTestCases.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/ChangeDataTypeTestCases.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/ChangeDataTypeTestCases.scala
index 67ea21e..31a8b6c 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/ChangeDataTypeTestCases.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/ChangeDataTypeTestCases.scala
@@ -149,19 +149,19 @@ class ChangeDataTypeTestCases extends Spark2QueryTest with BeforeAndAfterAll {
 
   test("test data type change for with pre-aggregate table should throw exception") {
     sql("drop table if exists preaggMain")
-    sql("drop table if exists preagg1")
+    sql("drop table if exists PreAggMain_preagg1")
     sql("create table preaggMain (a string, b string, c string) stored by 'carbondata'")
     sql(
-      "create table preagg1 stored BY 'carbondata' tblproperties('parent'='PreAggMain') as select" +
+      "create datamap preagg1 on table PreAggMain using 'preaggregate' as select" +
       " a,sum(b) from PreAggMain group by a")
     intercept[RuntimeException] {
       sql("alter table preaggmain drop columns(a)").show
     }.getMessage.contains("exists in pre-aggregate table")
     intercept[RuntimeException] {
-      sql("alter table preagg1 drop columns(a)").show
+      sql("alter table PreAggMain_preagg1 drop columns(a)").show
     }.getMessage.contains("cannot be dropped")
     sql("drop table if exists preaggMain")
-    sql("drop table if exists preagg1")
+    sql("drop table if exists PreAggMain_preagg1")
   }
 
   override def afterAll {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1c16afad/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/DropColumnTestCases.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/DropColumnTestCases.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/DropColumnTestCases.scala
index 1a1d5d5..b887771 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/DropColumnTestCases.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/DropColumnTestCases.scala
@@ -100,10 +100,10 @@ class DropColumnTestCases extends Spark2QueryTest with BeforeAndAfterAll {
 
   test("test dropping of column in pre-aggregate should throw exception") {
     sql("drop table if exists preaggMain")
-    sql("drop table if exists preagg1")
+    sql("drop table if exists preaggMain_preagg1")
     sql("create table preaggMain (a string, b string, c string) stored by 'carbondata'")
     sql(
-      "create table preagg1 stored BY 'carbondata' tblproperties('parent'='PreAggMain') as select" +
+      "create datamap preagg1 on table PreAggMain using 'preaggregate' as select" +
       " a,sum(b) from PreAggMain group by a")
     sql("alter table preaggmain drop columns(c)").show
     checkExistence(sql("desc table preaggmain"), false, "c")
@@ -111,7 +111,7 @@ class DropColumnTestCases extends Spark2QueryTest with BeforeAndAfterAll {
       sql("alter table preaggmain drop columns(a)").show
     }.getMessage.contains("cannot be dropped")
     sql("drop table if exists preaggMain")
-    sql("drop table if exists preagg1")
+    sql("drop table if exists preaggMain_preagg1")
   }
 
   override def afterAll {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1c16afad/integration/spark2/src/test/scala/org/apache/spark/util/CarbonCommandSuite.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/util/CarbonCommandSuite.scala b/integration/spark2/src/test/scala/org/apache/spark/util/CarbonCommandSuite.scala
index c65bcc4..f79ea2d 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/util/CarbonCommandSuite.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/util/CarbonCommandSuite.scala
@@ -143,14 +143,14 @@ class CarbonCommandSuite extends Spark2QueryTest with BeforeAndAfterAll {
 
   test("test if delete segments by id is unsupported for pre-aggregate tables") {
     dropTable("preaggMain")
-    dropTable("preagg1")
+    dropTable("preaggMain_preagg1")
     sql("create table preaggMain (a string, b string, c string) stored by 'carbondata'")
-    sql("create table preagg1 stored BY 'carbondata' tblproperties('parent'='PreAggMain') as select a,sum(b) from PreAggMain group by a")
+    sql("create datamap preagg1 on table PreAggMain using 'preaggregate' as select a,sum(b) from PreAggMain group by a")
     intercept[UnsupportedOperationException] {
       sql("delete from table preaggMain where segment.id in (1,2)")
     }.getMessage.contains("Delete segment operation is not supported on tables")
     intercept[UnsupportedOperationException] {
-      sql("delete from table preagg1 where segment.id in (1,2)")
+      sql("delete from table preaggMain_preagg1 where segment.id in (1,2)")
     }.getMessage.contains("Delete segment operation is not supported on pre-aggregate tables")
     dropTable("preaggMain")
     dropTable("preagg1")


[42/49] 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/f7f516ef
Tree: http://git-wip-us.apache.org/repos/asf/carbondata/tree/f7f516ef
Diff: http://git-wip-us.apache.org/repos/asf/carbondata/diff/f7f516ef

Branch: refs/heads/fgdatamap
Commit: f7f516ef665c98e43fce0427c40da933bb6f3185
Parents: 3d1d1ce
Author: kunal642 <ku...@gmail.com>
Authored: Wed Oct 18 20:09:04 2017 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Tue Nov 14 00:48:16 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/f7f516ef/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/f7f516ef/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/f7f516ef/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/f7f516ef/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/f7f516ef/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/f7f516ef/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/f7f516ef/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/f7f516ef/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/f7f516ef/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/f7f516ef/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/f7f516ef/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/f7f516ef/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 = {


[32/49] 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);
       }


[06/49] carbondata git commit: [CARBONDATA-1594] Add precision and scale to DecimalType

Posted by ra...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/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 e5cfc84..285abf4 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
@@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.metadata.CarbonTableIdentifier
-import org.apache.carbondata.core.metadata.datatype.{DataType, DataTypes}
+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}
@@ -207,16 +207,15 @@ class AlterTableColumnSchemaGenerator(
     alterTableModel.dimCols.foreach(field => {
       val encoders = new java.util.ArrayList[Encoding]()
       encoders.add(Encoding.DICTIONARY)
-      val columnSchema: ColumnSchema = getColumnSchema(
-        DataTypeConverterUtil.convertToCarbonType(field.dataType.getOrElse("")),
-        field.name.getOrElse(field.column),
-        isCol = true,
+      val columnSchema: ColumnSchema = TableNewProcessor.createColumnSchema(
+        field,
         encoders,
         isDimensionCol = true,
-        -1,
         field.precision,
         field.scale,
         field.schemaOrdinal + existingColsSize,
+        alterTableModel.highCardinalityDims,
+        alterTableModel.databaseName.getOrElse(dbName),
         isSortColumn(field.name.getOrElse(field.column)))
       allColumns ++= Seq(columnSchema)
       newCols ++= Seq(columnSchema)
@@ -225,17 +224,16 @@ class AlterTableColumnSchemaGenerator(
     allColumns ++= tableCols.filter(x => !x.isDimensionColumn)
     alterTableModel.msrCols.foreach(field => {
       val encoders = new java.util.ArrayList[Encoding]()
-      val columnSchema: ColumnSchema = getColumnSchema(
-        DataTypeConverterUtil.convertToCarbonType(field.dataType.getOrElse("")),
-        field.name.getOrElse(field.column),
-        isCol = true,
+      val columnSchema: ColumnSchema = TableNewProcessor.createColumnSchema(
+        field,
         encoders,
         isDimensionCol = false,
-        -1,
         field.precision,
         field.scale,
         field.schemaOrdinal + existingColsSize,
-        false)
+        alterTableModel.highCardinalityDims,
+        alterTableModel.databaseName.getOrElse(dbName)
+      )
       allColumns ++= Seq(columnSchema)
       newCols ++= Seq(columnSchema)
     })
@@ -299,33 +297,49 @@ class AlterTableColumnSchemaGenerator(
     newCols
   }
 
-  private def getColumnSchema(dataType: DataType, colName: String, isCol: Boolean,
-      encoders: java.util.List[Encoding], isDimensionCol: Boolean,
-      colGroup: Integer, precision: Integer, scale: Integer, schemaOrdinal: Int,
-      isSortColumn: Boolean): ColumnSchema = {
+}
+
+// TODO: move this to carbon store API
+object TableNewProcessor {
+  def apply(cm: TableModel): TableInfo = {
+    new TableNewProcessor(cm).process
+  }
+
+  def createColumnSchema(
+      field: Field,
+      encoders: java.util.List[Encoding],
+      isDimensionCol: Boolean,
+      precision: Int,
+      scale: Int,
+      schemaOrdinal: Int,
+      highCardinalityDims: Seq[String],
+      databaseName: String,
+      isSortColumn: Boolean = false): ColumnSchema = {
+    val dataType = DataTypeConverterUtil.convertToCarbonType(field.dataType.getOrElse(""))
+    if (DataTypes.isDecimal(dataType)) {
+      dataType.asInstanceOf[DecimalType].setPrecision(field.precision)
+      dataType.asInstanceOf[DecimalType].setScale(field.scale)
+    }
     val columnSchema = new ColumnSchema()
     columnSchema.setDataType(dataType)
+    val colName = field.name.getOrElse(field.column)
     columnSchema.setColumnName(colName)
-    if (alterTableModel.highCardinalityDims.contains(colName)) {
+    if (highCardinalityDims.contains(colName)) {
       encoders.remove(Encoding.DICTIONARY)
     }
     if (dataType == DataTypes.DATE) {
       encoders.add(Encoding.DIRECT_DICTIONARY)
     }
-    if (dataType == DataTypes.TIMESTAMP && !alterTableModel.highCardinalityDims.contains(colName)) {
+    if (dataType == DataTypes.TIMESTAMP && ! highCardinalityDims.contains(colName)) {
       encoders.add(Encoding.DIRECT_DICTIONARY)
     }
-    val colPropMap = new java.util.HashMap[String, String]()
     columnSchema.setEncodingList(encoders)
     val colUniqueIdGenerator = CarbonCommonFactory.getColumnUniqueIdGenerator
-    val columnUniqueId = colUniqueIdGenerator.generateUniqueId(
-      alterTableModel.databaseName.getOrElse(dbName),
-      columnSchema)
+    val columnUniqueId = colUniqueIdGenerator.generateUniqueId(databaseName, columnSchema)
     columnSchema.setColumnUniqueId(columnUniqueId)
     columnSchema.setColumnReferenceId(columnUniqueId)
-    columnSchema.setColumnar(isCol)
+    columnSchema.setColumnar(true)
     columnSchema.setDimensionColumn(isDimensionCol)
-    columnSchema.setColumnGroup(colGroup)
     columnSchema.setPrecision(precision)
     columnSchema.setScale(scale)
     columnSchema.setSchemaOrdinal(schemaOrdinal)
@@ -334,31 +348,25 @@ class AlterTableColumnSchemaGenerator(
     columnSchema
   }
 }
-object TableNewProcessor {
-  def apply(cm: TableModel): TableInfo = {
-    new TableNewProcessor(cm).process
-  }
-}
 
 class TableNewProcessor(cm: TableModel) {
 
-  var index = 0
-  var rowGroup = 0
-
   def getAllChildren(fieldChildren: Option[List[Field]]): Seq[ColumnSchema] = {
     var allColumns: Seq[ColumnSchema] = Seq[ColumnSchema]()
     fieldChildren.foreach(fields => {
       fields.foreach(field => {
         val encoders = new java.util.ArrayList[Encoding]()
         encoders.add(Encoding.DICTIONARY)
-        val columnSchema: ColumnSchema = getColumnSchema(
-          DataTypeConverterUtil.convertToCarbonType(field.dataType.getOrElse("")),
-          field.name.getOrElse(field.column), index,
-          isCol = true, encoders, isDimensionCol = true, rowGroup, field.precision, field.scale,
-          field.schemaOrdinal)
+        val columnSchema: ColumnSchema = TableNewProcessor.createColumnSchema(
+          field,
+          encoders,
+          isDimensionCol = true,
+          field.precision,
+          field.scale,
+          field.schemaOrdinal,
+          cm.highcardinalitydims.getOrElse(Seq()),
+          cm.databaseName)
         allColumns ++= Seq(columnSchema)
-        index = index + 1
-        rowGroup = rowGroup + 1
         if (field.children.get != null) {
           columnSchema.setNumberOfChild(field.children.get.size)
           allColumns ++= getAllChildren(field.children)
@@ -368,39 +376,6 @@ class TableNewProcessor(cm: TableModel) {
     allColumns
   }
 
-  def getColumnSchema(dataType: DataType, colName: String, index: Integer, isCol: Boolean,
-      encoders: java.util.List[Encoding], isDimensionCol: Boolean,
-      colGroup: Integer, precision: Integer, scale: Integer, schemaOrdinal: Int): ColumnSchema = {
-    val columnSchema = new ColumnSchema()
-    columnSchema.setDataType(dataType)
-    columnSchema.setColumnName(colName)
-    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.setColumnar(isCol)
-    columnSchema.setDimensionColumn(isDimensionCol)
-    columnSchema.setColumnGroup(colGroup)
-    columnSchema.setPrecision(precision)
-    columnSchema.setScale(scale)
-    columnSchema.setSchemaOrdinal(schemaOrdinal)
-    columnSchema.setSortColumn(false)
-    // 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 = {
@@ -414,38 +389,34 @@ class TableNewProcessor(cm: TableModel) {
       val field = cm.dimCols.find(keyDim equals _.column).get
       val encoders = new java.util.ArrayList[Encoding]()
       encoders.add(Encoding.DICTIONARY)
-      val columnSchema: ColumnSchema = getColumnSchema(
-        DataTypeConverterUtil.convertToCarbonType(field.dataType.getOrElse("")),
-        field.name.getOrElse(field.column),
-        index,
-        isCol = true,
+      val columnSchema: ColumnSchema = TableNewProcessor.createColumnSchema(
+        field,
         encoders,
         isDimensionCol = true,
-        -1,
         field.precision,
         field.scale,
-        field.schemaOrdinal)
+        field.schemaOrdinal,
+        cm.highcardinalitydims.getOrElse(Seq()),
+        cm.databaseName)
       columnSchema.setSortColumn(true)
       allColumns :+= columnSchema
       index = index + 1
     }
 
-    cm.dimCols.foreach(field => {
+    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 = getColumnSchema(
-          DataTypeConverterUtil.convertToCarbonType(field.dataType.getOrElse("")),
-          field.name.getOrElse(field.column),
-          index,
-          isCol = true,
+        val columnSchema: ColumnSchema = TableNewProcessor.createColumnSchema(
+          field,
           encoders,
           isDimensionCol = true,
-          -1,
           field.precision,
           field.scale,
-          field.schemaOrdinal)
+          field.schemaOrdinal,
+          cm.highcardinalitydims.getOrElse(Seq()),
+          cm.databaseName)
         allColumns :+= columnSchema
         index = index + 1
         if (field.children.isDefined && field.children.get != null) {
@@ -453,37 +424,37 @@ class TableNewProcessor(cm: TableModel) {
           allColumns ++= getAllChildren(field.children)
         }
       }
-    })
+    }
 
-    cm.msrCols.foreach(field => {
+    cm.msrCols.foreach { field =>
       val encoders = new java.util.ArrayList[Encoding]()
-      val columnSchema: ColumnSchema = getColumnSchema(
-        DataTypeConverterUtil.convertToCarbonType(field.dataType.getOrElse("")),
-        field.name.getOrElse(field.column),
-        index,
-        isCol = true,
+      val columnSchema: ColumnSchema = TableNewProcessor.createColumnSchema(
+        field,
         encoders,
         isDimensionCol = false,
-        -1,
         field.precision,
         field.scale,
-        field.schemaOrdinal)
+        field.schemaOrdinal,
+        cm.highcardinalitydims.getOrElse(Seq()),
+        cm.databaseName)
       allColumns :+= columnSchema
       index = index + 1
       measureCount += 1
-    })
+    }
 
     // Check if there is any duplicate measures or dimensions.
     // Its based on the dimension name and measure name
-    allColumns.groupBy(_.getColumnName).foreach(f => if (f._2.size > 1) {
-      val name = f._1
-      LOGGER.error(s"Duplicate column found with name: $name")
-      LOGGER.audit(
-        s"Validation failed for Create/Alter Table Operation " +
-            s"for ${ cm.databaseName }.${ cm.tableName }" +
-            s"Duplicate column found with name: $name")
-      sys.error(s"Duplicate dimensions found with name: $name")
-    })
+    allColumns.groupBy(_.getColumnName).foreach { f =>
+      if (f._2.size > 1) {
+        val name = f._1
+        LOGGER.error(s"Duplicate column found with name: $name")
+        LOGGER.audit(
+          s"Validation failed for Create/Alter Table Operation " +
+          s"for ${ cm.databaseName }.${ cm.tableName }" +
+          s"Duplicate column found with name: $name")
+        sys.error(s"Duplicate dimensions found with name: $name")
+      }
+    }
 
     val highCardinalityDims = cm.highcardinalitydims.getOrElse(Seq())
 
@@ -508,13 +479,21 @@ class TableNewProcessor(cm: TableModel) {
     // Adding dummy measure if no measure is provided
     if (measureCount == 0) {
       val encoders = new java.util.ArrayList[Encoding]()
-      val columnSchema: ColumnSchema = getColumnSchema(DataTypes.DOUBLE,
+      val field = Field(
         CarbonCommonConstants.DEFAULT_INVISIBLE_DUMMY_MEASURE,
-        index,
-        true,
+        Some(DataTypes.DOUBLE.getName),
+        Some(CarbonCommonConstants.DEFAULT_INVISIBLE_DUMMY_MEASURE),
+        None
+      )
+      val columnSchema: ColumnSchema = TableNewProcessor.createColumnSchema(
+        field,
         encoders,
-        false,
-        -1, 0, 0, schemaOrdinal = -1)
+        isDimensionCol = false,
+        field.precision,
+        field.scale,
+        -1,
+        cm.highcardinalitydims.getOrElse(Seq()),
+        cm.databaseName)
       columnSchema.setInvisible(true)
       allColumns :+= columnSchema
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java b/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java
index 1d3783e..64b440b 100644
--- a/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java
+++ b/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java
@@ -211,7 +211,7 @@ class VectorizedCarbonRecordReader extends AbstractRecordReader<Object> {
         fields[msr.getQueryOrder()] = new StructField(msr.getColumnName(),
             CarbonScalaUtil.convertCarbonToSparkDataType(msr.getMeasure().getDataType()), true,
             null);
-      } else if (dataType == DataTypes.DECIMAL) {
+      } else if (DataTypes.isDecimal(dataType)) {
         fields[msr.getQueryOrder()] = new StructField(msr.getColumnName(),
             new DecimalType(msr.getMeasure().getPrecision(), msr.getMeasure().getScale()), true,
             null);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDataFrameWriter.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDataFrameWriter.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDataFrameWriter.scala
index 98a37fa..44fbb37 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDataFrameWriter.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDataFrameWriter.scala
@@ -151,8 +151,7 @@ class CarbonDataFrameWriter(sqlContext: SQLContext, val dataFrame: DataFrame) {
       case DoubleType => CarbonType.DOUBLE.getName
       case TimestampType => CarbonType.TIMESTAMP.getName
       case DateType => CarbonType.DATE.getName
-      case decimal: DecimalType => s"${CarbonType.DECIMAL.getName} (${decimal.precision}" +
-                                   s", ${decimal.scale})"
+      case decimal: DecimalType => s"decimal(${decimal.precision}, ${decimal.scale})"
       case other => sys.error(s"unsupported type: $other")
     }
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/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 ef89771..91c07de 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
@@ -164,46 +164,49 @@ case class CarbonDictionaryDecoder(
              |}
              """.stripMargin
 
-            val caseCode = getDictionaryColumnIds(index)._3.getDataType match {
-              case CarbonDataTypes.INT =>
-                s"""
-                   |int $value = Integer.parseInt(new String($valueIntern,
-                   |org.apache.carbondata.core.constants.CarbonCommonConstants
-                   |.DEFAULT_CHARSET_CLASS));
-                 """.stripMargin
-              case CarbonDataTypes.SHORT =>
-                s"""
-                   |short $value =
-                   |Short.parseShort(new String($valueIntern,
-                   |org.apache.carbondata.core.constants.CarbonCommonConstants
-                   |.DEFAULT_CHARSET_CLASS));
-                 """.stripMargin
-              case CarbonDataTypes.DOUBLE =>
-                s"""
-                   |double $value =
-                   |Double.parseDouble(new String($valueIntern,
-                   |org.apache.carbondata.core.constants.CarbonCommonConstants
-                   |.DEFAULT_CHARSET_CLASS));
-                 """.stripMargin
-              case CarbonDataTypes.LONG =>
-                s"""
-                   |long $value =
-                   |Long.parseLong(new String($valueIntern,
-                   |org.apache.carbondata.core.constants.CarbonCommonConstants
-                   |.DEFAULT_CHARSET_CLASS));
-                 """.stripMargin
-              case CarbonDataTypes.DECIMAL =>
+            val caseCode =
+              if (CarbonDataTypes.isDecimal(getDictionaryColumnIds(index)._3.getDataType)) {
                 s"""
                    |org.apache.spark.sql.types.Decimal $value =
                    |Decimal.apply(new java.math.BigDecimal(
                    |new String($valueIntern, org.apache.carbondata.core.constants
                    |.CarbonCommonConstants.DEFAULT_CHARSET_CLASS)));
                  """.stripMargin
-              case _ =>
-                s"""
-                   | UTF8String $value = UTF8String.fromBytes($valueIntern);
+              } else {
+                getDictionaryColumnIds(index)._3.getDataType match {
+                  case CarbonDataTypes.INT =>
+                    s"""
+                       |int $value = Integer.parseInt(new String($valueIntern,
+                       |org.apache.carbondata.core.constants.CarbonCommonConstants
+                       |.DEFAULT_CHARSET_CLASS));
                  """.stripMargin
-            }
+                  case CarbonDataTypes.SHORT =>
+                    s"""
+                       |short $value =
+                       |Short.parseShort(new String($valueIntern,
+                       |org.apache.carbondata.core.constants.CarbonCommonConstants
+                       |.DEFAULT_CHARSET_CLASS));
+                 """.stripMargin
+                  case CarbonDataTypes.DOUBLE =>
+                    s"""
+                       |double $value =
+                       |Double.parseDouble(new String($valueIntern,
+                       |org.apache.carbondata.core.constants.CarbonCommonConstants
+                       |.DEFAULT_CHARSET_CLASS));
+                 """.stripMargin
+                  case CarbonDataTypes.LONG =>
+                    s"""
+                       |long $value =
+                       |Long.parseLong(new String($valueIntern,
+                       |org.apache.carbondata.core.constants.CarbonCommonConstants
+                       |.DEFAULT_CHARSET_CLASS));
+                 """.stripMargin
+                  case _ =>
+                    s"""
+                       | UTF8String $value = UTF8String.fromBytes($valueIntern);
+                 """.stripMargin
+                }
+              }
           code +=
             s"""
                |$caseCode
@@ -381,29 +384,31 @@ object CarbonDictionaryDecoder {
    */
   def convertCarbonToSparkDataType(carbonDimension: CarbonDimension,
       relation: CarbonRelation): types.DataType = {
-    carbonDimension.getDataType match {
-      case CarbonDataTypes.STRING => StringType
-      case CarbonDataTypes.SHORT => ShortType
-      case CarbonDataTypes.INT => IntegerType
-      case CarbonDataTypes.LONG => LongType
-      case CarbonDataTypes.DOUBLE => DoubleType
-      case CarbonDataTypes.BOOLEAN => BooleanType
-      case CarbonDataTypes.DECIMAL =>
-        val scale: Int = carbonDimension.getColumnSchema.getScale
-        val precision: Int = carbonDimension.getColumnSchema.getPrecision
-        if (scale == 0 && precision == 0) {
-          DecimalType(18, 2)
-        } else {
-          DecimalType(precision, scale)
-        }
-      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) }>")
+    if (CarbonDataTypes.isDecimal(carbonDimension.getDataType)) {
+      val scale: Int = carbonDimension.getColumnSchema.getScale
+      val precision: Int = carbonDimension.getColumnSchema.getPrecision
+      if (scale == 0 && precision == 0) {
+        DecimalType(18, 2)
+      } else {
+        DecimalType(precision, scale)
+      }
+    } else {
+      carbonDimension.getDataType match {
+        case CarbonDataTypes.STRING => StringType
+        case CarbonDataTypes.SHORT => ShortType
+        case CarbonDataTypes.INT => IntegerType
+        case CarbonDataTypes.LONG => LongType
+        case CarbonDataTypes.DOUBLE => DoubleType
+        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) }>")
+      }
     }
   }
 
@@ -464,34 +469,6 @@ class CarbonDecoderRDD(
     }
   }
 
-  def convertCarbonToSparkDataType(carbonDimension: CarbonDimension,
-      relation: CarbonRelation): types.DataType = {
-    carbonDimension.getDataType match {
-      case CarbonDataTypes.STRING => StringType
-      case CarbonDataTypes.SHORT => ShortType
-      case CarbonDataTypes.INT => IntegerType
-      case CarbonDataTypes.LONG => LongType
-      case CarbonDataTypes.DOUBLE => DoubleType
-      case CarbonDataTypes.BOOLEAN => BooleanType
-      case CarbonDataTypes.DECIMAL =>
-        val scale: Int = carbonDimension.getColumnSchema.getScale
-        val precision: Int = carbonDimension.getColumnSchema.getPrecision
-        if (scale == 0 && precision == 0) {
-          DecimalType(18, 2)
-        } else {
-          DecimalType(precision, scale)
-        }
-      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) }>")
-    }
-  }
-
   val getDictionaryColumnIds = {
     val dictIds: Array[(String, ColumnIdentifier, CarbonDimension)] = output.map { a =>
       val attr = aliasMap.getOrElse(a, a)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonRelation.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonRelation.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonRelation.scala
index 6bac0da..2c476ed 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonRelation.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonRelation.scala
@@ -16,7 +16,6 @@
  */
 package org.apache.spark.sql.hive
 
-import java.util
 import java.util.LinkedHashSet
 
 import scala.Array.canBuildFrom
@@ -29,7 +28,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Stati
 import org.apache.spark.sql.types._
 
 import org.apache.carbondata.core.datastore.impl.FileFactory
-import org.apache.carbondata.core.metadata.datatype.DataTypes.DECIMAL
+import org.apache.carbondata.core.metadata.datatype.DataTypes
 import org.apache.carbondata.core.metadata.schema.table.column.{CarbonColumn, CarbonDimension}
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager
 import org.apache.carbondata.core.util.path.CarbonStorePath
@@ -164,7 +163,7 @@ case class CarbonRelation(
 
   def addDecimalScaleAndPrecision(dimval: CarbonColumn, dataType: String): String = {
     var dType = dataType
-    if (dimval.getDataType == DECIMAL) {
+    if (DataTypes.isDecimal(dimval.getDataType)) {
       dType +=
       "(" + dimval.getColumnSchema.getPrecision + "," + dimval.getColumnSchema.getScale + ")"
     }
@@ -184,7 +183,7 @@ case class CarbonRelation(
 
   def addDecimalScaleAndPrecision(dimval: CarbonDimension, dataType: String): String = {
     var dType = dataType
-    if (dimval.getDataType == DECIMAL) {
+    if (DataTypes.isDecimal(dimval.getDataType)) {
       dType +=
       "(" + dimval.getColumnSchema.getPrecision + "," + dimval.getColumnSchema.getScale + ")"
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/processing/src/main/java/org/apache/carbondata/processing/loading/partition/impl/HashPartitionerImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/partition/impl/HashPartitionerImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/partition/impl/HashPartitionerImpl.java
index a41f734..f24d24f 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/partition/impl/HashPartitionerImpl.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/partition/impl/HashPartitionerImpl.java
@@ -42,7 +42,7 @@ public class HashPartitionerImpl implements Partitioner<Object[]> {
       if (dataType == DataTypes.SHORT || dataType == DataTypes.INT || dataType == DataTypes.LONG) {
         hashes[i] = new IntegralHash(indexes.get(i));
       } else if (dataType == DataTypes.DOUBLE || dataType == DataTypes.FLOAT ||
-          dataType == DataTypes.DECIMAL) {
+          DataTypes.isDecimal(dataType)) {
         hashes[i] = new DecimalHash(indexes.get(i));
       } else {
         hashes[i] = new StringHash(indexes.get(i));

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeCarbonRowPage.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeCarbonRowPage.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeCarbonRowPage.java
index 1d8f941..e5583c2 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeCarbonRowPage.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeCarbonRowPage.java
@@ -147,7 +147,7 @@ public class UnsafeCarbonRowPage {
           Double doubleVal = (Double) value;
           CarbonUnsafe.getUnsafe().putDouble(baseObject, address + size, doubleVal);
           size += 8;
-        } else if (dataType == DataTypes.DECIMAL) {
+        } else if (DataTypes.isDecimal(dataType)) {
           BigDecimal decimalVal = (BigDecimal) value;
           byte[] bigDecimalInBytes = DataTypeUtil.bigDecimalToByte(decimalVal);
           CarbonUnsafe.getUnsafe()
@@ -233,7 +233,7 @@ public class UnsafeCarbonRowPage {
           Double doubleVal = CarbonUnsafe.getUnsafe().getDouble(baseObject, address + size);
           size += 8;
           rowToFill[dimensionSize + mesCount] = doubleVal;
-        } else if (dataType == DataTypes.DECIMAL) {
+        } else if (DataTypes.isDecimal(dataType)) {
           short aShort = CarbonUnsafe.getUnsafe().getShort(baseObject, address + size);
           byte[] bigDecimalInBytes = new byte[aShort];
           size += 2;
@@ -315,7 +315,7 @@ public class UnsafeCarbonRowPage {
           double doubleVal = CarbonUnsafe.getUnsafe().getDouble(baseObject, address + size);
           size += 8;
           stream.writeDouble(doubleVal);
-        } else if (dataType == DataTypes.DECIMAL) {
+        } else if (DataTypes.isDecimal(dataType)) {
           short aShort = CarbonUnsafe.getUnsafe().getShort(baseObject, address + size);
           byte[] bigDecimalInBytes = new byte[aShort];
           size += 2;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java
index 404a521..3972b1c 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java
@@ -335,7 +335,7 @@ public class UnsafeSortTempFileChunkHolder implements SortTempChunkHolder {
             row[dimensionCount + mesCount] = stream.readLong();
           } else if (dataType == DataTypes.DOUBLE) {
             row[dimensionCount + mesCount] = stream.readDouble();
-          } else if (dataType == DataTypes.DECIMAL) {
+          } else if (DataTypes.isDecimal(dataType)) {
             short aShort = stream.readShort();
             byte[] bigDecimalInBytes = new byte[aShort];
             stream.readFully(bigDecimalInBytes);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeIntermediateFileMerger.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeIntermediateFileMerger.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeIntermediateFileMerger.java
index 9f7d6c3..0c71adc 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeIntermediateFileMerger.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeIntermediateFileMerger.java
@@ -329,7 +329,7 @@ public class UnsafeIntermediateFileMerger implements Callable<Void> {
         } else if (dataType == DataTypes.DOUBLE) {
           rowData.putDouble(size, (Double) value);
           size += 8;
-        } else if (dataType == DataTypes.DECIMAL) {
+        } else if (DataTypes.isDecimal(dataType)) {
           byte[] bigDecimalInBytes = (byte[]) value;
           rowData.putShort(size, (short) bigDecimalInBytes.length);
           size += 2;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java b/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java
index a361f3a..187ba06 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java
@@ -257,7 +257,7 @@ public class CompactionResultSortProcessor extends AbstractResultProcessor {
    * @return
    */
   private Object getConvertedMeasureValue(Object value, DataType type) {
-    if (type == DataTypes.DECIMAL) {
+    if (DataTypes.isDecimal(type)) {
       if (value != null) {
         value = ((Decimal) value).toJavaBigDecimal();
       }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/IntermediateFileMerger.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/IntermediateFileMerger.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/IntermediateFileMerger.java
index ebc811c..266e69a 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/IntermediateFileMerger.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/IntermediateFileMerger.java
@@ -357,7 +357,7 @@ public class IntermediateFileMerger implements Callable<Void> {
             stream.writeLong((long) NonDictionaryUtil.getMeasure(fieldIndex, row));
           } else if (dataType == DataTypes.DOUBLE) {
             stream.writeDouble((Double) NonDictionaryUtil.getMeasure(fieldIndex, row));
-          } else if (dataType == DataTypes.DECIMAL) {
+          } else if (DataTypes.isDecimal(dataType)) {
             byte[] bigDecimalInBytes = (byte[]) NonDictionaryUtil.getMeasure(fieldIndex, row);
             stream.writeInt(bigDecimalInBytes.length);
             stream.write(bigDecimalInBytes);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortDataRows.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortDataRows.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortDataRows.java
index 8a60657..5b9e091 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortDataRows.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortDataRows.java
@@ -305,7 +305,7 @@ public class SortDataRows {
               stream.writeLong((Long) value);
             } else if (dataType == DataTypes.DOUBLE) {
               stream.writeDouble((Double) value);
-            } else if (dataType == DataTypes.DECIMAL) {
+            } else if (DataTypes.isDecimal(dataType)) {
               BigDecimal val = (BigDecimal) value;
               byte[] bigDecimalInBytes = DataTypeUtil.bigDecimalToByte(val);
               stream.writeInt(bigDecimalInBytes.length);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortTempFileChunkHolder.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortTempFileChunkHolder.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortTempFileChunkHolder.java
index 91bc83c..2f87cf7 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortTempFileChunkHolder.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortTempFileChunkHolder.java
@@ -355,7 +355,7 @@ public class SortTempFileChunkHolder implements Comparable<SortTempFileChunkHold
             measures[index++] = stream.readLong();
           } else if (dataType == DataTypes.DOUBLE) {
             measures[index++] = stream.readDouble();
-          } else if (dataType == DataTypes.DECIMAL) {
+          } else if (DataTypes.isDecimal(dataType)) {
             int len = stream.readInt();
             byte[] buff = new byte[len];
             stream.readFully(buff);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerColumnar.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerColumnar.java b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerColumnar.java
index fdf44cf..7882cd4 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerColumnar.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerColumnar.java
@@ -504,7 +504,7 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
         new ArrayList<Integer>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
     DataType[] type = model.getMeasureDataType();
     for (int j = 0; j < type.length; j++) {
-      if (type[j] != DataTypes.BYTE && type[j] != DataTypes.DECIMAL) {
+      if (type[j] != DataTypes.BYTE && !DataTypes.isDecimal(type[j])) {
         otherMeasureIndexList.add(j);
       } else {
         customMeasureIndexList.add(j);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/processing/src/main/java/org/apache/carbondata/processing/store/TablePage.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/TablePage.java b/processing/src/main/java/org/apache/carbondata/processing/store/TablePage.java
index d2cf1c4..6a9aba1 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/store/TablePage.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/store/TablePage.java
@@ -114,14 +114,13 @@ public class TablePage {
     for (int i = 0; i < measurePages.length; i++) {
       TableSpec.MeasureSpec spec = model.getTableSpec().getMeasureSpec(i);
       ColumnPage page;
-      if (spec.getSchemaDataType() == DataTypes.DECIMAL) {
+      if (DataTypes.isDecimal(spec.getSchemaDataType())) {
         page = ColumnPage.newDecimalPage(spec, dataTypes[i], pageSize);
       } else {
         page = ColumnPage.newPage(spec, dataTypes[i], pageSize);
       }
       page.setStatsCollector(
-          PrimitivePageStatsCollector.newInstance(
-              dataTypes[i], spec.getScale(), spec.getPrecision()));
+          PrimitivePageStatsCollector.newInstance(dataTypes[i]));
       measurePages[i] = page;
     }
     boolean hasNoDictionary = noDictDimensionPages.length > 0;
@@ -183,7 +182,7 @@ public class TablePage {
 
       // in compaction flow the measure with decimal type will come as Spark decimal.
       // need to convert it to byte array.
-      if (measurePages[i].getDataType() == DataTypes.DECIMAL &&
+      if (DataTypes.isDecimal(measurePages[i].getDataType()) &&
           model.isCompactionFlow() &&
           value != null) {
         value = ((Decimal) value).toJavaBigDecimal();


[09/49] carbondata git commit: [Tests] Fix BTreeBlockFinderTest variable mistake

Posted by ra...@apache.org.
[Tests] Fix BTreeBlockFinderTest variable mistake

There are some obvious spelling mistake leading to some variable unused, and it may change the unit test's meaning

This closes #1462


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

Branch: refs/heads/fgdatamap
Commit: f812e41c5333ad54449d3b514becd0fddb9c5024
Parents: f209e8e
Author: lishuming <al...@126.com>
Authored: Thu Nov 2 23:47:05 2017 +0800
Committer: chenliang613 <ch...@huawei.com>
Committed: Sat Nov 4 15:29:34 2017 +0800

----------------------------------------------------------------------
 .../core/datastore/impl/btree/BTreeBlockFinderTest.java        | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/f812e41c/core/src/test/java/org/apache/carbondata/core/datastore/impl/btree/BTreeBlockFinderTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/datastore/impl/btree/BTreeBlockFinderTest.java b/core/src/test/java/org/apache/carbondata/core/datastore/impl/btree/BTreeBlockFinderTest.java
index d6081a3..d874037 100644
--- a/core/src/test/java/org/apache/carbondata/core/datastore/impl/btree/BTreeBlockFinderTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/datastore/impl/btree/BTreeBlockFinderTest.java
@@ -197,7 +197,7 @@ public class BTreeBlockFinderTest extends TestCase {
         buffer1.put((byte) 1);
         buffer1.putInt(i + 10);
         buffer1.array();
-        byte[] noDictionaryEndKey = buffer.array();
+        byte[] noDictionaryEndKey = buffer1.array();
         DataFileFooter footer =
             getFileFooter(startKey, endKey, noDictionaryStartKey, noDictionaryEndKey);
         list.add(footer);
@@ -232,7 +232,7 @@ public class BTreeBlockFinderTest extends TestCase {
         buffer1.putShort((short) 2);
         buffer1.putInt(i + 10);
         buffer1.array();
-        byte[] noDictionaryEndKey = buffer.array();
+        byte[] noDictionaryEndKey = buffer1.array();
         DataFileFooter footer =
             getFileMatadataWithOnlyNoDictionaryKey(startKey, endKey, noDictionaryStartKey,
                 noDictionaryEndKey);
@@ -268,7 +268,7 @@ public class BTreeBlockFinderTest extends TestCase {
         buffer1.put((byte) 1);
         buffer1.putInt(i + 10);
         buffer1.array();
-        byte[] noDictionaryEndKey = buffer.array();
+        byte[] noDictionaryEndKey = buffer1.array();
         DataFileFooter footer =
             getFileFooterWithOnlyDictionaryKey(startKey, endKey, noDictionaryStartKey,
                 noDictionaryEndKey);


[46/49] 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/31699185
Tree: http://git-wip-us.apache.org/repos/asf/carbondata/tree/31699185
Diff: http://git-wip-us.apache.org/repos/asf/carbondata/diff/31699185

Branch: refs/heads/fgdatamap
Commit: 3169918523a42c8f2c1d612052a7680514372bf9
Parents: 40c31e8
Author: kumarvishal <ku...@gmail.com>
Authored: Sat Oct 21 16:31:15 2017 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Tue Nov 14 00:48:16 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/31699185/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/31699185/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/31699185/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/31699185/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/31699185/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/31699185/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/31699185/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/31699185/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/31699185/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/31699185/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
 }


[13/49] carbondata git commit: [CARBONDATA-1669] Clean up code in CarbonDataRDDFactory

Posted by ra...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/0578ba0f/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/AlterTableDropCarbonPartitionCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/AlterTableDropCarbonPartitionCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/AlterTableDropCarbonPartitionCommand.scala
index e0b891a..9b16060 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/AlterTableDropCarbonPartitionCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/AlterTableDropCarbonPartitionCommand.scala
@@ -18,32 +18,37 @@
 package org.apache.spark.sql.execution.command.partition
 
 import java.util
+import java.util.concurrent.{Executors, ExecutorService, Future}
 
 import scala.collection.JavaConverters._
 
-import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
-import org.apache.spark.sql.execution.command.{AlterTableDropPartitionModel, DataProcessCommand, RunnableCommand, SchemaProcessCommand}
-import org.apache.spark.sql.hive.{CarbonMetaStore, CarbonRelation}
+import org.apache.spark.sql.{CarbonEnv, Row, SparkSession, SQLContext}
+import org.apache.spark.sql.execution.command.{AlterTableDropPartitionModel, DataProcessCommand, DropPartitionCallableModel, RunnableCommand, SchemaProcessCommand}
+import org.apache.spark.sql.hive.CarbonRelation
 import org.apache.spark.util.AlterTableUtil
 
 import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
 import org.apache.carbondata.core.cache.CacheProvider
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datamap.DataMapStoreManager
 import org.apache.carbondata.core.locks.{ICarbonLock, LockUsage}
-import org.apache.carbondata.core.metadata.{CarbonMetadata, CarbonTableIdentifier}
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonMetadata}
 import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl
-import org.apache.carbondata.core.metadata.schema.PartitionInfo
 import org.apache.carbondata.core.metadata.schema.partition.PartitionType
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.mutate.CarbonUpdateUtil
-import org.apache.carbondata.core.util.CarbonUtil
+import org.apache.carbondata.core.statusmanager.SegmentStatusManager
+import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
 import org.apache.carbondata.core.util.path.CarbonStorePath
 import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, CarbonLoadModel}
-import org.apache.carbondata.spark.rdd.CarbonDataRDDFactory
+import org.apache.carbondata.processing.util.CarbonLoaderUtil
+import org.apache.carbondata.spark.partition.DropPartitionCallable
 
 case class AlterTableDropCarbonPartitionCommand(
     model: AlterTableDropPartitionModel)
   extends RunnableCommand with DataProcessCommand with SchemaProcessCommand {
-  val oldPartitionIds: util.ArrayList[Int] = new util.ArrayList[Int]()
+
+  private val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getName)
+  private val oldPartitionIds: util.ArrayList[Int] = new util.ArrayList[Int]()
 
   override def run(sparkSession: SparkSession): Seq[Row] = {
     if (model.partitionId.equals("0")) {
@@ -55,7 +60,6 @@ case class AlterTableDropCarbonPartitionCommand(
   }
 
   override def processSchema(sparkSession: SparkSession): Seq[Row] = {
-    val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getName)
     val dbName = model.databaseName.getOrElse(sparkSession.catalog.currentDatabase)
     val tableName = model.tableName
     val carbonMetaStore = CarbonEnv.getInstance(sparkSession).carbonMetastore
@@ -151,7 +155,7 @@ case class AlterTableDropCarbonPartitionCommand(
       carbonLoadModel.setStorePath(relation.tableMeta.storePath)
       val loadStartTime = CarbonUpdateUtil.readCurrentTime
       carbonLoadModel.setFactTimeStamp(loadStartTime)
-      CarbonDataRDDFactory.alterTableDropPartition(
+      alterTableDropPartition(
         sparkSession.sqlContext,
         model.partitionId,
         carbonLoadModel,
@@ -173,4 +177,111 @@ case class AlterTableDropCarbonPartitionCommand(
     LOGGER.audit(s"Alter table drop partition is successful for table $dbName.$tableName")
     Seq.empty
   }
+
+  private def alterTableDropPartition(sqlContext: SQLContext,
+      partitionId: String,
+      carbonLoadModel: CarbonLoadModel,
+      dropWithData: Boolean,
+      oldPartitionIds: List[Int]): Unit = {
+    LOGGER.audit(s"Drop partition request received for table " +
+                 s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
+    try {
+      startDropThreads(
+        sqlContext,
+        carbonLoadModel,
+        partitionId,
+        dropWithData,
+        oldPartitionIds)
+    } catch {
+      case e: Exception =>
+        LOGGER.error(s"Exception in start dropping partition thread. ${ e.getMessage }")
+        throw e
+    }
+  }
+
+  private def startDropThreads(sqlContext: SQLContext,
+      carbonLoadModel: CarbonLoadModel,
+      partitionId: String,
+      dropWithData: Boolean,
+      oldPartitionIds: List[Int]): Unit = {
+    val numberOfCores = CarbonProperties.getInstance().getProperty(
+      CarbonCommonConstants.NUM_CORES_ALT_PARTITION,
+        CarbonCommonConstants.DEFAULT_NUMBER_CORES)
+    val executor : ExecutorService = Executors.newFixedThreadPool(numberOfCores.toInt)
+    try {
+      val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
+      val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
+      val segmentStatusManager = new SegmentStatusManager(absoluteTableIdentifier)
+      val validSegments = segmentStatusManager.getValidAndInvalidSegments.getValidSegments.asScala
+      val threadArray: Array[Thread] = new Array[Thread](validSegments.size)
+      var i = 0
+      for (segmentId: String <- validSegments) {
+        threadArray(i) = dropPartitionThread(sqlContext, carbonLoadModel, executor,
+          segmentId, partitionId, dropWithData, oldPartitionIds)
+        threadArray(i).start()
+        i += 1
+      }
+      for (thread <- threadArray) {
+        thread.join()
+      }
+      val identifier = AbsoluteTableIdentifier.from(carbonLoadModel.getStorePath,
+        carbonLoadModel.getDatabaseName, carbonLoadModel.getTableName)
+      val refresher = DataMapStoreManager.getInstance().getTableSegmentRefresher(identifier)
+      refresher.refreshSegments(validSegments.asJava)
+    } catch {
+      case e: Exception =>
+        LOGGER.error(s"Exception when dropping partition: ${ e.getMessage }")
+    } finally {
+      executor.shutdown()
+      try {
+        CarbonLoaderUtil.deletePartialLoadDataIfExist(carbonLoadModel, false)
+      } catch {
+        case e: Exception =>
+          LOGGER.error(s"Exception in dropping partition thread while deleting partial load file" +
+                       s" ${ e.getMessage }")
+      }
+    }
+  }
+}
+
+case class dropPartitionThread(sqlContext: SQLContext,
+    carbonLoadModel: CarbonLoadModel,
+    executor: ExecutorService,
+    segmentId: String,
+    partitionId: String,
+    dropWithData: Boolean,
+    oldPartitionIds: List[Int]) extends Thread {
+
+  private val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getName)
+
+  override def run(): Unit = {
+    try {
+      executeDroppingPartition(sqlContext, carbonLoadModel, executor,
+        segmentId, partitionId, dropWithData, oldPartitionIds)
+    } catch {
+      case e: Exception =>
+        val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getName)
+        LOGGER.error(s"Exception in dropping partition thread: ${ e.getMessage } }")
+    }
+  }
+
+  private def executeDroppingPartition(sqlContext: SQLContext,
+      carbonLoadModel: CarbonLoadModel,
+      executor: ExecutorService,
+      segmentId: String,
+      partitionId: String,
+      dropWithData: Boolean,
+      oldPartitionIds: List[Int]): Unit = {
+    val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
+    val model = new DropPartitionCallableModel(carbonLoadModel,
+      segmentId, partitionId, oldPartitionIds, dropWithData, carbonTable, sqlContext)
+    val future: Future[Void] = executor.submit(new DropPartitionCallable(model))
+    try {
+      future.get
+    } catch {
+      case e: Exception =>
+        LOGGER.error(e, s"Exception in partition drop thread ${ e.getMessage }")
+        throw e
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/0578ba0f/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/AlterTableSplitCarbonPartitionCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/AlterTableSplitCarbonPartitionCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/AlterTableSplitCarbonPartitionCommand.scala
index 12bf31e..c3a918c 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/AlterTableSplitCarbonPartitionCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/AlterTableSplitCarbonPartitionCommand.scala
@@ -19,29 +19,32 @@ package org.apache.spark.sql.execution.command.partition
 
 import java.text.SimpleDateFormat
 import java.util
+import java.util.concurrent.{Executors, ExecutorService, Future}
 
 import scala.collection.JavaConverters._
 
-import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
+import org.apache.spark.sql.{CarbonEnv, Row, SparkSession, SQLContext}
 import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.execution.command.{AlterTableSplitPartitionModel, DataProcessCommand, RunnableCommand, SchemaProcessCommand}
-import org.apache.spark.sql.hive.{CarbonMetaStore, CarbonRelation}
+import org.apache.spark.sql.execution.command.{AlterTableSplitPartitionModel, DataProcessCommand, RunnableCommand, SchemaProcessCommand, SplitPartitionCallableModel}
+import org.apache.spark.sql.hive.CarbonRelation
 import org.apache.spark.util.{AlterTableUtil, PartitionUtils}
 
 import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
 import org.apache.carbondata.core.cache.CacheProvider
 import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datamap.DataMapStoreManager
 import org.apache.carbondata.core.locks.{ICarbonLock, LockUsage}
-import org.apache.carbondata.core.metadata.{CarbonMetadata, CarbonTableIdentifier}
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonMetadata}
 import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl
 import org.apache.carbondata.core.metadata.schema.PartitionInfo
 import org.apache.carbondata.core.metadata.schema.partition.PartitionType
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.mutate.CarbonUpdateUtil
+import org.apache.carbondata.core.statusmanager.SegmentStatusManager
 import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
 import org.apache.carbondata.core.util.path.CarbonStorePath
 import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, CarbonLoadModel}
-import org.apache.carbondata.spark.rdd.CarbonDataRDDFactory
+import org.apache.carbondata.processing.util.CarbonLoaderUtil
+import org.apache.carbondata.spark.partition.SplitPartitionCallable
 
 /**
  * Command for Alter Table Add & Split partition
@@ -51,7 +54,8 @@ case class AlterTableSplitCarbonPartitionCommand(
     splitPartitionModel: AlterTableSplitPartitionModel)
   extends RunnableCommand with DataProcessCommand with SchemaProcessCommand {
 
-  val oldPartitionIds: util.ArrayList[Int] = new util.ArrayList[Int]()
+  private val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getName)
+  private val oldPartitionIds: util.ArrayList[Int] = new util.ArrayList[Int]()
 
   // TODO will add rollback function in case of process data failure
   override def run(sparkSession: SparkSession): Seq[Row] = {
@@ -60,7 +64,6 @@ case class AlterTableSplitCarbonPartitionCommand(
   }
 
   override def processSchema(sparkSession: SparkSession): Seq[Row] = {
-    val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getName)
     val dbName = splitPartitionModel.databaseName.getOrElse(sparkSession.catalog.currentDatabase)
     val carbonMetaStore = CarbonEnv.getInstance(sparkSession).carbonMetastore
     val tableName = splitPartitionModel.tableName
@@ -114,8 +117,7 @@ case class AlterTableSplitCarbonPartitionCommand(
     Seq.empty
   }
 
-  private def updatePartitionInfo(partitionInfo: PartitionInfo,
-      partitionIds: List[Int]) = {
+  private def updatePartitionInfo(partitionInfo: PartitionInfo, partitionIds: List[Int]): Unit = {
     val dateFormatter = new SimpleDateFormat(CarbonProperties.getInstance
       .getProperty(CarbonCommonConstants.CARBON_DATE_FORMAT,
         CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT))
@@ -161,7 +163,7 @@ case class AlterTableSplitCarbonPartitionCommand(
       carbonLoadModel.setStorePath(storePath)
       val loadStartTime = CarbonUpdateUtil.readCurrentTime
       carbonLoadModel.setFactTimeStamp(loadStartTime)
-      CarbonDataRDDFactory.alterTableSplitPartition(
+      alterTableSplitPartition(
         sparkSession.sqlContext,
         splitPartitionModel.partitionId.toInt.toString,
         carbonLoadModel,
@@ -185,4 +187,136 @@ case class AlterTableSplitCarbonPartitionCommand(
     }
     Seq.empty
   }
+
+  private def alterTableSplitPartition(
+      sqlContext: SQLContext,
+      partitionId: String,
+      carbonLoadModel: CarbonLoadModel,
+      oldPartitionIdList: List[Int]
+  ): Unit = {
+    LOGGER.audit(s"Add partition request received for table " +
+                 s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
+    try {
+      startSplitThreads(sqlContext,
+        carbonLoadModel,
+        partitionId,
+        oldPartitionIdList)
+    } catch {
+      case e: Exception =>
+        LOGGER.error(s"Exception in start splitting partition thread. ${ e.getMessage }")
+        throw e
+    }
+  }
+
+  private def startSplitThreads(sqlContext: SQLContext,
+      carbonLoadModel: CarbonLoadModel,
+      partitionId: String,
+      oldPartitionIdList: List[Int]): Unit = {
+    val numberOfCores = CarbonProperties.getInstance()
+      .getProperty(CarbonCommonConstants.NUM_CORES_ALT_PARTITION,
+        CarbonCommonConstants.DEFAULT_NUMBER_CORES)
+    val executor : ExecutorService = Executors.newFixedThreadPool(numberOfCores.toInt)
+    try {
+      val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
+      val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
+      val segmentStatusManager = new SegmentStatusManager(absoluteTableIdentifier)
+      val validSegments = segmentStatusManager.getValidAndInvalidSegments.getValidSegments.asScala
+      val threadArray: Array[SplitThread] = new Array[SplitThread](validSegments.size)
+      var i = 0
+      validSegments.foreach { segmentId =>
+        threadArray(i) = SplitThread(sqlContext, carbonLoadModel, executor,
+          segmentId, partitionId, oldPartitionIdList)
+        threadArray(i).start()
+        i += 1
+      }
+      threadArray.foreach {
+        thread => thread.join()
+      }
+      val identifier = AbsoluteTableIdentifier.from(carbonLoadModel.getStorePath,
+        carbonLoadModel.getDatabaseName, carbonLoadModel.getTableName)
+      val refresher = DataMapStoreManager.getInstance().getTableSegmentRefresher(identifier)
+      refresher.refreshSegments(validSegments.asJava)
+    } catch {
+      case e: Exception =>
+        LOGGER.error(s"Exception when split partition: ${ e.getMessage }")
+        throw e
+    } finally {
+      executor.shutdown()
+      try {
+        CarbonLoaderUtil.deletePartialLoadDataIfExist(carbonLoadModel, false)
+      } catch {
+        case e: Exception =>
+          LOGGER.error(s"Exception in add/split partition thread while deleting partial load file" +
+                       s" ${ e.getMessage }")
+      }
+    }
+  }
+}
+
+case class SplitThread(sqlContext: SQLContext,
+    carbonLoadModel: CarbonLoadModel,
+    executor: ExecutorService,
+    segmentId: String,
+    partitionId: String,
+    oldPartitionIdList: List[Int]) extends Thread {
+
+  private val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getName)
+
+  override def run(): Unit = {
+    var triggeredSplitPartitionStatus = false
+    var exception: Exception = null
+    try {
+      executePartitionSplit(sqlContext,
+        carbonLoadModel, executor, segmentId, partitionId, oldPartitionIdList)
+      triggeredSplitPartitionStatus = true
+    } catch {
+      case e: Exception =>
+        val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getName)
+        LOGGER.error(s"Exception in partition split thread: ${ e.getMessage } }")
+        exception = e
+    }
+    if (!triggeredSplitPartitionStatus) {
+      throw new Exception("Exception in split partition " + exception.getMessage)
+    }
+  }
+
+  private def executePartitionSplit( sqlContext: SQLContext,
+      carbonLoadModel: CarbonLoadModel,
+      executor: ExecutorService,
+      segment: String,
+      partitionId: String,
+      oldPartitionIdList: List[Int]): Unit = {
+    val futureList: util.List[Future[Void]] = new util.ArrayList[Future[Void]](
+      CarbonCommonConstants.DEFAULT_COLLECTION_SIZE
+    )
+    scanSegmentsForSplitPartition(futureList, executor, segment, partitionId,
+      sqlContext, carbonLoadModel, oldPartitionIdList)
+    try {
+      futureList.asScala.foreach { future =>
+        future.get
+      }
+    } catch {
+      case e: Exception =>
+        LOGGER.error(e, s"Exception in partition split thread ${ e.getMessage }")
+        throw e
+    }
+  }
+
+  private def scanSegmentsForSplitPartition(futureList: util.List[Future[Void]],
+      executor: ExecutorService,
+      segmentId: String,
+      partitionId: String,
+      sqlContext: SQLContext,
+      carbonLoadModel: CarbonLoadModel,
+      oldPartitionIdList: List[Int]): Unit = {
+
+    val splitModel = SplitPartitionCallableModel(carbonLoadModel,
+      segmentId,
+      partitionId,
+      oldPartitionIdList,
+      sqlContext)
+
+    val future: Future[Void] = executor.submit(new SplitPartitionCallable(splitModel))
+    futureList.add(future)
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/0578ba0f/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableRevertTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableRevertTestCase.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableRevertTestCase.scala
index 3fcad74..29de05b 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableRevertTestCase.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableRevertTestCase.scala
@@ -104,7 +104,7 @@ class AlterTableRevertTestCase extends Spark2QueryTest with BeforeAndAfterAll {
       sql("alter table reverttest rename to revert")
     }
     AlterTableUtil.releaseLocks(locks)
-    assert(exception.getMessage == "Alter table rename table operation failed: Table is locked for updation. Please try after some time")
+    assert(exception.getMessage == "Alter table rename table operation failed: Acquire table lock failed after retry, please try after some time")
   }
 
   override def afterAll() {


[35/49] 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/fgdatamap
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;


[10/49] carbondata git commit: [CARBONDATA-1667] Remove direct load related code

Posted by ra...@apache.org.
[CARBONDATA-1667] Remove direct load related code

This closes #1465


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

Branch: refs/heads/fgdatamap
Commit: 6f6897191819994da2066584721c462f03184cc6
Parents: f812e41
Author: Jacky Li <ja...@qq.com>
Authored: Sat Nov 4 18:53:46 2017 +0800
Committer: QiangCai <qi...@qq.com>
Committed: Mon Nov 6 12:21:32 2017 +0800

----------------------------------------------------------------------
 .../presto/util/CarbonDataStoreCreator.scala    |  1 -
 .../spark/rdd/NewCarbonDataLoadRDD.scala        | 35 ++++----------
 .../carbondata/spark/util/DataLoadingUtil.scala |  1 -
 .../spark/rdd/CarbonDataRDDFactory.scala        | 49 ++++++--------------
 .../loading/model/CarbonLoadModel.java          | 11 -----
 5 files changed, 25 insertions(+), 72 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/6f689719/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala
----------------------------------------------------------------------
diff --git a/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala b/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala
index acee71b..09cddfe 100644
--- a/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala
+++ b/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala
@@ -118,7 +118,6 @@ object CarbonDataStoreCreator {
       loadModel.setBadRecordsAction(
         TableOptionConstant.BAD_RECORDS_ACTION.getName + "," +
         "force")
-      loadModel.setDirectLoad(true)
       loadModel.setIsEmptyDataBadRecord(
         DataLoadProcessorConstants.IS_EMPTY_DATA_BAD_RECORD +
         "," +

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6f689719/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
index 49b708c..1d6ad70 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
@@ -196,13 +196,7 @@ class NewCarbonDataLoadRDD[K, V](
     if (isTableSplitPartition) {
       // for table split partition
       var splits: Array[TableSplit] = null
-
-      if (carbonLoadModel.isDirectLoad) {
-        splits = CarbonQueryUtil.getTableSplitsForDirectLoad(carbonLoadModel.getFactFilePath)
-      } else {
-        splits = CarbonQueryUtil.getTableSplits(carbonLoadModel.getDatabaseName,
-          carbonLoadModel.getTableName, null)
-      }
+      splits = CarbonQueryUtil.getTableSplitsForDirectLoad(carbonLoadModel.getFactFilePath)
 
       splits.zipWithIndex.map { s =>
         // filter the same partition unique id, because only one will match, so get 0 element
@@ -289,15 +283,10 @@ class NewCarbonDataLoadRDD[K, V](
           val split = theSplit.asInstanceOf[CarbonTableSplitPartition]
           logInfo("Input split: " + split.serializableHadoopSplit.value)
           carbonLoadModel.setTaskNo(String.valueOf(theSplit.index))
-          if (carbonLoadModel.isDirectLoad) {
-            model = carbonLoadModel.getCopyWithPartition(
-                split.serializableHadoopSplit.value.getPartition.getUniqueID,
-                split.serializableHadoopSplit.value.getPartition.getFilesPath,
-                carbonLoadModel.getCsvHeader, carbonLoadModel.getCsvDelimiter)
-          } else {
-            model = carbonLoadModel.getCopyWithPartition(
-                split.serializableHadoopSplit.value.getPartition.getUniqueID)
-          }
+          model = carbonLoadModel.getCopyWithPartition(
+              split.serializableHadoopSplit.value.getPartition.getUniqueID,
+              split.serializableHadoopSplit.value.getPartition.getFilesPath,
+              carbonLoadModel.getCsvHeader, carbonLoadModel.getCsvDelimiter)
           partitionID = split.serializableHadoopSplit.value.getPartition.getUniqueID
           StandardLogService.setThreadName(StandardLogService
             .getPartitionID(model.getCarbonDataLoadSchema.getCarbonTable.getTableUniqueName)
@@ -320,15 +309,11 @@ class NewCarbonDataLoadRDD[K, V](
               split.serializableHadoopSplit, split.nodeBlocksDetail.length)
           val blocksID = gernerateBlocksID
           carbonLoadModel.setTaskNo(String.valueOf(theSplit.index))
-          if (carbonLoadModel.isDirectLoad) {
-            val filelist: java.util.List[String] = new java.util.ArrayList[String](
-                CarbonCommonConstants.CONSTANT_SIZE_TEN)
-            CarbonQueryUtil.splitFilePath(carbonLoadModel.getFactFilePath, filelist, ",")
-            model = carbonLoadModel.getCopyWithPartition(partitionID, filelist,
-                carbonLoadModel.getCsvHeader, carbonLoadModel.getCsvDelimiter)
-          } else {
-            model = carbonLoadModel.getCopyWithPartition(partitionID)
-          }
+          val filelist: java.util.List[String] = new java.util.ArrayList[String](
+              CarbonCommonConstants.CONSTANT_SIZE_TEN)
+          CarbonQueryUtil.splitFilePath(carbonLoadModel.getFactFilePath, filelist, ",")
+          model = carbonLoadModel.getCopyWithPartition(partitionID, filelist,
+              carbonLoadModel.getCsvHeader, carbonLoadModel.getCsvDelimiter)
           StandardLogService.setThreadName(StandardLogService
             .getPartitionID(model.getCarbonDataLoadSchema.getCarbonTable.getTableUniqueName)
             , ThreadLocalTaskInfo.getCarbonTaskInfo.getTaskId + "")

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6f689719/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataLoadingUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataLoadingUtil.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataLoadingUtil.scala
index 445fdbb..5a24d7d 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataLoadingUtil.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataLoadingUtil.scala
@@ -285,7 +285,6 @@ object DataLoadingUtil {
     carbonLoadModel.setCsvDelimiter(CarbonUtil.unescapeChar(delimeter))
     carbonLoadModel.setCsvHeader(fileHeader)
     carbonLoadModel.setColDictFilePath(column_dict)
-    carbonLoadModel.setDirectLoad(true)
     carbonLoadModel.setCsvHeaderColumns(CommonUtil.getCsvHeaderColumns(carbonLoadModel))
 
     val validatedMaxColumns = CommonUtil.validateMaxColumns(

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6f689719/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
index 628d444..cfd8cff 100644
--- a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
@@ -653,40 +653,21 @@ object CarbonDataRDDFactory {
          * 3) output Array[(partitionID,Array[BlockDetails])] to blocksGroupBy
          */
           var splits = Array[TableSplit]()
-          if (carbonLoadModel.isDirectLoad) {
-            // get all table Splits, this part means files were divide to different partitions
-            splits = CarbonQueryUtil.getTableSplitsForDirectLoad(carbonLoadModel.getFactFilePath)
-            // get all partition blocks from file list
-            blocksGroupBy = splits.map {
-              split =>
-                val pathBuilder = new StringBuilder()
-                for (path <- split.getPartition.getFilesPath.asScala) {
-                  pathBuilder.append(path).append(",")
-                }
-                if (pathBuilder.nonEmpty) {
-                  pathBuilder.substring(0, pathBuilder.size - 1)
-                }
-                (split.getPartition.getUniqueID, SparkUtil.getSplits(pathBuilder.toString(),
-                  sqlContext.sparkContext
-                ))
-            }
-          } else {
-            // get all table Splits,when come to this, means data have been partition
-            splits = CarbonQueryUtil.getTableSplits(carbonLoadModel.getDatabaseName,
-              carbonLoadModel.getTableName, null)
-            // get all partition blocks from factFilePath/uniqueID/
-            blocksGroupBy = splits.map {
-              split =>
-                val pathBuilder = new StringBuilder()
-                pathBuilder.append(carbonLoadModel.getFactFilePath)
-                if (!carbonLoadModel.getFactFilePath.endsWith("/")
-                    && !carbonLoadModel.getFactFilePath.endsWith("\\")) {
-                  pathBuilder.append("/")
-                }
-                pathBuilder.append(split.getPartition.getUniqueID).append("/")
-                (split.getPartition.getUniqueID,
-                    SparkUtil.getSplits(pathBuilder.toString, sqlContext.sparkContext))
-            }
+          // get all table Splits, this part means files were divide to different partitions
+          splits = CarbonQueryUtil.getTableSplitsForDirectLoad(carbonLoadModel.getFactFilePath)
+          // get all partition blocks from file list
+          blocksGroupBy = splits.map {
+            split =>
+              val pathBuilder = new StringBuilder()
+              for (path <- split.getPartition.getFilesPath.asScala) {
+                pathBuilder.append(path).append(",")
+              }
+              if (pathBuilder.nonEmpty) {
+                pathBuilder.substring(0, pathBuilder.size - 1)
+              }
+              (split.getPartition.getUniqueID, SparkUtil.getSplits(pathBuilder.toString(),
+                sqlContext.sparkContext
+              ))
           }
         } else {
           /*

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6f689719/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModel.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModel.java b/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModel.java
index 39ee270..6a156a6 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModel.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModel.java
@@ -53,7 +53,6 @@ public class CarbonLoadModel implements Serializable {
   private String complexDelimiterLevel1;
   private String complexDelimiterLevel2;
 
-  private boolean isDirectLoad;
   private List<LoadMetadataDetails> loadMetadataDetails;
   private transient SegmentUpdateStatusManager segmentUpdateStatusManager;
 
@@ -206,14 +205,6 @@ public class CarbonLoadModel implements Serializable {
     this.complexDelimiterLevel2 = complexDelimiterLevel2;
   }
 
-  public boolean isDirectLoad() {
-    return isDirectLoad;
-  }
-
-  public void setDirectLoad(boolean isDirectLoad) {
-    this.isDirectLoad = isDirectLoad;
-  }
-
   public String getAllDictPath() {
     return allDictPath;
   }
@@ -383,7 +374,6 @@ public class CarbonLoadModel implements Serializable {
     copy.isRetentionRequest = isRetentionRequest;
     copy.csvHeader = csvHeader;
     copy.csvHeaderColumns = csvHeaderColumns;
-    copy.isDirectLoad = isDirectLoad;
     copy.csvDelimiter = csvDelimiter;
     copy.complexDelimiterLevel1 = complexDelimiterLevel1;
     copy.complexDelimiterLevel2 = complexDelimiterLevel2;
@@ -434,7 +424,6 @@ public class CarbonLoadModel implements Serializable {
     copyObj.carbonDataLoadSchema = carbonDataLoadSchema;
     copyObj.csvHeader = header;
     copyObj.csvHeaderColumns = csvHeaderColumns;
-    copyObj.isDirectLoad = true;
     copyObj.csvDelimiter = delimiter;
     copyObj.complexDelimiterLevel1 = complexDelimiterLevel1;
     copyObj.complexDelimiterLevel2 = complexDelimiterLevel2;


[07/49] carbondata git commit: [CARBONDATA-1594] Add precision and scale to DecimalType

Posted by ra...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImpl.java
index 7a80c88..87e7dea 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImpl.java
@@ -113,9 +113,13 @@ public class ExcludeFilterExecuterImpl implements FilterExecuter {
       DataType msrType = getMeasureDataType(msrColumnEvaluatorInfo);
       for (int i = 0; i < ColumnPages.length; i++) {
         BitSet bitSet =
-            getFilteredIndexesForMeasure(measureRawColumnChunk.convertToColumnPage(i),
-                measureRawColumnChunk.getRowCount()[i], useBitsetPipeLine,
-                blockChunkHolder.getBitSetGroup(), i, msrType);;
+            getFilteredIndexesForMeasure(
+                measureRawColumnChunk.convertToColumnPage(i),
+                measureRawColumnChunk.getRowCount()[i],
+                useBitsetPipeLine,
+                blockChunkHolder.getBitSetGroup(),
+                i,
+                msrType);
         bitSetGroup.setBitSet(bitSet, i);
       }
       return bitSetGroup;
@@ -132,15 +136,14 @@ public class ExcludeFilterExecuterImpl implements FilterExecuter {
       return DataTypes.INT;
     } else if (msrColumnEvaluatorInfo.getType() == DataTypes.LONG) {
       return DataTypes.LONG;
-    } else if (msrColumnEvaluatorInfo.getType() == DataTypes.DECIMAL) {
-      return DataTypes.DECIMAL;
+    } else if (DataTypes.isDecimal(msrColumnEvaluatorInfo.getType())) {
+      return DataTypes.createDefaultDecimalType();
     } else {
       return DataTypes.DOUBLE;
     }
   }
 
-  protected BitSet getFilteredIndexes(ColumnPage columnPage,
-      int numerOfRows, DataType msrType) {
+  private BitSet getFilteredIndexes(ColumnPage columnPage, int numerOfRows, DataType msrType) {
     // Here the algorithm is
     // Get the measure values from the chunk. compare sequentially with the
     // the filter values. The one that matches sets it Bitset.
@@ -311,6 +314,7 @@ public class ExcludeFilterExecuterImpl implements FilterExecuter {
     }
     return bitSet;
   }
+
   private BitSet setFilterdIndexToBitSetWithColumnIndex(
       DimensionColumnDataChunk dimensionColumnDataChunk, int numerOfRows) {
     BitSet bitSet = new BitSet(numerOfRows);
@@ -383,14 +387,14 @@ public class ExcludeFilterExecuterImpl implements FilterExecuter {
   }
 
   @Override public void readBlocks(BlocksChunkHolder blockChunkHolder) throws IOException {
-    if (isDimensionPresentInCurrentBlock == true) {
+    if (isDimensionPresentInCurrentBlock) {
       int blockIndex = segmentProperties.getDimensionOrdinalToBlockMapping()
           .get(dimColEvaluatorInfo.getColumnIndex());
       if (null == blockChunkHolder.getDimensionRawDataChunk()[blockIndex]) {
         blockChunkHolder.getDimensionRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
             .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex);
       }
-    } else if (isMeasurePresentInCurrentBlock == true) {
+    } else if (isMeasurePresentInCurrentBlock) {
       int blockIndex = segmentProperties.getMeasuresOrdinalToBlockMapping()
           .get(msrColumnEvaluatorInfo.getColumnIndex());
       if (null == blockChunkHolder.getMeasureRawDataChunk()[blockIndex]) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImpl.java
index 843da71..0022a72 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImpl.java
@@ -156,8 +156,8 @@ public class IncludeFilterExecuterImpl implements FilterExecuter {
       return DataTypes.INT;
     } else if (msrColumnEvaluatorInfo.getType() == DataTypes.LONG) {
       return DataTypes.LONG;
-    } else if (msrColumnEvaluatorInfo.getType() == DataTypes.DECIMAL) {
-      return DataTypes.DECIMAL;
+    } else if (DataTypes.isDecimal(msrColumnEvaluatorInfo.getType())) {
+      return DataTypes.createDefaultDecimalType();
     } else {
       return DataTypes.DOUBLE;
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/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 dbf8d4c..10664c8 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
@@ -359,8 +359,8 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
         msrType = DataTypes.INT;
       } else if (dataType == DataTypes.LONG) {
         msrType = DataTypes.LONG;
-      } else if (dataType == DataTypes.DECIMAL) {
-        msrType = DataTypes.DECIMAL;
+      } else if (DataTypes.isDecimal(dataType)) {
+        msrType = DataTypes.createDefaultDecimalType();
       } else {
         msrType = DataTypes.DOUBLE;
       }
@@ -386,7 +386,7 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
         msrValue = (int) columnPage.getLong(index);
       } else if (msrType == DataTypes.LONG) {
         msrValue = columnPage.getLong(index);
-      } else if (msrType == DataTypes.DECIMAL) {
+      } else if (DataTypes.isDecimal(msrType)) {
         BigDecimal bigDecimalValue = columnPage.getDecimal(index);
         if (null != bigDecimalValue
             && msrColumnEvalutorInfo.getCarbonColumn().getColumnSchema().getScale()

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/main/java/org/apache/carbondata/core/scan/partition/PartitionUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/partition/PartitionUtil.java b/core/src/main/java/org/apache/carbondata/core/scan/partition/PartitionUtil.java
index 85e6cef..676cf48 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/partition/PartitionUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/partition/PartitionUtil.java
@@ -66,7 +66,7 @@ public class PartitionUtil {
         return dateFormatter.parse(data).getTime();
       } else if (actualDataType == DataTypes.TIMESTAMP) {
         return timestampFormatter.parse(data).getTime();
-      } else if (actualDataType == DataTypes.DECIMAL) {
+      } else if (DataTypes.isDecimal(actualDataType)) {
         return new BigDecimal(data);
       } else {
         return data;
@@ -102,7 +102,7 @@ public class PartitionUtil {
         return Long.parseLong(data);
       } else if (actualDataType == DataTypes.DATE || actualDataType == DataTypes.TIMESTAMP) {
         return Long.parseLong(data) / 1000;
-      } else if (actualDataType == DataTypes.DECIMAL) {
+      } else if (DataTypes.isDecimal(actualDataType)) {
         return new BigDecimal(data);
       } else {
         return data;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/main/java/org/apache/carbondata/core/scan/result/vector/MeasureDataVectorProcessor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/vector/MeasureDataVectorProcessor.java b/core/src/main/java/org/apache/carbondata/core/scan/result/vector/MeasureDataVectorProcessor.java
index cf6c88b..5916fd3 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/result/vector/MeasureDataVectorProcessor.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/result/vector/MeasureDataVectorProcessor.java
@@ -372,7 +372,7 @@ public class MeasureDataVectorProcessor {
         return new IntegralMeasureVectorFiller();
       } else if (dataType == DataTypes.LONG) {
         return new LongMeasureVectorFiller();
-      } else if (dataType == DataTypes.DECIMAL) {
+      } else if (DataTypes.isDecimal(dataType)) {
         return new DecimalMeasureVectorFiller();
       } else {
         return new DefaultMeasureVectorFiller();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/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 cba9931..aca2fd6 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
@@ -37,6 +37,7 @@ import org.apache.carbondata.core.metadata.blocklet.index.BlockletIndex;
 import org.apache.carbondata.core.metadata.blocklet.index.BlockletMinMaxIndex;
 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.column.ColumnSchema;
 import org.apache.carbondata.core.reader.CarbonIndexFileReader;
@@ -262,8 +263,13 @@ public abstract class AbstractDataFileFooterConverter {
     wrapperColumnSchema.setColumnUniqueId(externalColumnSchema.getColumn_id());
     wrapperColumnSchema.setColumnName(externalColumnSchema.getColumn_name());
     wrapperColumnSchema.setColumnar(externalColumnSchema.isColumnar());
-    wrapperColumnSchema
-        .setDataType(thriftDataTyopeToWrapperDataType(externalColumnSchema.data_type));
+    DataType dataType = thriftDataTyopeToWrapperDataType(externalColumnSchema.data_type);
+    if (DataTypes.isDecimal(dataType)) {
+      DecimalType decimalType = (DecimalType) dataType;
+      decimalType.setPrecision(externalColumnSchema.getPrecision());
+      decimalType.setScale(externalColumnSchema.getScale());
+    }
+    wrapperColumnSchema.setDataType(dataType);
     wrapperColumnSchema.setDimensionColumn(externalColumnSchema.isDimension());
     List<Encoding> encoders = new ArrayList<Encoding>();
     for (org.apache.carbondata.format.Encoding encoder : externalColumnSchema.getEncoders()) {
@@ -369,7 +375,7 @@ public abstract class AbstractDataFileFooterConverter {
       case DOUBLE:
         return DataTypes.DOUBLE;
       case DECIMAL:
-        return DataTypes.DECIMAL;
+        return DataTypes.createDefaultDecimalType();
       case DATE:
         return DataTypes.DATE;
       case TIMESTAMP:

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/main/java/org/apache/carbondata/core/util/CarbonMetadataUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/CarbonMetadataUtil.java b/core/src/main/java/org/apache/carbondata/core/util/CarbonMetadataUtil.java
index d232d61..6579a6f 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/CarbonMetadataUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/CarbonMetadataUtil.java
@@ -363,7 +363,7 @@ public class CarbonMetadataUtil {
       firstBuffer.flip();
       secondBuffer.flip();
       return (int) (firstBuffer.getLong() - secondBuffer.getLong());
-    } else if (dataType == DataTypes.DECIMAL) {
+    } else if (DataTypes.isDecimal(dataType)) {
       return DataTypeUtil.byteToBigDecimal(first).compareTo(DataTypeUtil.byteToBigDecimal(second));
     } else {
       throw new IllegalArgumentException("Invalid data type");

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
index 77789c5..17a4b5f 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
@@ -1848,6 +1848,7 @@ public final class CarbonUtil {
     return map;
   }
 
+  // TODO: move this to carbon store API as it is related to TableInfo creation
   public static TableInfo convertGsonToTableInfo(Map<String, String> properties) {
     Gson gson = new Gson();
     String partsNo = properties.get("carbonSchemaPartsNo");
@@ -1864,9 +1865,35 @@ public final class CarbonUtil {
       builder.append(part);
     }
     TableInfo tableInfo = gson.fromJson(builder.toString(), TableInfo.class);
+
+    // The tableInfo is deserialized from GSON string, need to update the scale and
+    // precision if there are any decimal field, because DecimalType is added in Carbon 1.3,
+    // If it is not updated, read compactibility will be break for table generated before Carbon 1.3
+    updateDecimalType(tableInfo);
     return tableInfo;
   }
 
+  // Update decimal type inside `tableInfo` to set scale and precision, if there are any decimal
+  private static void updateDecimalType(TableInfo tableInfo) {
+    List<ColumnSchema> deserializedColumns = tableInfo.getFactTable().getListOfColumns();
+    for (ColumnSchema column : deserializedColumns) {
+      DataType dataType = column.getDataType();
+      if (DataTypes.isDecimal(dataType)) {
+        column.setDataType(DataTypes.createDecimalType(column.getPrecision(), column.getScale()));
+      }
+    }
+    if (tableInfo.getFactTable().getPartitionInfo() != null) {
+      List<ColumnSchema> partitionColumns =
+          tableInfo.getFactTable().getPartitionInfo().getColumnSchemaList();
+      for (ColumnSchema column : partitionColumns) {
+        DataType dataType = column.getDataType();
+        if (DataTypes.isDecimal(dataType)) {
+          column.setDataType(DataTypes.createDecimalType(column.getPrecision(), column.getScale()));
+        }
+      }
+    }
+  }
+
   /**
    * Removes schema from properties
    * @param properties
@@ -1964,7 +1991,7 @@ public final class CarbonUtil {
       b.putDouble((double) value);
       b.flip();
       return b.array();
-    } else if (dataType == DataTypes.DECIMAL) {
+    } else if (DataTypes.isDecimal(dataType)) {
       return DataTypeUtil.bigDecimalToByte((BigDecimal) value);
     } else if (dataType == DataTypes.BYTE_ARRAY) {
       return (byte[]) value;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java b/core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java
index 01e34a7..0961a63 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java
@@ -27,8 +27,6 @@ import java.text.ParseException;
 import java.text.SimpleDateFormat;
 import java.util.Arrays;
 import java.util.Date;
-import java.util.HashMap;
-import java.util.Map;
 
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
@@ -51,7 +49,6 @@ public final class DataTypeUtil {
    */
   private static final LogService LOGGER =
       LogServiceFactory.getLogService(DataTypeUtil.class.getName());
-  private static final Map<String, String> dataTypeDisplayNames;
 
   private static final ThreadLocal<DateFormat> timeStampformatter = new ThreadLocal<DateFormat>() {
     @Override protected DateFormat initialValue() {
@@ -71,23 +68,6 @@ public final class DataTypeUtil {
     }
   };
 
-  static {
-    dataTypeDisplayNames = new HashMap<String, String>(16);
-    dataTypeDisplayNames.put(DataTypes.DATE.toString(), DataTypes.DATE.getName());
-    dataTypeDisplayNames.put(DataTypes.LONG.toString(), DataTypes.LONG.getName());
-    dataTypeDisplayNames.put(DataTypes.INT.toString(), DataTypes.INT.getName());
-    dataTypeDisplayNames.put(DataTypes.FLOAT.toString(), DataTypes.FLOAT.getName());
-    dataTypeDisplayNames.put(DataTypes.BOOLEAN.toString(), DataTypes.BOOLEAN.getName());
-    dataTypeDisplayNames.put(DataTypes.NULL.toString(), DataTypes.NULL.getName());
-    dataTypeDisplayNames.put(DataTypes.DECIMAL.toString(), DataTypes.DECIMAL.getName());
-    dataTypeDisplayNames.put(DataTypes.ARRAY.toString(), DataTypes.ARRAY.getName());
-    dataTypeDisplayNames.put(DataTypes.STRUCT.toString(), DataTypes.STRUCT.getName());
-    dataTypeDisplayNames.put(DataTypes.TIMESTAMP.toString(), DataTypes.TIMESTAMP.getName());
-    dataTypeDisplayNames.put(DataTypes.DATE.toString(), DataTypes.DATE.getName());
-    dataTypeDisplayNames.put(DataTypes.SHORT.toString(), DataTypes.SHORT.getName());
-    dataTypeDisplayNames.put(DataTypes.STRING.toString(), DataTypes.STRING.getName());
-  }
-
   /**
    * DataType converter for different computing engines
    */
@@ -105,7 +85,7 @@ public final class DataTypeUtil {
       CarbonMeasure carbonMeasure) {
     if (dataType == DataTypes.BOOLEAN) {
       return BooleanConvert.parseBoolean(msrValue);
-    } else if (dataType == DataTypes.DECIMAL) {
+    } else if (DataTypes.isDecimal(dataType)) {
       BigDecimal bigDecimal =
           new BigDecimal(msrValue).setScale(carbonMeasure.getScale(), RoundingMode.HALF_UP);
       return normalizeDecimalValue(bigDecimal, carbonMeasure.getPrecision());
@@ -137,7 +117,7 @@ public final class DataTypeUtil {
       return (int) bb.getLong();
     } else if (dataType == DataTypes.LONG) {
       return bb.getLong();
-    } else if (dataType == DataTypes.DECIMAL) {
+    } else if (DataTypes.isDecimal(dataType)) {
       return byteToBigDecimal(data);
     } else {
       return bb.getDouble();
@@ -154,7 +134,7 @@ public final class DataTypeUtil {
       return (int) measurePage.getLong(index);
     } else if (dataType == DataTypes.LONG) {
       return measurePage.getLong(index);
-    } else if (dataType == DataTypes.DECIMAL) {
+    } else if (DataTypes.isDecimal(dataType)) {
       BigDecimal bigDecimalMsrValue = measurePage.getDecimal(index);
       if (null != bigDecimalMsrValue && carbonMeasure.getScale() > bigDecimalMsrValue.scale()) {
         bigDecimalMsrValue =
@@ -163,22 +143,13 @@ public final class DataTypeUtil {
       if (null != bigDecimalMsrValue) {
         return normalizeDecimalValue(bigDecimalMsrValue, carbonMeasure.getPrecision());
       } else {
-        return bigDecimalMsrValue;
+        return null;
       }
     } else {
       return measurePage.getDouble(index);
     }
   }
 
-
-  /**
-   * @param dataType
-   * @return
-   */
-  public static String getColumnDataTypeDisplayName(String dataType) {
-    return dataTypeDisplayNames.get(dataType);
-  }
-
   /**
    * This method will check the digits before dot with the max precision allowed
    *
@@ -239,55 +210,6 @@ public final class DataTypeUtil {
   }
 
   /**
-   * returns the SqlStatement.Type of corresponding string value
-   *
-   * @param dataTypeStr
-   * @return return the SqlStatement.Type
-   */
-  public static DataType getDataType(String dataTypeStr) {
-    DataType dataType = null;
-    switch (dataTypeStr) {
-      case "BOOLEAN":
-        dataType = DataTypes.BOOLEAN;
-        break;
-      case "DATE":
-        dataType = DataTypes.DATE;
-        break;
-      case "TIMESTAMP":
-        dataType = DataTypes.TIMESTAMP;
-        break;
-      case "STRING":
-        dataType = DataTypes.STRING;
-        break;
-      case "INT":
-        dataType = DataTypes.INT;
-        break;
-      case "SMALLINT":
-        dataType = DataTypes.SHORT;
-        break;
-      case "LONG":
-        dataType = DataTypes.LONG;
-        break;
-      case "DOUBLE":
-        dataType = DataTypes.DOUBLE;
-        break;
-      case "DECIMAL":
-        dataType = DataTypes.DECIMAL;
-        break;
-      case "ARRAY":
-        dataType = DataTypes.ARRAY;
-        break;
-      case "STRUCT":
-        dataType = DataTypes.STRUCT;
-        break;
-      case "MAP":
-      default:
-        dataType = DataTypes.STRING;
-    }
-    return dataType;
-  }
-
-  /**
    * Below method will be used to convert the data passed to its actual data
    * type
    *
@@ -365,7 +287,7 @@ public final class DataTypeUtil {
           LOGGER.error("Cannot convert value to Time/Long type value" + e.getMessage());
           return null;
         }
-      } else if (actualDataType == DataTypes.DECIMAL) {
+      } else if (DataTypes.isDecimal(actualDataType)) {
         if (data.isEmpty()) {
           return null;
         }
@@ -518,7 +440,7 @@ public final class DataTypeUtil {
           LOGGER.error("Cannot convert value to Time/Long type value" + e.getMessage());
           return null;
         }
-      } else if (dataType == DataTypes.DECIMAL) {
+      } else if (DataTypes.isDecimal(dataType)) {
         String data7 = new String(dataInBytes, CarbonCommonConstants.DEFAULT_CHARSET_CLASS);
         if (data7.isEmpty()) {
           return null;
@@ -585,7 +507,7 @@ public final class DataTypeUtil {
       // validation will not be done for timestamp datatype as for timestamp direct dictionary
       // is generated. No dictionary file is created for timestamp datatype column
       DataType dataType = dimension.getDataType();
-      if (dataType == DataTypes.DECIMAL) {
+      if (DataTypes.isDecimal(dataType)) {
         return parseStringToBigDecimal(value, dimension);
       } else if (dataType == DataTypes.SHORT || dataType == DataTypes.INT ||
           dataType == DataTypes.LONG) {
@@ -618,7 +540,7 @@ public final class DataTypeUtil {
     }
     try {
       DataType dataType = dimension.getDataType();
-      if (dataType == DataTypes.DECIMAL) {
+      if (DataTypes.isDecimal(dataType)) {
         return parseStringToBigDecimal(value, dimension);
       } else if (dataType == DataTypes.INT) {
         Integer.parseInt(value);
@@ -721,7 +643,7 @@ public final class DataTypeUtil {
             return null;
           }
         }
-      } else if (dataType == DataTypes.DECIMAL) {
+      } else if (DataTypes.isDecimal(dataType)) {
         String parsedValue = parseStringToBigDecimal(data, columnSchema);
         if (null == parsedValue) {
           return null;
@@ -749,7 +671,7 @@ public final class DataTypeUtil {
     try {
       Object parsedValue = null;
       DataType dataType = columnSchema.getDataType();
-      if (dataType == DataTypes.DECIMAL) {
+      if (DataTypes.isDecimal(dataType)) {
         return parseStringToBigDecimal(value, columnSchema);
       } else if (dataType == DataTypes.SHORT || dataType == DataTypes.INT ||
           dataType == DataTypes.LONG) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/main/java/org/apache/carbondata/core/util/comparator/Comparator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/comparator/Comparator.java b/core/src/main/java/org/apache/carbondata/core/util/comparator/Comparator.java
index d1beb16..c162396 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/comparator/Comparator.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/comparator/Comparator.java
@@ -37,7 +37,7 @@ public final class Comparator {
     } else if (dataType == DataTypes.LONG || dataType == DataTypes.DATE
         || dataType == DataTypes.TIMESTAMP) {
       return new LongSerializableComparator();
-    } else if (dataType == DataTypes.DECIMAL) {
+    } else if (DataTypes.isDecimal(dataType)) {
       return new BigDecimalSerializableComparator();
     } else {
       return new ByteArraySerializableComparator();
@@ -61,7 +61,7 @@ public final class Comparator {
       return new LongSerializableComparator();
     } else if (dataType == DataTypes.DOUBLE) {
       return new DoubleSerializableComparator();
-    } else if (dataType == DataTypes.DECIMAL) {
+    } else if (DataTypes.isDecimal(dataType)) {
       return new BigDecimalSerializableComparator();
     } else {
       throw new IllegalArgumentException("Unsupported data type");

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/main/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortModel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortModel.java b/core/src/main/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortModel.java
index 4396d6a..5756c0d 100644
--- a/core/src/main/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortModel.java
+++ b/core/src/main/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortModel.java
@@ -82,7 +82,7 @@ public class CarbonDictionarySortModel implements Comparable<CarbonDictionarySor
         return -1;
       }
       return d1.compareTo(d2);
-    } else if (dataType == DataTypes.DECIMAL) {
+    } else if (DataTypes.isDecimal(dataType)) {
       java.math.BigDecimal val1 = null;
       java.math.BigDecimal val2 = null;
       try {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ColumnDictionaryInfoTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ColumnDictionaryInfoTest.java b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ColumnDictionaryInfoTest.java
index 33121e2..cb0892f 100644
--- a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ColumnDictionaryInfoTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ColumnDictionaryInfoTest.java
@@ -25,6 +25,7 @@ import java.util.concurrent.CopyOnWriteArrayList;
 import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
+import org.apache.carbondata.core.metadata.datatype.DecimalType;
 import org.apache.carbondata.core.util.CarbonUtil;
 
 import mockit.Mock;
@@ -244,7 +245,7 @@ public class ColumnDictionaryInfoTest {
   }
 
   @Test public void testGetIncrementalSurrogateKeyFromDictionaryWithDecimalType() {
-    columnDictionaryInfo = new ColumnDictionaryInfo(DataTypes.DECIMAL);
+    columnDictionaryInfo = new ColumnDictionaryInfo(DataTypes.createDefaultDecimalType());
 
     List<String> evaluateResultList = Arrays.asList("150011.550");
     List<byte[]> byteValuesOfFilterMembers = convertListElementsIntoByteArray(evaluateResultList);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/test/java/org/apache/carbondata/core/datastore/page/encoding/RLECodecSuite.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/datastore/page/encoding/RLECodecSuite.java b/core/src/test/java/org/apache/carbondata/core/datastore/page/encoding/RLECodecSuite.java
index be898c2..8be5f9d 100644
--- a/core/src/test/java/org/apache/carbondata/core/datastore/page/encoding/RLECodecSuite.java
+++ b/core/src/test/java/org/apache/carbondata/core/datastore/page/encoding/RLECodecSuite.java
@@ -46,9 +46,9 @@ public class RLECodecSuite {
     TestData(byte[] inputByteData, byte[] expectedEncodedByteData) throws IOException, MemoryException {
       this.inputByteData = inputByteData;
       inputBytePage = ColumnPage.newPage(
-          new TableSpec.ColumnSpec("test", DataTypes.BYTE, ColumnType.MEASURE),
+          TableSpec.ColumnSpec.newInstance("test", DataTypes.BYTE, ColumnType.MEASURE),
           DataTypes.BYTE, inputByteData.length);
-      inputBytePage.setStatsCollector(PrimitivePageStatsCollector.newInstance(DataTypes.BYTE, 0, 0));
+      inputBytePage.setStatsCollector(PrimitivePageStatsCollector.newInstance(DataTypes.BYTE));
       for (int i = 0; i < inputByteData.length; i++) {
         inputBytePage.putData(i, inputByteData[i]);
       }
@@ -131,7 +131,7 @@ public class RLECodecSuite {
   private void testBytePageDecode(byte[] inputBytes, byte[] expectedDecodedBytes) throws IOException, MemoryException {
     RLECodec codec = new RLECodec();
     RLEEncoderMeta meta = new RLEEncoderMeta(
-        new TableSpec.ColumnSpec("test", DataTypes.BYTE, ColumnType.MEASURE),
+        TableSpec.ColumnSpec.newInstance("test", DataTypes.BYTE, ColumnType.MEASURE),
         DataTypes.BYTE, expectedDecodedBytes.length, null);
     ColumnPageDecoder decoder = codec.createDecoder(meta);
     ColumnPage page = decoder.decode(inputBytes, 0, inputBytes.length);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/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 4a3ef32..42c0ad6 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,6 +23,8 @@ 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;
@@ -874,7 +876,7 @@ public class ThriftWrapperSchemaConverterImplTest {
       }
 
       @Mock public DataType getDataType() {
-        return DataTypes.DECIMAL;
+        return DataTypes.createDefaultDecimalType();
       }
 
       @Mock public String getColumnName() {
@@ -1158,7 +1160,12 @@ public class ThriftWrapperSchemaConverterImplTest {
   @Test public void testFromWrapperToExternalColumnSchemaForDatatypeNullCase() {
 
     org.apache.carbondata.format.ColumnSchema thriftColumnSchema =
-        new org.apache.carbondata.format.ColumnSchema(null, "columnName", "1", true, encoders,
+        new org.apache.carbondata.format.ColumnSchema(
+            org.apache.carbondata.format.DataType.INT,
+            "columnName",
+            "1",
+            true,
+            encoders,
             true);
     thriftColumnSchema.setSchemaOrdinal(1);
 
@@ -1172,7 +1179,7 @@ public class ThriftWrapperSchemaConverterImplTest {
       }
 
       @Mock public DataType getDataType() {
-        return null;
+        return DataTypes.INT;
       }
 
       @Mock public String getColumnName() {
@@ -1548,9 +1555,7 @@ long time =1112745600000L;
     wrapperColumnSchema.setDimensionColumn(true);
     wrapperColumnSchema.setEncodingList(encodings);
     wrapperColumnSchema.setNumberOfChild(1);
-    wrapperColumnSchema.setPrecision(1);
     wrapperColumnSchema.setColumnGroup(1);
-    wrapperColumnSchema.setScale(1);
     wrapperColumnSchema.setDefaultValue(new byte[] { 1, 2 });
     wrapperColumnSchema.setColumnProperties(columnPropertyMap);
     wrapperColumnSchema.setInvisible(true);
@@ -1686,8 +1691,9 @@ long time =1112745600000L;
   @Test public void testFromExternalToWrapperColumnSchemaForDatatypeNullCase() {
 
     org.apache.carbondata.format.ColumnSchema thriftColumnSchema =
-        new org.apache.carbondata.format.ColumnSchema(null, "columnName", "1", true, encoders,
-            true);
+        new org.apache.carbondata.format.ColumnSchema(
+            org.apache.carbondata.format.DataType.STRING,
+            "columnName", "1", true, encoders, true);
     ColumnSchema actualResult =
         thriftWrapperSchemaConverter.fromExternalToWrapperColumnSchema(thriftColumnSchema);
     Boolean expectedResult = true;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/test/java/org/apache/carbondata/core/scan/executor/util/RestructureUtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/executor/util/RestructureUtilTest.java b/core/src/test/java/org/apache/carbondata/core/scan/executor/util/RestructureUtilTest.java
index 048b194..172a53e 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/executor/util/RestructureUtilTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/executor/util/RestructureUtilTest.java
@@ -45,22 +45,27 @@ public class RestructureUtilTest {
     encodingList.add(Encoding.DICTIONARY);
     ColumnSchema columnSchema1 = new ColumnSchema();
     columnSchema1.setColumnName("Id");
+    columnSchema1.setDataType(DataTypes.STRING);
     columnSchema1.setColumnUniqueId(UUID.randomUUID().toString());
     columnSchema1.setEncodingList(encodingList);
     ColumnSchema columnSchema2 = new ColumnSchema();
     columnSchema2.setColumnName("Name");
+    columnSchema2.setDataType(DataTypes.STRING);
     columnSchema2.setColumnUniqueId(UUID.randomUUID().toString());
     columnSchema2.setEncodingList(encodingList);
     ColumnSchema columnSchema3 = new ColumnSchema();
     columnSchema3.setColumnName("Age");
+    columnSchema3.setDataType(DataTypes.INT);
     columnSchema3.setColumnUniqueId(UUID.randomUUID().toString());
     columnSchema3.setEncodingList(encodingList);
     ColumnSchema columnSchema4 = new ColumnSchema();
     columnSchema4.setColumnName("Salary");
+    columnSchema4.setDataType(DataTypes.INT);
     columnSchema4.setColumnUniqueId(UUID.randomUUID().toString());
     columnSchema4.setEncodingList(encodingList);
     ColumnSchema columnSchema5 = new ColumnSchema();
     columnSchema5.setColumnName("Address");
+    columnSchema5.setDataType(DataTypes.STRING);
     columnSchema5.setColumnUniqueId(UUID.randomUUID().toString());
     columnSchema5.setEncodingList(encodingList);
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/test/java/org/apache/carbondata/core/scan/expression/ExpressionResultTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/expression/ExpressionResultTest.java b/core/src/test/java/org/apache/carbondata/core/scan/expression/ExpressionResultTest.java
index cdc7fb9..1be4c2e 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/expression/ExpressionResultTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/expression/ExpressionResultTest.java
@@ -26,6 +26,7 @@ import java.util.List;
 import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
+import org.apache.carbondata.core.metadata.datatype.DecimalType;
 import org.apache.carbondata.core.scan.expression.exception.FilterIllegalMemberException;
 
 import org.junit.BeforeClass;
@@ -255,7 +256,7 @@ public class ExpressionResultTest {
   }
 
   @Test public void testGetDecimalForDecimal() throws FilterIllegalMemberException {
-    ExpressionResult expressionResultForDecimal = new ExpressionResult(DataTypes.DECIMAL, 5);
+    ExpressionResult expressionResultForDecimal = new ExpressionResult(DataTypes.createDefaultDecimalType(), 5);
     BigDecimal actualValue = expressionResultForDecimal.getDecimal();
     BigDecimal expectedValue = new BigDecimal(5);
     assertEquals(expectedValue, actualValue);
@@ -403,8 +404,8 @@ public class ExpressionResultTest {
   }
 
   @Test public void testEqualsForDecimal() {
-    ExpressionResult expressionResult = new ExpressionResult(DataTypes.DECIMAL, 5);
-    ExpressionResult objToCompare = new ExpressionResult(DataTypes.DECIMAL, 6);
+    ExpressionResult expressionResult = new ExpressionResult(DataTypes.createDefaultDecimalType(), 5);
+    ExpressionResult objToCompare = new ExpressionResult(DataTypes.createDefaultDecimalType(), 6);
     boolean result = expressionResult.equals(objToCompare);
     assertFalse(result);
   }
@@ -443,8 +444,8 @@ public class ExpressionResultTest {
   }
 
   @Test public void compareToForDecimal() {
-    ExpressionResult obj = new ExpressionResult(DataTypes.DECIMAL, 5);
-    ExpressionResult expressionResult = new ExpressionResult(DataTypes.DECIMAL, 6);
+    ExpressionResult obj = new ExpressionResult(DataTypes.createDefaultDecimalType(), 5);
+    ExpressionResult expressionResult = new ExpressionResult(DataTypes.createDefaultDecimalType(), 6);
     int actualValue = expressionResult.compareTo(obj);
     int expectedValue = 1;
     assertEquals(expectedValue, actualValue);
@@ -452,7 +453,7 @@ public class ExpressionResultTest {
 
   @Test public void compareToForException() {
     ExpressionResult obj = new ExpressionResult(DataTypes.INT, 5);
-    ExpressionResult expressionResult = new ExpressionResult(DataTypes.DECIMAL, 6);
+    ExpressionResult expressionResult = new ExpressionResult(DataTypes.createDefaultDecimalType(), 6);
     int actualValue = expressionResult.compareTo(obj);
     int expectedValue = -1;
     assertEquals(expectedValue, actualValue);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/EqualToExpressionUnitTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/EqualToExpressionUnitTest.java b/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/EqualToExpressionUnitTest.java
index 51bb0fe..cef1953 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/EqualToExpressionUnitTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/EqualToExpressionUnitTest.java
@@ -25,6 +25,7 @@ import java.util.Date;
 
 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.scan.expression.ColumnExpression;
 import org.apache.carbondata.core.scan.expression.ExpressionResult;
 import org.apache.carbondata.core.scan.expression.exception.FilterIllegalMemberException;
@@ -294,7 +295,7 @@ public class EqualToExpressionUnitTest {
 
   @Test public void testEvaluateForEqualToExpressionWithDecimalDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("contact", DataTypes.DECIMAL);
+    ColumnExpression right = new ColumnExpression("contact", DataTypes.createDefaultDecimalType());
     right.setColIndex(0);
     equalToExpression = new EqualToExpression(right, right);
     RowImpl value = new RowImpl();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/GreaterThanEqualToExpressionUnitTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/GreaterThanEqualToExpressionUnitTest.java b/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/GreaterThanEqualToExpressionUnitTest.java
index 0375a6d..de45721 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/GreaterThanEqualToExpressionUnitTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/GreaterThanEqualToExpressionUnitTest.java
@@ -25,6 +25,7 @@ import java.util.Date;
 
 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.scan.expression.ColumnExpression;
 import org.apache.carbondata.core.scan.expression.ExpressionResult;
 import org.apache.carbondata.core.scan.expression.exception.FilterIllegalMemberException;
@@ -205,9 +206,9 @@ public class GreaterThanEqualToExpressionUnitTest {
 
   @Test public void testEvaluateForGreaterThanEqualToExpressionWithDecimalDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("contact", DataTypes.DECIMAL);
+    ColumnExpression right = new ColumnExpression("contact", DataTypes.createDefaultDecimalType());
     right.setColIndex(0);
-    ColumnExpression left = new ColumnExpression("contact", DataTypes.DECIMAL);
+    ColumnExpression left = new ColumnExpression("contact", DataTypes.createDefaultDecimalType());
     left.setColIndex(1);
     greaterThanEqualToExpression = new GreaterThanEqualToExpression(left, right);
     RowImpl value = new RowImpl();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/GreaterThanExpressionUnitTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/GreaterThanExpressionUnitTest.java b/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/GreaterThanExpressionUnitTest.java
index 1940069..ef32f69 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/GreaterThanExpressionUnitTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/GreaterThanExpressionUnitTest.java
@@ -25,6 +25,7 @@ import java.util.Date;
 
 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.scan.expression.ColumnExpression;
 import org.apache.carbondata.core.scan.expression.ExpressionResult;
 import org.apache.carbondata.core.scan.expression.exception.FilterIllegalMemberException;
@@ -251,9 +252,9 @@ public class GreaterThanExpressionUnitTest {
 
   @Test public void testEvaluateForGreaterThanExpressionWithDecimalDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("contact", DataTypes.DECIMAL);
+    ColumnExpression right = new ColumnExpression("contact", DataTypes.createDefaultDecimalType());
     right.setColIndex(0);
-    ColumnExpression left = new ColumnExpression("contact", DataTypes.DECIMAL);
+    ColumnExpression left = new ColumnExpression("contact", DataTypes.createDefaultDecimalType());
     left.setColIndex(1);
     greaterThanExpression = new GreaterThanExpression(left, right);
     RowImpl value = new RowImpl();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/InExpressionUnitTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/InExpressionUnitTest.java b/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/InExpressionUnitTest.java
index d31ec31..aa3c208 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/InExpressionUnitTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/InExpressionUnitTest.java
@@ -25,6 +25,7 @@ import java.util.Date;
 
 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.scan.expression.ColumnExpression;
 import org.apache.carbondata.core.scan.expression.ExpressionResult;
 import org.apache.carbondata.core.scan.expression.exception.FilterIllegalMemberException;
@@ -203,9 +204,9 @@ public class InExpressionUnitTest {
 
   @Test public void testEvaluateForInExpressionWithDecimalDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression left = new ColumnExpression("left_contact", DataTypes.DECIMAL);
+    ColumnExpression left = new ColumnExpression("left_contact", DataTypes.createDefaultDecimalType());
     left.setColIndex(0);
-    ColumnExpression right = new ColumnExpression("right_contact", DataTypes.DECIMAL);
+    ColumnExpression right = new ColumnExpression("right_contact", DataTypes.createDefaultDecimalType());
     right.setColIndex(1);
     inExpression = new InExpression(left, right);
     RowImpl value = new RowImpl();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/LessThanEqualToExpressionUnitTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/LessThanEqualToExpressionUnitTest.java b/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/LessThanEqualToExpressionUnitTest.java
index a2a5d2a..9b0aa66 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/LessThanEqualToExpressionUnitTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/LessThanEqualToExpressionUnitTest.java
@@ -25,6 +25,7 @@ import java.util.Date;
 
 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.scan.expression.ColumnExpression;
 import org.apache.carbondata.core.scan.expression.ExpressionResult;
 import org.apache.carbondata.core.scan.expression.exception.FilterIllegalMemberException;
@@ -254,9 +255,9 @@ public class LessThanEqualToExpressionUnitTest {
 
   @Test public void testEvaluateForLessThanEqualToExpressionWithDecimalDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("right_contact", DataTypes.DECIMAL);
+    ColumnExpression right = new ColumnExpression("right_contact", DataTypes.createDefaultDecimalType());
     right.setColIndex(0);
-    ColumnExpression left = new ColumnExpression("left_contact", DataTypes.DECIMAL);
+    ColumnExpression left = new ColumnExpression("left_contact", DataTypes.createDefaultDecimalType());
     left.setColIndex(1);
     lessThanEqualToExpression = new LessThanEqualToExpression(left, right);
     RowImpl value = new RowImpl();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/LessThanExpressionUnitTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/LessThanExpressionUnitTest.java b/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/LessThanExpressionUnitTest.java
index d385dc1..4bd02af 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/LessThanExpressionUnitTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/LessThanExpressionUnitTest.java
@@ -25,6 +25,7 @@ import java.util.Date;
 
 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.scan.expression.ColumnExpression;
 import org.apache.carbondata.core.scan.expression.ExpressionResult;
 import org.apache.carbondata.core.scan.expression.exception.FilterIllegalMemberException;
@@ -251,9 +252,9 @@ public class LessThanExpressionUnitTest {
 
  @Test public void testEvaluateForLessThanExpressionWithDecimalDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("contact", DataTypes.DECIMAL);
+    ColumnExpression right = new ColumnExpression("contact", DataTypes.createDefaultDecimalType());
     right.setColIndex(0);
-    ColumnExpression left = new ColumnExpression("contact", DataTypes.DECIMAL);
+    ColumnExpression left = new ColumnExpression("contact", DataTypes.createDefaultDecimalType());
     left.setColIndex(1);
     lessThanExpression = new LessThanExpression(left, right);
     RowImpl value = new RowImpl();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/NotEqualsExpressionUnitTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/NotEqualsExpressionUnitTest.java b/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/NotEqualsExpressionUnitTest.java
index e7fa544..5ee6a15 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/NotEqualsExpressionUnitTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/NotEqualsExpressionUnitTest.java
@@ -25,6 +25,7 @@ import java.util.Date;
 
 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.scan.expression.ColumnExpression;
 import org.apache.carbondata.core.scan.expression.ExpressionResult;
 import org.apache.carbondata.core.scan.expression.exception.FilterIllegalMemberException;
@@ -261,9 +262,9 @@ public class NotEqualsExpressionUnitTest {
 
   @Test public void testEvaluateWithForNotEqualsExpressionDecimalDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression right = new ColumnExpression("contact", DataTypes.DECIMAL);
+    ColumnExpression right = new ColumnExpression("contact", DataTypes.createDefaultDecimalType());
     right.setColIndex(1);
-    ColumnExpression left = new ColumnExpression("contact", DataTypes.DECIMAL);
+    ColumnExpression left = new ColumnExpression("contact", DataTypes.createDefaultDecimalType());
     left.setColIndex(0);
     notEqualsExpression = new NotEqualsExpression(left, right);
     RowImpl value = new RowImpl();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/NotInExpressionUnitTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/NotInExpressionUnitTest.java b/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/NotInExpressionUnitTest.java
index 5758625..c797ce9 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/NotInExpressionUnitTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/expression/conditional/NotInExpressionUnitTest.java
@@ -25,6 +25,7 @@ import java.util.Date;
 
 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.scan.expression.ColumnExpression;
 import org.apache.carbondata.core.scan.expression.ExpressionResult;
 import org.apache.carbondata.core.scan.expression.exception.FilterIllegalMemberException;
@@ -202,9 +203,9 @@ public class NotInExpressionUnitTest {
 
   @Test public void testEvaluateForNotInExpressionWithDecimalDataType()
       throws FilterUnsupportedException, FilterIllegalMemberException {
-    ColumnExpression left = new ColumnExpression("left_contact", DataTypes.DECIMAL);
+    ColumnExpression left = new ColumnExpression("left_contact", DataTypes.createDefaultDecimalType());
     left.setColIndex(0);
-    ColumnExpression right = new ColumnExpression("right_contact", DataTypes.DECIMAL);
+    ColumnExpression right = new ColumnExpression("right_contact", DataTypes.createDefaultDecimalType());
     right.setColIndex(1);
     notInExpression = new NotInExpression(left, right);
     RowImpl value = new RowImpl();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/test/java/org/apache/carbondata/core/scan/filter/FilterUtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/filter/FilterUtilTest.java b/core/src/test/java/org/apache/carbondata/core/scan/filter/FilterUtilTest.java
index a6f483b..f1f05f0 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/filter/FilterUtilTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/filter/FilterUtilTest.java
@@ -27,6 +27,7 @@ import org.apache.carbondata.core.datastore.IndexKey;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
 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.column.CarbonColumn;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
@@ -149,7 +150,7 @@ public class FilterUtilTest extends AbstractDictionaryCacheTest {
     String dictionaryVal = "1111111";
     String memberVal = "1111";
     int actualResult =
-        FilterUtil.compareFilterKeyBasedOnDataType(dictionaryVal, memberVal, DataTypes.DECIMAL);
+        FilterUtil.compareFilterKeyBasedOnDataType(dictionaryVal, memberVal, DataTypes.createDefaultDecimalType());
     int expectedResult = 1;
     assertEquals(expectedResult, actualResult);
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/test/java/org/apache/carbondata/core/util/CarbonMetadataUtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/util/CarbonMetadataUtilTest.java b/core/src/test/java/org/apache/carbondata/core/util/CarbonMetadataUtilTest.java
index ca78e66..4cf3d11 100644
--- a/core/src/test/java/org/apache/carbondata/core/util/CarbonMetadataUtilTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/util/CarbonMetadataUtilTest.java
@@ -189,7 +189,7 @@ public class CarbonMetadataUtilTest {
 
     final EncodedColumnPage measure = new EncodedColumnPage(new DataChunk2(), new byte[]{0,1},
         PrimitivePageStatsCollector.newInstance(
-        org.apache.carbondata.core.metadata.datatype.DataTypes.BYTE, 0, 0));
+        org.apache.carbondata.core.metadata.datatype.DataTypes.BYTE));
     new MockUp<EncodedTablePage>() {
       @SuppressWarnings("unused") @Mock
       public EncodedColumnPage getMeasure(int measureIndex) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/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 658dda9..56c59e1 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
@@ -40,6 +40,7 @@ import org.apache.carbondata.core.metadata.blocklet.DataFileFooter;
 import org.apache.carbondata.core.metadata.blocklet.datachunk.DataChunk;
 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.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
@@ -508,7 +509,7 @@ public class CarbonUtilTest {
   }
 
   @Test public void testForHasDataTypes() {
-    DataType[] dataTypes = { DataTypes.DECIMAL, DataTypes.BOOLEAN, DataTypes.INT };
+    DataType[] dataTypes = {DataTypes.createDefaultDecimalType(), DataTypes.BOOLEAN, DataTypes.INT };
     assertTrue(CarbonUtil.hasDataType(DataTypes.BOOLEAN, dataTypes));
     assertTrue(!CarbonUtil.hasDataType(DataTypes.DATE, dataTypes));
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/test/java/org/apache/carbondata/core/util/DataTypeUtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/util/DataTypeUtilTest.java b/core/src/test/java/org/apache/carbondata/core/util/DataTypeUtilTest.java
index b6ea765..c944d18 100644
--- a/core/src/test/java/org/apache/carbondata/core/util/DataTypeUtilTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/util/DataTypeUtilTest.java
@@ -21,6 +21,7 @@ import java.math.BigDecimal;
 import java.math.BigInteger;
 
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
+import org.apache.carbondata.core.metadata.datatype.DecimalType;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
 
@@ -30,9 +31,7 @@ import static junit.framework.TestCase.assertEquals;
 import static junit.framework.TestCase.assertTrue;
 import static org.apache.carbondata.core.util.DataTypeUtil.bigDecimalToByte;
 import static org.apache.carbondata.core.util.DataTypeUtil.byteToBigDecimal;
-import static org.apache.carbondata.core.util.DataTypeUtil.getColumnDataTypeDisplayName;
 import static org.apache.carbondata.core.util.DataTypeUtil.getDataBasedOnDataType;
-import static org.apache.carbondata.core.util.DataTypeUtil.getDataType;
 import static org.apache.carbondata.core.util.DataTypeUtil.getMeasureValueBasedOnDataType;
 import static org.apache.carbondata.core.util.DataTypeUtil.normalizeIntAndLongValues;
 
@@ -40,7 +39,7 @@ public class DataTypeUtilTest {
 
   @Test public void testGetColumnDataTypeDisplayName() {
     String expected = DataTypes.INT.getName();
-    String result = getColumnDataTypeDisplayName("INT");
+    String result = "INT";
     assertEquals(expected, result);
 
   }
@@ -60,22 +59,6 @@ public class DataTypeUtilTest {
     assertTrue(result == result);
   }
 
-  @Test public void testGetDataType() {
-    assertEquals(DataTypes.TIMESTAMP, getDataType("TIMESTAMP"));
-    assertEquals(DataTypes.DATE, getDataType("DATE"));
-    assertEquals(DataTypes.STRING, getDataType("STRING"));
-    assertEquals(DataTypes.INT, getDataType("INT"));
-    assertEquals(DataTypes.SHORT, getDataType("SMALLINT"));
-    assertEquals(DataTypes.LONG, getDataType("LONG"));
-    assertEquals(DataTypes.DOUBLE, getDataType("DOUBLE"));
-    assertEquals(DataTypes.DECIMAL, getDataType("DECIMAL"));
-    assertEquals(DataTypes.ARRAY, getDataType("ARRAY"));
-    assertEquals(DataTypes.STRUCT, getDataType("STRUCT"));
-    assertEquals(DataTypes.STRING, getDataType("MAP"));
-    assertEquals(DataTypes.STRING, getDataType("default"));
-
-  }
-
   @Test public void testGetDataBasedOnDataType() throws NumberFormatException {
     String data = " ";
     if (data.isEmpty()) {
@@ -87,7 +70,7 @@ public class DataTypeUtilTest {
     assertEquals(getDataBasedOnDataType("0", DataTypes.LONG), 0L);
     java.math.BigDecimal javaDecVal = new java.math.BigDecimal(1);
     scala.math.BigDecimal scalaDecVal = new scala.math.BigDecimal(javaDecVal);
-    assertEquals(getDataBasedOnDataType("1", DataTypes.DECIMAL),
+    assertEquals(getDataBasedOnDataType("1", DataTypes.createDefaultDecimalType()),
         DataTypeUtil.getDataTypeConverter().convertToDecimal(scalaDecVal));
     assertEquals(getDataBasedOnDataType("default", DataTypes.NULL),
         DataTypeUtil.getDataTypeConverter().convertFromStringToUTF8String("default"));

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/test/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortModelTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortModelTest.java b/core/src/test/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortModelTest.java
index a42b214..f89128d 100644
--- a/core/src/test/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortModelTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortModelTest.java
@@ -18,6 +18,7 @@ package org.apache.carbondata.core.writer.sortindex;
 
 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.junit.Test;
 
@@ -69,27 +70,27 @@ public class CarbonDictionarySortModelTest {
   }
 
   @Test public void testCompareToForDataTypeDecimalCase() {
-    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.DECIMAL, "72.34");
+    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.createDefaultDecimalType(), "72.34");
     CarbonDictionarySortModel testCarbonDictionarySortModel =
-        new CarbonDictionarySortModel(2, DataTypes.DECIMAL, "56.78");
+        new CarbonDictionarySortModel(2, DataTypes.createDefaultDecimalType(), "56.78");
     int expectedResult = 1;
     int actualResult = carbonDictionarySortModel.compareTo(testCarbonDictionarySortModel);
     assertEquals(actualResult, expectedResult);
   }
 
   @Test public void testCompareToForDataTypeDecimalExceptionCase() {
-    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.DECIMAL, "decimal");
+    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.createDefaultDecimalType(), "decimal");
     CarbonDictionarySortModel testCarbonDictionarySortModel =
-        new CarbonDictionarySortModel(2, DataTypes.DECIMAL, "@NU#LL$!");
+        new CarbonDictionarySortModel(2, DataTypes.createDefaultDecimalType(), "@NU#LL$!");
     int expectedResult = -1;
     int actualResult = carbonDictionarySortModel.compareTo(testCarbonDictionarySortModel);
     assertEquals(actualResult, expectedResult);
   }
 
   @Test public void testCompareToForDataTypeDecimalExceptionCaseForOtherObject() {
-    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.DECIMAL, "15.24");
+    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.createDefaultDecimalType(), "15.24");
     CarbonDictionarySortModel testCarbonDictionarySortModel =
-        new CarbonDictionarySortModel(2, DataTypes.DECIMAL, "@NU#LL$!");
+        new CarbonDictionarySortModel(2, DataTypes.createDefaultDecimalType(), "@NU#LL$!");
     int expectedResult = -1;
     int actualResult = carbonDictionarySortModel.compareTo(testCarbonDictionarySortModel);
     assertEquals(actualResult, expectedResult);
@@ -126,28 +127,28 @@ public class CarbonDictionarySortModelTest {
   }
 
   @Test public void testHashCode() {
-    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.DECIMAL, "15.24");
+    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.createDefaultDecimalType(), "15.24");
     int actualResult = carbonDictionarySortModel.hashCode();
     int expectedResult = 46877260;
     assertTrue(actualResult == expectedResult);
   }
 
   @Test public void testHashCodeNullCaseForMemberValue() {
-    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.DECIMAL, null);
+    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.createDefaultDecimalType(), null);
     int actualResult = carbonDictionarySortModel.hashCode();
     int expectedResult = 0;
     assertEquals(actualResult, expectedResult);
   }
 
   @Test public void testEquals() {
-    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.DECIMAL, "15.24");
+    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.createDefaultDecimalType(), "15.24");
     CarbonDictionarySortModel testCarbonDictionarySortModel = carbonDictionarySortModel;
     boolean result = carbonDictionarySortModel.equals(testCarbonDictionarySortModel);
     assertTrue(result);
   }
 
   @Test public void testEqualsMemberValueNullCase() {
-    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.DECIMAL, null);
+    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.createDefaultDecimalType(), null);
     CarbonDictionarySortModel testCarbonDictionarySortModel =
         new CarbonDictionarySortModel(1, DataTypes.BOOLEAN, "false");
     boolean result = carbonDictionarySortModel.equals(testCarbonDictionarySortModel);
@@ -155,7 +156,7 @@ public class CarbonDictionarySortModelTest {
   }
 
   @Test public void testEqualsWhenMemberValueDiffers() {
-    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.DECIMAL, "12.45");
+    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.createDefaultDecimalType(), "12.45");
     CarbonDictionarySortModel testCarbonDictionarySortModel =
         new CarbonDictionarySortModel(1, DataTypes.BOOLEAN, "false");
     boolean result = carbonDictionarySortModel.equals(testCarbonDictionarySortModel);
@@ -163,15 +164,15 @@ public class CarbonDictionarySortModelTest {
   }
 
   @Test public void testEqualsWhenMemberValueIsSame() {
-    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.DECIMAL, "12.45");
+    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.createDefaultDecimalType(), "12.45");
     CarbonDictionarySortModel testCarbonDictionarySortModel =
-        new CarbonDictionarySortModel(1, DataTypes.DECIMAL, "12.45");
+        new CarbonDictionarySortModel(1, DataTypes.createDefaultDecimalType(), "12.45");
     boolean result = carbonDictionarySortModel.equals(testCarbonDictionarySortModel);
     assertTrue(result);
   }
 
   @Test public void testEqualsForDifferentObjects() {
-    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.DECIMAL, "12.45");
+    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.createDefaultDecimalType(), "12.45");
     Object testCarbonDictionarySortModel = new Object();
     boolean result = carbonDictionarySortModel.equals(testCarbonDictionarySortModel);
     assertFalse(result);
@@ -187,9 +188,9 @@ public class CarbonDictionarySortModelTest {
   }
 
   @Test public void testCompareToForDataTypeDecimalExceptionCaseForDifferentObject() {
-    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.DECIMAL, "12.il");
+    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.createDefaultDecimalType(), "12.il");
     CarbonDictionarySortModel testCarbonDictionarySortModel =
-        new CarbonDictionarySortModel(2, DataTypes.DECIMAL, "12.89");
+        new CarbonDictionarySortModel(2, DataTypes.createDefaultDecimalType(), "12.89");
     int expectedResult = 1;
     int actualResult = carbonDictionarySortModel.compareTo(testCarbonDictionarySortModel);
     assertEquals(actualResult, expectedResult);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/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 36ae65c..05ad93a 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
@@ -239,7 +239,7 @@ public class CarbonDictionaryDecodeReadSupport<T> implements CarbonReadSupport<T
       return new TimestampWritable(new Timestamp((long) obj / 1000));
     } else if (dataType == DataTypes.STRING) {
       return new Text(obj.toString());
-    } else if (dataType == DataTypes.DECIMAL) {
+    } else if (DataTypes.isDecimal(dataType)) {
       return new HiveDecimalWritable(HiveDecimal.create(new java.math.BigDecimal(obj.toString())));
     } else {
       throw new IOException("unsupported data type:" + dataType);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/integration/presto/src/main/java/org/apache/carbondata/presto/CarbonTypeUtil.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbonTypeUtil.java b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbonTypeUtil.java
index e2c9c68..ddc8d9e 100644
--- a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbonTypeUtil.java
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbonTypeUtil.java
@@ -20,7 +20,7 @@ public class CarbonTypeUtil {
         return DataTypes.DoubleType;
     } else if (carbonDataType == org.apache.carbondata.core.metadata.datatype.DataTypes.BOOLEAN) {
         return DataTypes.BooleanType;
-    } else if (carbonDataType == org.apache.carbondata.core.metadata.datatype.DataTypes.DECIMAL) {
+    } else if (org.apache.carbondata.core.metadata.datatype.DataTypes.isDecimal(carbonDataType)) {
         return DataTypes.createDecimalType();
     } else if (carbonDataType == org.apache.carbondata.core.metadata.datatype.DataTypes.TIMESTAMP) {
         return DataTypes.TimestampType;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/integration/presto/src/main/java/org/apache/carbondata/presto/CarbonVectorizedRecordReader.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbonVectorizedRecordReader.java b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbonVectorizedRecordReader.java
index 2db2d23..f129474 100644
--- a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbonVectorizedRecordReader.java
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbonVectorizedRecordReader.java
@@ -205,7 +205,7 @@ class CarbonVectorizedRecordReader extends AbstractRecordReader<Object> {
         fields[msr.getQueryOrder()] = new StructField(msr.getColumnName(),
             CarbonTypeUtil.convertCarbonToSparkDataType(msr.getMeasure().getDataType()), true,
             null);
-      } else if (dataType == DataTypes.DECIMAL) {
+      } else if (DataTypes.isDecimal(dataType)) {
         fields[msr.getQueryOrder()] = new StructField(msr.getColumnName(),
             new DecimalType(msr.getMeasure().getPrecision(), msr.getMeasure().getScale()), true,
             null);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/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 4a9b7ed..fb9a0ba 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
@@ -76,8 +76,10 @@ public class CarbondataColumnHandle implements ColumnHandle {
     return columnUniqueId;
   }
 
-  @JsonCreator public CarbondataColumnHandle(@JsonProperty("connectorId") String connectorId,
-      @JsonProperty("columnName") String columnName, @JsonProperty("columnType") Type columnType,
+  @JsonCreator public CarbondataColumnHandle(
+      @JsonProperty("connectorId") String connectorId,
+      @JsonProperty("columnName") String columnName,
+      @JsonProperty("columnType") Type columnType,
       @JsonProperty("ordinalPosition") int ordinalPosition,
       @JsonProperty("keyOrdinal") int keyOrdinal,
       @JsonProperty("columnGroupOrdinal") int columnGroupOrdinal,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataMetadata.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataMetadata.java b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataMetadata.java
index 99109d1..f106a08 100755
--- a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataMetadata.java
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataMetadata.java
@@ -232,7 +232,7 @@ public class CarbondataMetadata implements ConnectorMetadata {
       return BigintType.BIGINT;
     } else if (colType == DataTypes.FLOAT || colType == DataTypes.DOUBLE) {
       return DoubleType.DOUBLE;
-    } else if (colType == DataTypes.DECIMAL) {
+    } else if (DataTypes.isDecimal(colType)) {
       if (columnSchema.getPrecision() > 0) {
         return DecimalType.createDecimalType(columnSchema.getPrecision(), columnSchema.getScale());
       } else {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/integration/presto/src/main/java/org/apache/carbondata/presto/PrestoFilterUtil.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/PrestoFilterUtil.java b/integration/presto/src/main/java/org/apache/carbondata/presto/PrestoFilterUtil.java
index c9fb177..0982254 100644
--- a/integration/presto/src/main/java/org/apache/carbondata/presto/PrestoFilterUtil.java
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/PrestoFilterUtil.java
@@ -79,7 +79,10 @@ public class PrestoFilterUtil {
     else if (colType == DateType.DATE) return DataTypes.DATE;
     else if (colType == TimestampType.TIMESTAMP) return DataTypes.TIMESTAMP;
     else if (colType.equals(DecimalType.createDecimalType(carbondataColumnHandle.getPrecision(),
-        carbondataColumnHandle.getScale()))) return DataTypes.DECIMAL;
+        carbondataColumnHandle.getScale())))
+      return org.apache.carbondata.core.metadata.datatype.DataTypes.createDecimalType(
+          carbondataColumnHandle.getPrecision(),
+          carbondataColumnHandle.getScale());
     else return DataTypes.STRING;
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala
----------------------------------------------------------------------
diff --git a/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala b/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala
index b8031a2..acee71b 100644
--- a/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala
+++ b/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala
@@ -40,7 +40,7 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.fileoperations.{AtomicFileOperations, AtomicFileOperationsImpl, FileWriteOperation}
 import org.apache.carbondata.core.metadata.converter.{SchemaConverter, ThriftWrapperSchemaConverterImpl}
-import org.apache.carbondata.core.metadata.datatype.{DataType, DataTypes}
+import org.apache.carbondata.core.metadata.datatype.DataTypes
 import org.apache.carbondata.core.metadata.encoder.Encoding
 import org.apache.carbondata.core.metadata.schema.table.column.{CarbonColumn, CarbonDimension, CarbonMeasure, ColumnSchema}
 import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, TableInfo, TableSchema}
@@ -236,7 +236,7 @@ object CarbonDataStoreCreator {
     val bonus: ColumnSchema = new ColumnSchema()
     bonus.setColumnName("bonus")
     bonus.setColumnar(true)
-    bonus.setDataType(DataTypes.DECIMAL)
+    bonus.setDataType(DataTypes.createDecimalType(10, 4))
     bonus.setPrecision(10)
     bonus.setScale(4)
     bonus.setEncodingList(encodings)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/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 cf37a18..2043ecf 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
@@ -25,12 +25,11 @@ 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}
+import org.apache.carbondata.core.metadata.datatype.{DataType => CarbonDataType, DataTypes => CarbonDataTypes, DecimalType => CarbonDecimalType}
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn
 
 object CarbonScalaUtil {
-  def convertSparkToCarbonDataType(
-      dataType: org.apache.spark.sql.types.DataType): CarbonDataType = {
+  def convertSparkToCarbonDataType(dataType: DataType): CarbonDataType = {
     dataType match {
       case StringType => CarbonDataTypes.STRING
       case ShortType => CarbonDataTypes.SHORT
@@ -44,7 +43,9 @@ object CarbonScalaUtil {
       case ArrayType(_, _) => CarbonDataTypes.ARRAY
       case StructType(_) => CarbonDataTypes.STRUCT
       case NullType => CarbonDataTypes.NULL
-      case _ => CarbonDataTypes.DECIMAL
+      case decimal: DecimalType =>
+        CarbonDataTypes.createDecimalType(decimal.precision, decimal.scale)
+      case _ => throw new UnsupportedOperationException("getting " + dataType + " from spark")
     }
   }
 
@@ -66,27 +67,19 @@ object CarbonScalaUtil {
   }
 
   def convertCarbonToSparkDataType(dataType: CarbonDataType): types.DataType = {
-    dataType match {
-      case CarbonDataTypes.STRING => StringType
-      case CarbonDataTypes.SHORT => ShortType
-      case CarbonDataTypes.INT => IntegerType
-      case CarbonDataTypes.LONG => LongType
-      case CarbonDataTypes.DOUBLE => DoubleType
-      case CarbonDataTypes.BOOLEAN => BooleanType
-      case CarbonDataTypes.DECIMAL => DecimalType.SYSTEM_DEFAULT
-      case CarbonDataTypes.TIMESTAMP => TimestampType
-      case CarbonDataTypes.DATE => DateType
-    }
-  }
-
-  def updateDataType(
-      currentDataType: org.apache.spark.sql.types.DataType): org.apache.spark.sql.types.DataType = {
-    currentDataType match {
-      case decimal: DecimalType =>
-        val scale = currentDataType.asInstanceOf[DecimalType].scale
-        DecimalType(DecimalType.MAX_PRECISION, scale)
-      case _ =>
-        currentDataType
+    if (CarbonDataTypes.isDecimal(dataType)) {
+      DecimalType.SYSTEM_DEFAULT
+    } else {
+      dataType match {
+        case CarbonDataTypes.STRING => StringType
+        case CarbonDataTypes.SHORT => ShortType
+        case CarbonDataTypes.INT => IntegerType
+        case CarbonDataTypes.LONG => LongType
+        case CarbonDataTypes.DOUBLE => DoubleType
+        case CarbonDataTypes.BOOLEAN => BooleanType
+        case CarbonDataTypes.TIMESTAMP => TimestampType
+        case CarbonDataTypes.DATE => DateType
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/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 89a3ac3..0460e30 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
@@ -17,7 +17,7 @@
 
 package org.apache.carbondata.spark.util
 
-import org.apache.carbondata.core.metadata.datatype.{DataType, DataTypes}
+import org.apache.carbondata.core.metadata.datatype.{DataType, DataTypes, DecimalType}
 import org.apache.carbondata.format.{DataType => ThriftDataType}
 
 object DataTypeConverterUtil {
@@ -37,8 +37,8 @@ object DataTypeConverterUtil {
       case "numeric" => DataTypes.DOUBLE
       case "double" => DataTypes.DOUBLE
       case "float" => DataTypes.DOUBLE
-      case "decimal" => DataTypes.DECIMAL
-      case FIXED_DECIMAL(_, _) => DataTypes.DECIMAL
+      case "decimal" => DataTypes.createDefaultDecimalType
+      case FIXED_DECIMAL(_, _) => DataTypes.createDefaultDecimalType
       case "timestamp" => DataTypes.TIMESTAMP
       case "date" => DataTypes.DATE
       case "array" => DataTypes.ARRAY
@@ -60,8 +60,8 @@ object DataTypeConverterUtil {
       case "numerictype" => DataTypes.DOUBLE
       case "doubletype" => DataTypes.DOUBLE
       case "floattype" => DataTypes.DOUBLE
-      case "decimaltype" => DataTypes.DECIMAL
-      case FIXED_DECIMALTYPE(_, _) => DataTypes.DECIMAL
+      case "decimaltype" => DataTypes.createDefaultDecimalType
+      case FIXED_DECIMALTYPE(_, _) => DataTypes.createDefaultDecimalType
       case "timestamptype" => DataTypes.TIMESTAMP
       case "datetype" => DataTypes.DATE
       case others =>
@@ -80,19 +80,22 @@ object DataTypeConverterUtil {
   }
 
   def convertToString(dataType: DataType): String = {
-    dataType match {
-      case DataTypes.BOOLEAN => "boolean"
+    if (DataTypes.isDecimal(dataType)) {
+      "decimal"
+    } else {
+      dataType match {
+        case DataTypes.BOOLEAN => "boolean"
       case DataTypes.STRING => "string"
-      case DataTypes.SHORT => "smallint"
-      case DataTypes.INT => "int"
-      case DataTypes.LONG => "bigint"
-      case DataTypes.DOUBLE => "double"
-      case DataTypes.FLOAT => "double"
-      case DataTypes.DECIMAL => "decimal"
-      case DataTypes.TIMESTAMP => "timestamp"
-      case DataTypes.DATE => "date"
-      case DataTypes.ARRAY => "array"
-      case DataTypes.STRUCT => "struct"
+        case DataTypes.SHORT => "smallint"
+        case DataTypes.INT => "int"
+        case DataTypes.LONG => "bigint"
+        case DataTypes.DOUBLE => "double"
+        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/f209e8ee/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 1163b3f..edb471e 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
@@ -622,7 +622,7 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
         dimFields += field
       } else if (dictIncludeCols.exists(x => x.equalsIgnoreCase(field.column))) {
         dimFields += field
-      } else if (DataTypeUtil.getDataType(field.dataType.get.toUpperCase) == DataTypes.TIMESTAMP &&
+      } else if (field.dataType.get.toUpperCase.equals("TIMESTAMP") &&
                  !dictIncludeCols.exists(x => x.equalsIgnoreCase(field.column))) {
         noDictionaryDims :+= field.column
         dimFields += field


[14/49] carbondata git commit: [CARBONDATA-1669] Clean up code in CarbonDataRDDFactory

Posted by ra...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/0578ba0f/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
index 1ad25c3..47d7c95 100644
--- a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
@@ -34,19 +34,17 @@ import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat, FileSplit}
 import org.apache.spark.{SparkEnv, SparkException, TaskContext}
 import org.apache.spark.rdd.{DataLoadCoalescedRDD, DataLoadPartitionCoalescer, NewHadoopRDD, RDD}
 import org.apache.spark.sql.{CarbonEnv, DataFrame, Row, SQLContext}
-import org.apache.spark.sql.execution.command.{AlterTableModel, CompactionModel, ExecutionErrors, UpdateTableModel}
+import org.apache.spark.sql.execution.command.{CompactionModel, ExecutionErrors, UpdateTableModel}
 import org.apache.spark.sql.hive.DistributionUtil
-import org.apache.spark.util.SparkUtil
 
 import org.apache.carbondata.common.constants.LoggerAction
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.datamap.DataMapStoreManager
 import org.apache.carbondata.core.datastore.block.{Distributable, TableBlockInfo}
 import org.apache.carbondata.core.dictionary.server.DictionaryServer
 import org.apache.carbondata.core.locks.{CarbonLockFactory, ICarbonLock, LockUsage}
-import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier, ColumnarFormatVersion}
-import org.apache.carbondata.core.metadata.datatype.{DataType, DataTypes}
+import org.apache.carbondata.core.metadata.{CarbonTableIdentifier, ColumnarFormatVersion}
+import org.apache.carbondata.core.metadata.datatype.DataTypes
 import org.apache.carbondata.core.metadata.schema.partition.PartitionType
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.mutate.CarbonUpdateUtil
@@ -61,7 +59,7 @@ import org.apache.carbondata.processing.loading.csvinput.CSVInputFormat
 import org.apache.carbondata.processing.loading.csvinput.StringArrayWritable
 import org.apache.carbondata.processing.loading.exception.CarbonDataLoadingException
 import org.apache.carbondata.processing.loading.exception.NoRetryException
-import org.apache.carbondata.processing.loading.model.CarbonLoadModel
+import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, CarbonLoadModel}
 import org.apache.carbondata.processing.loading.sort.SortScopeOptions
 import org.apache.carbondata.processing.merger.{CarbonCompactionUtil, CarbonDataMergerUtil, CompactionType}
 import org.apache.carbondata.processing.util.{CarbonDataProcessorUtil, CarbonLoaderUtil}
@@ -77,153 +75,6 @@ object CarbonDataRDDFactory {
 
   private val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
 
-  def alterTableForCompaction(sqlContext: SQLContext,
-      alterTableModel: AlterTableModel,
-      carbonLoadModel: CarbonLoadModel,
-      storeLocation: String): Unit = {
-    var compactionSize: Long = 0
-    var compactionType: CompactionType = CompactionType.MINOR_COMPACTION
-    if (alterTableModel.compactionType.equalsIgnoreCase("major")) {
-      compactionSize = CarbonDataMergerUtil.getCompactionSize(CompactionType.MAJOR_COMPACTION)
-      compactionType = CompactionType.MAJOR_COMPACTION
-    } else if (alterTableModel.compactionType
-      .equalsIgnoreCase(CompactionType.IUD_UPDDEL_DELTA_COMPACTION.toString)) {
-      compactionType = CompactionType.IUD_UPDDEL_DELTA_COMPACTION
-      if (alterTableModel.segmentUpdateStatusManager.get != None) {
-        carbonLoadModel
-          .setSegmentUpdateStatusManager(alterTableModel.segmentUpdateStatusManager.get)
-
-        carbonLoadModel
-          .setLoadMetadataDetails(alterTableModel.segmentUpdateStatusManager.get
-            .getLoadMetadataDetails.toList.asJava)
-      }
-    } else if (alterTableModel.compactionType.
-      equalsIgnoreCase(CompactionType.SEGMENT_INDEX_COMPACTION.toString)) {
-      compactionType = CompactionType.SEGMENT_INDEX_COMPACTION
-    } else {
-      compactionType = CompactionType.MINOR_COMPACTION
-    }
-
-    LOGGER.audit(s"Compaction request received for table " +
-        s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
-    val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
-
-    if (null == carbonLoadModel.getLoadMetadataDetails) {
-      CommonUtil.readLoadMetadataDetails(carbonLoadModel)
-    }
-    if (compactionType == CompactionType.SEGMENT_INDEX_COMPACTION) {
-      // Just launch job to merge index and return
-      CommonUtil.mergeIndexFiles(sqlContext.sparkContext,
-        carbonLoadModel.getLoadMetadataDetails.asScala.map(_.getLoadName),
-        carbonLoadModel.getStorePath,
-        carbonTable)
-      return
-    }
-    // reading the start time of data load.
-    val loadStartTime : Long =
-    if (alterTableModel.factTimeStamp.isEmpty) {
-      CarbonUpdateUtil.readCurrentTime
-    } else {
-      alterTableModel.factTimeStamp.get
-    }
-    carbonLoadModel.setFactTimeStamp(loadStartTime)
-
-    val isCompactionTriggerByDDl = true
-    val compactionModel = CompactionModel(compactionSize,
-      compactionType,
-      carbonTable,
-      isCompactionTriggerByDDl
-    )
-
-    val isConcurrentCompactionAllowed = CarbonProperties.getInstance()
-        .getProperty(CarbonCommonConstants.ENABLE_CONCURRENT_COMPACTION,
-          CarbonCommonConstants.DEFAULT_ENABLE_CONCURRENT_COMPACTION
-        )
-        .equalsIgnoreCase("true")
-
-    // if system level compaction is enabled then only one compaction can run in the system
-    // if any other request comes at this time then it will create a compaction request file.
-    // so that this will be taken up by the compaction process which is executing.
-    if (!isConcurrentCompactionAllowed) {
-      LOGGER.info("System level compaction lock is enabled.")
-      handleCompactionForSystemLocking(sqlContext,
-        carbonLoadModel,
-        storeLocation,
-        compactionType,
-        carbonTable,
-        compactionModel
-      )
-    } else {
-      // normal flow of compaction
-      val lock = CarbonLockFactory
-          .getCarbonLockObj(carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
-            LockUsage.COMPACTION_LOCK
-          )
-
-      if (lock.lockWithRetries()) {
-        LOGGER.info("Acquired the compaction lock for table" +
-            s" ${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
-        try {
-          startCompactionThreads(sqlContext,
-            carbonLoadModel,
-            storeLocation,
-            compactionModel,
-            lock
-          )
-        } catch {
-          case e: Exception =>
-            LOGGER.error(s"Exception in start compaction thread. ${ e.getMessage }")
-            lock.unlock()
-            throw e
-        }
-      } else {
-        LOGGER.audit("Not able to acquire the compaction lock for table " +
-            s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
-        LOGGER.error(s"Not able to acquire the compaction lock for table" +
-            s" ${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
-        sys.error("Table is already locked for compaction. Please try after some time.")
-      }
-    }
-  }
-
-  def alterTableSplitPartition(sqlContext: SQLContext,
-      partitionId: String,
-      carbonLoadModel: CarbonLoadModel,
-      oldPartitionIdList: List[Int]): Unit = {
-    LOGGER.audit(s"Add partition request received for table " +
-         s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
-    try {
-      startSplitThreads(sqlContext,
-        carbonLoadModel,
-        partitionId,
-        oldPartitionIdList)
-    } catch {
-      case e: Exception =>
-        LOGGER.error(s"Exception in start splitting partition thread. ${ e.getMessage }")
-        throw e
-    }
-  }
-
-  def alterTableDropPartition(sqlContext: SQLContext,
-      partitionId: String,
-      carbonLoadModel: CarbonLoadModel,
-      dropWithData: Boolean,
-      oldPartitionIds: List[Int]): Unit = {
-    LOGGER.audit(s"Drop partition request received for table " +
-                 s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
-    try {
-      startDropThreads(sqlContext,
-        carbonLoadModel,
-        partitionId,
-        dropWithData,
-        oldPartitionIds)
-    } catch {
-      case e: Exception =>
-        LOGGER.error(s"Exception in start dropping partition thread. ${ e.getMessage }")
-        throw e
-    }
-  }
-
   def handleCompactionForSystemLocking(sqlContext: SQLContext,
       carbonLoadModel: CarbonLoadModel,
       storeLocation: String,
@@ -288,15 +139,17 @@ object CarbonDataRDDFactory {
 
     val compactionThread = new Thread {
       override def run(): Unit = {
-
         try {
           // compaction status of the table which is triggered by the user.
           var triggeredCompactionStatus = false
           var exception: Exception = null
           try {
-            DataManagementFunc.executeCompaction(carbonLoadModel: CarbonLoadModel,
-              compactionModel: CompactionModel,
-              executor, sqlContext, storeLocation
+            DataManagementFunc.executeCompaction(
+              carbonLoadModel,
+              compactionModel,
+              executor,
+              sqlContext,
+              storeLocation
             )
             triggeredCompactionStatus = true
           } catch {
@@ -305,18 +158,18 @@ object CarbonDataRDDFactory {
               exception = e
           }
           // continue in case of exception also, check for all the tables.
-          val isConcurrentCompactionAllowed = CarbonProperties.getInstance()
-              .getProperty(CarbonCommonConstants.ENABLE_CONCURRENT_COMPACTION,
-                CarbonCommonConstants.DEFAULT_ENABLE_CONCURRENT_COMPACTION
-              ).equalsIgnoreCase("true")
+          val isConcurrentCompactionAllowed = CarbonProperties.getInstance().getProperty(
+            CarbonCommonConstants.ENABLE_CONCURRENT_COMPACTION,
+            CarbonCommonConstants.DEFAULT_ENABLE_CONCURRENT_COMPACTION
+          ).equalsIgnoreCase("true")
 
           if (!isConcurrentCompactionAllowed) {
             LOGGER.info("System level compaction lock is enabled.")
             val skipCompactionTables = ListBuffer[CarbonTableIdentifier]()
-            var tableForCompaction = CarbonCompactionUtil
-              .getNextTableToCompact(CarbonEnv.getInstance(sqlContext.sparkSession).carbonMetastore
+            var tableForCompaction = CarbonCompactionUtil.getNextTableToCompact(
+              CarbonEnv.getInstance(sqlContext.sparkSession).carbonMetastore
                 .listAllTables(sqlContext.sparkSession).toArray,
-                skipCompactionTables.toList.asJava)
+              skipCompactionTables.toList.asJava)
             while (null != tableForCompaction) {
               LOGGER.info("Compaction request has been identified for table " +
                   s"${ tableForCompaction.getDatabaseName }." +
@@ -325,11 +178,10 @@ object CarbonDataRDDFactory {
               val metadataPath = table.getMetaDataFilepath
               val compactionType = CarbonCompactionUtil.determineCompactionType(metadataPath)
 
-              val newCarbonLoadModel = new CarbonLoadModel()
-              DataManagementFunc.prepareCarbonLoadModel(table, newCarbonLoadModel)
+              val newCarbonLoadModel = prepareCarbonLoadModel(table)
 
               val compactionSize = CarbonDataMergerUtil
-                  .getCompactionSize(CompactionType.MAJOR_COMPACTION)
+                .getCompactionSize(CompactionType.MAJOR_COMPACTION)
 
               val newcompactionModel = CompactionModel(compactionSize,
                 compactionType,
@@ -386,137 +238,25 @@ object CarbonDataRDDFactory {
     compactionThread.run()
   }
 
-  case class SplitThread(sqlContext: SQLContext,
-      carbonLoadModel: CarbonLoadModel,
-      executor: ExecutorService,
-      segmentId: String,
-      partitionId: String,
-      oldPartitionIdList: List[Int]) extends Thread {
-      override def run(): Unit = {
-        var triggeredSplitPartitionStatus = false
-        var exception: Exception = null
-        try {
-          DataManagementFunc.executePartitionSplit(sqlContext,
-            carbonLoadModel, executor, segmentId, partitionId, oldPartitionIdList)
-          triggeredSplitPartitionStatus = true
-        } catch {
-          case e: Exception =>
-            LOGGER.error(s"Exception in partition split thread: ${ e.getMessage } }")
-          exception = e
-        }
-        if (triggeredSplitPartitionStatus == false) {
-          throw new Exception("Exception in split partition " + exception.getMessage)
-        }
-      }
-  }
-
-  case class dropPartitionThread(sqlContext: SQLContext,
-      carbonLoadModel: CarbonLoadModel,
-      executor: ExecutorService,
-      segmentId: String,
-      partitionId: String,
-      dropWithData: Boolean,
-      oldPartitionIds: List[Int]) extends Thread {
-    override def run(): Unit = {
-      try {
-        DataManagementFunc.executeDroppingPartition(sqlContext, carbonLoadModel, executor,
-          segmentId, partitionId, dropWithData, oldPartitionIds)
-      } catch {
-        case e: Exception =>
-          LOGGER.error(s"Exception in dropping partition thread: ${ e.getMessage } }")
-      }
-    }
-  }
-
-  def startSplitThreads(sqlContext: SQLContext,
-      carbonLoadModel: CarbonLoadModel,
-      partitionId: String,
-      oldPartitionIdList: List[Int]): Unit = {
-    val numberOfCores = CarbonProperties.getInstance()
-      .getProperty(CarbonCommonConstants.NUM_CORES_ALT_PARTITION,
-        CarbonCommonConstants.DEFAULT_NUMBER_CORES)
-    val executor : ExecutorService = Executors.newFixedThreadPool(numberOfCores.toInt)
-    try {
-      val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
-      val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
-      val segmentStatusManager = new SegmentStatusManager(absoluteTableIdentifier)
-      val validSegments = segmentStatusManager.getValidAndInvalidSegments.getValidSegments.asScala
-      val threadArray: Array[SplitThread] = new Array[SplitThread](validSegments.size)
-      var i = 0
-      validSegments.foreach { segmentId =>
-        threadArray(i) = SplitThread(sqlContext, carbonLoadModel, executor,
-          segmentId, partitionId, oldPartitionIdList)
-        threadArray(i).start()
-        i += 1
-      }
-      threadArray.foreach {
-        thread => thread.join()
-      }
-      val identifier = AbsoluteTableIdentifier.from(carbonLoadModel.getStorePath,
-        carbonLoadModel.getDatabaseName, carbonLoadModel.getTableName)
-      val refresher = DataMapStoreManager.getInstance().getTableSegmentRefresher(identifier)
-      refresher.refreshSegments(validSegments.asJava)
-    } catch {
-      case e: Exception =>
-        LOGGER.error(s"Exception when split partition: ${ e.getMessage }")
-      throw e
-    } finally {
-      executor.shutdown()
-      try {
-        CarbonLoaderUtil.deletePartialLoadDataIfExist(carbonLoadModel, false)
-      } catch {
-        case e: Exception =>
-          LOGGER.error(s"Exception in add/split partition thread while deleting partial load file" +
-                       s" ${ e.getMessage }")
-      }
-    }
-  }
-
-  def startDropThreads(sqlContext: SQLContext,
-      carbonLoadModel: CarbonLoadModel,
-      partitionId: String,
-      dropWithData: Boolean,
-      oldPartitionIds: List[Int]): Unit = {
-    val numberOfCores = CarbonProperties.getInstance()
-    .getProperty(CarbonCommonConstants.NUM_CORES_ALT_PARTITION,
-      CarbonCommonConstants.DEFAULT_NUMBER_CORES)
-    val executor : ExecutorService = Executors.newFixedThreadPool(numberOfCores.toInt)
-    try {
-      val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
-      val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
-      val segmentStatusManager = new SegmentStatusManager(absoluteTableIdentifier)
-      val validSegments = segmentStatusManager.getValidAndInvalidSegments.getValidSegments.asScala
-      val threadArray: Array[Thread] = new Array[Thread](validSegments.size)
-      var i = 0
-      for (segmentId: String <- validSegments) {
-        threadArray(i) = dropPartitionThread(sqlContext, carbonLoadModel, executor,
-            segmentId, partitionId, dropWithData, oldPartitionIds)
-        threadArray(i).start()
-        i += 1
-      }
-      for (thread <- threadArray) {
-        thread.join()
-      }
-      val identifier = AbsoluteTableIdentifier.from(carbonLoadModel.getStorePath,
-        carbonLoadModel.getDatabaseName, carbonLoadModel.getTableName)
-      val refresher = DataMapStoreManager.getInstance().getTableSegmentRefresher(identifier)
-      refresher.refreshSegments(validSegments.asJava)
-    } catch {
-      case e: Exception =>
-        LOGGER.error(s"Exception when dropping partition: ${ e.getMessage }")
-    } finally {
-      executor.shutdown()
-      try {
-        CarbonLoaderUtil.deletePartialLoadDataIfExist(carbonLoadModel, false)
-      } catch {
-        case e: Exception =>
-          LOGGER.error(s"Exception in dropping partition thread while deleting partial load file" +
-                       s" ${ e.getMessage }")
-      }
-    }
+  private def prepareCarbonLoadModel(
+      table: CarbonTable
+  ): CarbonLoadModel = {
+    val loadModel = new CarbonLoadModel
+    loadModel.setTableName(table.getFactTableName)
+    val dataLoadSchema = new CarbonDataLoadSchema(table)
+    // Need to fill dimension relation
+    loadModel.setCarbonDataLoadSchema(dataLoadSchema)
+    loadModel.setTableName(table.getCarbonTableIdentifier.getTableName)
+    loadModel.setDatabaseName(table.getCarbonTableIdentifier.getDatabaseName)
+    loadModel.setStorePath(table.getStorePath)
+    CommonUtil.readLoadMetadataDetails(loadModel)
+    val loadStartTime = CarbonUpdateUtil.readCurrentTime()
+    loadModel.setFactTimeStamp(loadStartTime)
+    loadModel
   }
 
-  def loadCarbonData(sqlContext: SQLContext,
+  def loadCarbonData(
+      sqlContext: SQLContext,
       carbonLoadModel: CarbonLoadModel,
       storePath: String,
       columnar: Boolean,
@@ -524,425 +264,91 @@ object CarbonDataRDDFactory {
       result: Option[DictionaryServer],
       overwriteTable: Boolean,
       dataFrame: Option[DataFrame] = None,
-      updateModel: Option[UpdateTableModel] = None): Unit = {
+      updateModel: Option[UpdateTableModel] = None
+  ): Unit = {
     val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
     // for handling of the segment Merging.
-    def handleSegmentMerging(): Unit = {
-      LOGGER.info(s"compaction need status is" +
-          s" ${ CarbonDataMergerUtil.checkIfAutoLoadMergingRequired() }")
-      if (CarbonDataMergerUtil.checkIfAutoLoadMergingRequired()) {
-        LOGGER.audit(s"Compaction request received for table " +
-            s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
-        val compactionSize = 0
-        val isCompactionTriggerByDDl = false
-        val compactionModel = CompactionModel(compactionSize,
-          CompactionType.MINOR_COMPACTION,
-          carbonTable,
-          isCompactionTriggerByDDl
-        )
-        var storeLocation = ""
-        val configuredStore = Util.getConfiguredLocalDirs(SparkEnv.get.conf)
-        if (null != configuredStore && configuredStore.nonEmpty) {
-          storeLocation = configuredStore(Random.nextInt(configuredStore.length))
-        }
-        if (storeLocation == null) {
-          storeLocation = System.getProperty("java.io.tmpdir")
-        }
-        storeLocation = storeLocation + "/carbonstore/" + System.nanoTime()
-
-        val isConcurrentCompactionAllowed = CarbonProperties.getInstance()
-            .getProperty(CarbonCommonConstants.ENABLE_CONCURRENT_COMPACTION,
-              CarbonCommonConstants.DEFAULT_ENABLE_CONCURRENT_COMPACTION
-            )
-            .equalsIgnoreCase("true")
-
-        if (!isConcurrentCompactionAllowed) {
-
-          handleCompactionForSystemLocking(sqlContext,
-            carbonLoadModel,
-            storeLocation,
-            CompactionType.MINOR_COMPACTION,
-            carbonTable,
-            compactionModel
-          )
-        } else {
-          val lock = CarbonLockFactory
-              .getCarbonLockObj(carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
-                LockUsage.COMPACTION_LOCK
-              )
 
-          if (lock.lockWithRetries()) {
-            LOGGER.info("Acquired the compaction lock.")
-            try {
-              startCompactionThreads(sqlContext,
-                carbonLoadModel,
-                storeLocation,
-                compactionModel,
-                lock
-              )
-            } catch {
-              case e: Exception =>
-                LOGGER.error(s"Exception in start compaction thread. ${ e.getMessage }")
-                lock.unlock()
-                throw e
-            }
-          } else {
-            LOGGER.audit("Not able to acquire the compaction lock for table " +
-                s"${ carbonLoadModel.getDatabaseName }.${
-                  carbonLoadModel
-                      .getTableName
-                }")
-            LOGGER.error("Not able to acquire the compaction lock for table " +
-                s"${ carbonLoadModel.getDatabaseName }.${
-                  carbonLoadModel
-                      .getTableName
-                }")
-          }
-        }
-      }
+    LOGGER.audit(s"Data load request has been received for table" +
+                 s" ${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
+    // Check if any load need to be deleted before loading new data
+    DataManagementFunc.deleteLoadsAndUpdateMetadata(carbonLoadModel.getDatabaseName,
+      carbonLoadModel.getTableName, storePath, isForceDeletion = false, carbonTable)
+    var status: Array[(String, (LoadMetadataDetails, ExecutionErrors))] = null
+    var res: Array[List[(String, (LoadMetadataDetails, ExecutionErrors))]] = null
+
+    // create new segment folder  in carbon store
+    if (updateModel.isEmpty) {
+      CarbonLoaderUtil.checkAndCreateCarbonDataLocation(storePath,
+        carbonLoadModel.getSegmentId, carbonTable)
     }
-
-    def updateStatus(status: Array[(String, (LoadMetadataDetails, ExecutionErrors))],
-        loadStatus: String) = {
-      val metadataDetails = if (status != null && status(0) != null) {
-        status(0)._2._1
-      } else {
-        new LoadMetadataDetails
-      }
-      CarbonLoaderUtil
-        .populateNewLoadMetaEntry(metadataDetails,
-          loadStatus,
-          carbonLoadModel.getFactTimeStamp,
-          true)
-      val success = CarbonLoaderUtil.recordLoadMetadata(metadataDetails,
-        carbonLoadModel, false, overwriteTable)
-      if (!success) {
-        val errorMessage = "Dataload failed due to failure in table status updation."
-        LOGGER.audit("Data load is failed for " +
-                     s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
-        LOGGER.error("Dataload failed due to failure in table status updation.")
-        throw new Exception(errorMessage)
-      } else if (!carbonLoadModel.isRetentionRequest) {
-        // TODO : Handle it
-        LOGGER.info("********Database updated**********")
-      }
-    }
-
+    var loadStatus = CarbonCommonConstants.STORE_LOADSTATUS_SUCCESS
+    var errorMessage: String = "DataLoad failure"
+    var executorMessage: String = ""
+    val isSortTable = carbonTable.getNumberOfSortColumns > 0
+    val sortScope = CarbonDataProcessorUtil.getSortScope(carbonLoadModel.getSortScope)
     try {
-      LOGGER.audit(s"Data load request has been received for table" +
-          s" ${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
-      // Check if any load need to be deleted before loading new data
-      DataManagementFunc.deleteLoadsAndUpdateMetadata(carbonLoadModel.getDatabaseName,
-        carbonLoadModel.getTableName, storePath, false, carbonTable)
-      var blocksGroupBy: Array[(String, Array[BlockDetails])] = null
-      var status: Array[(String, (LoadMetadataDetails, ExecutionErrors))] = null
-      var res: Array[List[(String, (LoadMetadataDetails, ExecutionErrors))]] = null
-
-      def loadDataFile(): Unit = {
-        /*
-         * when data load handle by node partition
-         * 1)clone the hadoop configuration,and set the file path to the configuration
-         * 2)use org.apache.hadoop.mapreduce.lib.input.TextInputFormat to get splits,size info
-         * 3)use CarbonLoaderUtil.nodeBlockMapping to get mapping info of node and block,
-         *   for locally writing carbondata files(one file one block) in nodes
-         * 4)use NewCarbonDataLoadRDD to load data and write to carbondata files
-         */
-        val hadoopConfiguration = new Configuration(sqlContext.sparkContext.hadoopConfiguration)
-        // FileUtils will skip file which is no csv, and return all file path which split by ','
-        val filePaths = carbonLoadModel.getFactFilePath
-        hadoopConfiguration.set(FileInputFormat.INPUT_DIR, filePaths)
-        hadoopConfiguration.set(FileInputFormat.INPUT_DIR_RECURSIVE, "true")
-        hadoopConfiguration.set("io.compression.codecs",
-          """org.apache.hadoop.io.compress.GzipCodec,
-             org.apache.hadoop.io.compress.DefaultCodec,
-             org.apache.hadoop.io.compress.BZip2Codec""".stripMargin)
-
-        CommonUtil.configSplitMaxSize(sqlContext.sparkContext, filePaths, hadoopConfiguration)
-
-        val inputFormat = new org.apache.hadoop.mapreduce.lib.input.TextInputFormat
-        val jobContext = new Job(hadoopConfiguration)
-        val rawSplits = inputFormat.getSplits(jobContext).toArray
-        val blockList = rawSplits.map { inputSplit =>
-          val fileSplit = inputSplit.asInstanceOf[FileSplit]
-          new TableBlockInfo(fileSplit.getPath.toString,
-            fileSplit.getStart, "1",
-            fileSplit.getLocations, fileSplit.getLength, ColumnarFormatVersion.V1, null
-          ).asInstanceOf[Distributable]
-        }
-        // group blocks to nodes, tasks
-        val startTime = System.currentTimeMillis
-        val activeNodes = DistributionUtil
-            .ensureExecutorsAndGetNodeList(blockList, sqlContext.sparkContext)
-        val nodeBlockMapping =
-          CarbonLoaderUtil
-              .nodeBlockMapping(blockList.toSeq.asJava, -1, activeNodes.toList.asJava).asScala
-              .toSeq
-        val timeElapsed: Long = System.currentTimeMillis - startTime
-        LOGGER.info("Total Time taken in block allocation: " + timeElapsed)
-        LOGGER.info(s"Total no of blocks: ${ blockList.length }, " +
-            s"No.of Nodes: ${nodeBlockMapping.size}")
-        var str = ""
-        nodeBlockMapping.foreach(entry => {
-          val tableBlock = entry._2
-          str = str + "#Node: " + entry._1 + " no.of.blocks: " + tableBlock.size()
-          tableBlock.asScala.foreach(tableBlockInfo =>
-            if (!tableBlockInfo.getLocations.exists(hostentry =>
-              hostentry.equalsIgnoreCase(entry._1)
-            )) {
-              str = str + " , mismatch locations: " + tableBlockInfo.getLocations
-                  .foldLeft("")((a, b) => a + "," + b)
-            }
-          )
-          str = str + "\n"
-        }
-        )
-        LOGGER.info(str)
-        blocksGroupBy = nodeBlockMapping.map { entry =>
-          val blockDetailsList =
-            entry._2.asScala.map { distributable =>
-              val tableBlock = distributable.asInstanceOf[TableBlockInfo]
-              new BlockDetails(new Path(tableBlock.getFilePath),
-                tableBlock.getBlockOffset, tableBlock.getBlockLength, tableBlock.getLocations
-              )
-            }.toArray
-          (entry._1, blockDetailsList)
-        }.toArray
-
-        status = new NewCarbonDataLoadRDD(
-          sqlContext.sparkContext,
-          new DataLoadResultImpl(),
+      if (updateModel.isDefined) {
+        res = loadDataFrameForUpdate(
+          sqlContext,
+          dataFrame,
           carbonLoadModel,
-          blocksGroupBy
-        ).collect()
-      }
-
-      def loadDataFrame(): Unit = {
-        try {
-          val rdd = dataFrame.get.rdd
-
-          val nodeNumOfData = rdd.partitions.flatMap[String, Array[String]]{ p =>
-            DataLoadPartitionCoalescer.getPreferredLocs(rdd, p).map(_.host)
-          }.distinct.size
-          val nodes = DistributionUtil.ensureExecutorsByNumberAndGetNodeList(nodeNumOfData,
-            sqlContext.sparkContext)
-          val newRdd = new DataLoadCoalescedRDD[Row](rdd, nodes.toArray.distinct)
-
-          status = new NewDataFrameLoaderRDD(sqlContext.sparkContext,
-            new DataLoadResultImpl(),
-            carbonLoadModel,
-            newRdd).collect()
-
-        } catch {
-          case ex: Exception =>
-            LOGGER.error(ex, "load data frame failed")
-            throw ex
-        }
-      }
-
-      def loadDataFrameForUpdate(): Unit = {
-        val segmentUpdateParallelism = CarbonProperties.getInstance().getParallelismForSegmentUpdate
-
-        def triggerDataLoadForSegment(key: String, taskNo: Int,
-            iter: Iterator[Row]): Iterator[(String, (LoadMetadataDetails, ExecutionErrors))] = {
-          val rddResult = new updateResultImpl()
-          val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
-          val resultIter = new Iterator[(String, (LoadMetadataDetails, ExecutionErrors))] {
-            var partitionID = "0"
-            val loadMetadataDetails = new LoadMetadataDetails
-            val executionErrors = new ExecutionErrors(FailureCauses.NONE, "")
-            var uniqueLoadStatusId = ""
-            try {
-              val segId = key
-              val index = taskNo
-              uniqueLoadStatusId = carbonLoadModel.getTableName +
-                                   CarbonCommonConstants.UNDERSCORE +
-                                   (index + "_0")
-
-              // convert timestamp
-              val timeStampInLong = updateModel.get.updatedTimeStamp + ""
-              loadMetadataDetails.setPartitionCount(partitionID)
-              loadMetadataDetails.setLoadName(segId)
-              loadMetadataDetails.setLoadStatus(CarbonCommonConstants.STORE_LOADSTATUS_FAILURE)
-              carbonLoadModel.setPartitionId(partitionID)
-              carbonLoadModel.setSegmentId(segId)
-              carbonLoadModel.setTaskNo(String.valueOf(index))
-              carbonLoadModel.setFactTimeStamp(updateModel.get.updatedTimeStamp)
-
-              // During Block Spill case Increment of File Count and proper adjustment of Block
-              // naming is only done when AbstractFactDataWriter.java : initializeWriter get
-              // CarbondataFileName as null. For handling Block Spill not setting the
-              // CarbondataFileName in case of Update.
-              // carbonLoadModel.setCarbondataFileName(newBlockName)
-
-              // storeLocation = CarbonDataLoadRDD.initialize(carbonLoadModel, index)
-              loadMetadataDetails.setLoadStatus(CarbonCommonConstants.STORE_LOADSTATUS_SUCCESS)
-              UpdateDataLoad.DataLoadForUpdate(segId,
-                index,
-                iter,
-                carbonLoadModel,
-                loadMetadataDetails)
-            } catch {
-              case e: NoRetryException =>
-                loadMetadataDetails
-                  .setLoadStatus(CarbonCommonConstants.STORE_LOADSTATUS_PARTIAL_SUCCESS)
-                executionErrors.failureCauses = FailureCauses.BAD_RECORDS
-                executionErrors.errorMsg = e.getMessage
-                LOGGER.info("Bad Record Found")
-              case e: Exception =>
-                LOGGER.info("DataLoad failure")
-                LOGGER.error(e)
-                throw e
-            }
-
-            var finished = false
-
-            override def hasNext: Boolean = !finished
-
-            override def next(): (String, (LoadMetadataDetails, ExecutionErrors)) = {
-              finished = true
-              rddResult
-                .getKey(uniqueLoadStatusId,
-                  (loadMetadataDetails, executionErrors))
+          updateModel,
+          carbonTable)
+        res.foreach { resultOfSeg =>
+          resultOfSeg.foreach { resultOfBlock =>
+            if (resultOfBlock._2._1.getLoadStatus.equalsIgnoreCase(
+              CarbonCommonConstants.STORE_LOADSTATUS_FAILURE)) {
+              loadStatus = CarbonCommonConstants.STORE_LOADSTATUS_FAILURE
+              if (resultOfBlock._2._2.failureCauses == FailureCauses.NONE) {
+                updateModel.get.executorErrors.failureCauses = FailureCauses.EXECUTOR_FAILURE
+                updateModel.get.executorErrors.errorMsg = "Failure in the Executor."
+              } else {
+                updateModel.get.executorErrors = resultOfBlock._2._2
+              }
+            } else if (resultOfBlock._2._1.getLoadStatus.equalsIgnoreCase(
+              CarbonCommonConstants.STORE_LOADSTATUS_PARTIAL_SUCCESS)) {
+              loadStatus = CarbonCommonConstants.STORE_LOADSTATUS_PARTIAL_SUCCESS
+              updateModel.get.executorErrors.failureCauses = resultOfBlock._2._2.failureCauses
+              updateModel.get.executorErrors.errorMsg = resultOfBlock._2._2.errorMsg
             }
           }
-          resultIter
-        }
-
-        val updateRdd = dataFrame.get.rdd
-
-        // return directly if no rows to update
-        val noRowsToUpdate = updateRdd.isEmpty()
-        if (noRowsToUpdate) {
-          res = Array[List[(String, (LoadMetadataDetails, ExecutionErrors))]]()
-          return
-        }
-
-        // splitting as (key, value) i.e., (segment, updatedRows)
-        val keyRDD = updateRdd.map(row =>
-            (row.get(row.size - 1).toString, Row(row.toSeq.slice(0, row.size - 1): _*)))
-
-        val loadMetadataDetails = SegmentStatusManager.readLoadMetadata(
-          carbonTable.getMetaDataFilepath)
-        val segmentIds = loadMetadataDetails.map(_.getLoadName)
-        val segmentIdIndex = segmentIds.zipWithIndex.toMap
-        val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonLoadModel.getStorePath,
-          carbonTable.getCarbonTableIdentifier)
-        val segmentId2maxTaskNo = segmentIds.map { segId =>
-          (segId, CarbonUpdateUtil.getLatestTaskIdForSegment(segId, carbonTablePath))
-        }.toMap
-
-        class SegmentPartitioner(segIdIndex: Map[String, Int], parallelism: Int)
-          extends org.apache.spark.Partitioner {
-          override def numPartitions: Int = segmentIdIndex.size * parallelism
-
-          override def getPartition(key: Any): Int = {
-            val segId = key.asInstanceOf[String]
-            // partitionId
-            segmentIdIndex(segId) * parallelism + Random.nextInt(parallelism)
-          }
-        }
-
-        val partitionByRdd = keyRDD.partitionBy(new SegmentPartitioner(segmentIdIndex,
-          segmentUpdateParallelism))
-
-        // because partitionId=segmentIdIndex*parallelism+RandomPart and RandomPart<parallelism,
-        // so segmentIdIndex=partitionId/parallelism, this has been verified.
-        res = partitionByRdd.map(_._2).mapPartitions { partition =>
-          val partitionId = TaskContext.getPartitionId()
-          val segIdIndex = partitionId / segmentUpdateParallelism
-          val randomPart = partitionId - segIdIndex * segmentUpdateParallelism
-          val segId = segmentIds(segIdIndex)
-          val newTaskNo = segmentId2maxTaskNo(segId) + randomPart + 1
-
-          List(triggerDataLoadForSegment(segId, newTaskNo, partition).toList).toIterator
-        }.collect()
-      }
-
-      def loadDataForPartitionTable(): Unit = {
-        try {
-          val rdd = repartitionInputData(sqlContext, dataFrame, carbonLoadModel)
-          status = new PartitionTableDataLoaderRDD(sqlContext.sparkContext,
-            new DataLoadResultImpl(),
-            carbonLoadModel,
-            rdd).collect()
-        } catch {
-          case ex: Exception =>
-            LOGGER.error(ex, "load data failed for partition table")
-            throw ex
         }
-      }
-      // create new segment folder  in carbon store
-      if (updateModel.isEmpty) {
-        CarbonLoaderUtil.checkAndCreateCarbonDataLocation(storePath,
-          carbonLoadModel.getSegmentId, carbonTable)
-      }
-      var loadStatus = CarbonCommonConstants.STORE_LOADSTATUS_SUCCESS
-      var errorMessage: String = "DataLoad failure"
-      var executorMessage: String = ""
-      val isSortTable = carbonTable.getNumberOfSortColumns > 0
-      val sortScope = CarbonDataProcessorUtil.getSortScope(carbonLoadModel.getSortScope)
-      try {
-        if (updateModel.isDefined) {
-          loadDataFrameForUpdate()
-        } else if (carbonTable.getPartitionInfo(carbonTable.getFactTableName) != null) {
-          loadDataForPartitionTable()
+      } else {
+        status = if (carbonTable.getPartitionInfo(carbonTable.getFactTableName) != null) {
+          loadDataForPartitionTable(sqlContext, dataFrame, carbonLoadModel)
         } else if (isSortTable && sortScope.equals(SortScopeOptions.SortScope.GLOBAL_SORT)) {
-          LOGGER.audit("Using global sort for loading.")
-          status = DataLoadProcessBuilderOnSpark.loadDataUsingGlobalSort(sqlContext.sparkContext,
+          DataLoadProcessBuilderOnSpark.loadDataUsingGlobalSort(sqlContext.sparkContext,
             dataFrame, carbonLoadModel)
         } else if (dataFrame.isDefined) {
-          loadDataFrame()
+          loadDataFrame(sqlContext, dataFrame, carbonLoadModel)
         } else {
-          loadDataFile()
+          loadDataFile(sqlContext, carbonLoadModel)
         }
-        if (updateModel.isDefined) {
-
-          res.foreach(resultOfSeg => resultOfSeg.foreach(
-            resultOfBlock => {
-              if (resultOfBlock._2._1.getLoadStatus
-                .equalsIgnoreCase(CarbonCommonConstants.STORE_LOADSTATUS_FAILURE)) {
-                loadStatus = CarbonCommonConstants.STORE_LOADSTATUS_FAILURE
-                if (resultOfBlock._2._2.failureCauses == FailureCauses.NONE) {
-                  updateModel.get.executorErrors.failureCauses = FailureCauses.EXECUTOR_FAILURE
-                  updateModel.get.executorErrors.errorMsg = "Failure in the Executor."
-                }
-                else {
-                  updateModel.get.executorErrors = resultOfBlock._2._2
-                }
-              } else if (resultOfBlock._2._1.getLoadStatus
-                .equalsIgnoreCase(CarbonCommonConstants.STORE_LOADSTATUS_PARTIAL_SUCCESS)) {
-                loadStatus = CarbonCommonConstants.STORE_LOADSTATUS_PARTIAL_SUCCESS
-                updateModel.get.executorErrors.failureCauses = resultOfBlock._2._2.failureCauses
-                updateModel.get.executorErrors.errorMsg = resultOfBlock._2._2.errorMsg
-              }
-            }
-          ))
-
-        } else {
-          CommonUtil.mergeIndexFiles(sqlContext.sparkContext,
-            Seq(carbonLoadModel.getSegmentId), storePath, carbonTable)
-          val newStatusMap = scala.collection.mutable.Map.empty[String, String]
-          if (status.nonEmpty) {
-            status.foreach { eachLoadStatus =>
-              val state = newStatusMap.get(eachLoadStatus._1)
-              state match {
-                case Some(CarbonCommonConstants.STORE_LOADSTATUS_FAILURE) =>
-                  newStatusMap.put(eachLoadStatus._1, eachLoadStatus._2._1.getLoadStatus)
-                case Some(CarbonCommonConstants.STORE_LOADSTATUS_PARTIAL_SUCCESS)
-                  if eachLoadStatus._2._1.getLoadStatus ==
-                     CarbonCommonConstants.STORE_LOADSTATUS_SUCCESS =>
-                  newStatusMap.put(eachLoadStatus._1, eachLoadStatus._2._1.getLoadStatus)
-                case _ =>
-                  newStatusMap.put(eachLoadStatus._1, eachLoadStatus._2._1.getLoadStatus)
-              }
+        CommonUtil.mergeIndexFiles(sqlContext.sparkContext,
+          Seq(carbonLoadModel.getSegmentId), storePath, carbonTable)
+        val newStatusMap = scala.collection.mutable.Map.empty[String, String]
+        if (status.nonEmpty) {
+          status.foreach { eachLoadStatus =>
+            val state = newStatusMap.get(eachLoadStatus._1)
+            state match {
+              case Some(CarbonCommonConstants.STORE_LOADSTATUS_FAILURE) =>
+                newStatusMap.put(eachLoadStatus._1, eachLoadStatus._2._1.getLoadStatus)
+              case Some(CarbonCommonConstants.STORE_LOADSTATUS_PARTIAL_SUCCESS)
+                if eachLoadStatus._2._1.getLoadStatus ==
+                   CarbonCommonConstants.STORE_LOADSTATUS_SUCCESS =>
+                newStatusMap.put(eachLoadStatus._1, eachLoadStatus._2._1.getLoadStatus)
+              case _ =>
+                newStatusMap.put(eachLoadStatus._1, eachLoadStatus._2._1.getLoadStatus)
             }
+          }
 
           newStatusMap.foreach {
             case (key, value) =>
               if (value == CarbonCommonConstants.STORE_LOADSTATUS_FAILURE) {
                 loadStatus = CarbonCommonConstants.STORE_LOADSTATUS_FAILURE
               } else if (value == CarbonCommonConstants.STORE_LOADSTATUS_PARTIAL_SUCCESS &&
-                  !loadStatus.equals(CarbonCommonConstants.STORE_LOADSTATUS_FAILURE)) {
+                         !loadStatus.equals(CarbonCommonConstants.STORE_LOADSTATUS_FAILURE)) {
                 loadStatus = CarbonCommonConstants.STORE_LOADSTATUS_PARTIAL_SUCCESS
               }
           }
@@ -955,163 +361,402 @@ object CarbonDataRDDFactory {
           loadStatus = partitionStatus
         }
       }
-      } catch {
-        case ex: Throwable =>
-          loadStatus = CarbonCommonConstants.STORE_LOADSTATUS_FAILURE
-          ex match {
-            case sparkException: SparkException =>
-              if (sparkException.getCause.isInstanceOf[DataLoadingException] ||
-                  sparkException.getCause.isInstanceOf[CarbonDataLoadingException]) {
-                executorMessage = sparkException.getCause.getMessage
-                errorMessage = errorMessage + ": " + executorMessage
-              }
-            case _ =>
-              if (ex.getCause != null) {
-                executorMessage = ex.getCause.getMessage
-                errorMessage = errorMessage + ": " + executorMessage
-              }
-          }
-          LOGGER.info(errorMessage)
-          LOGGER.error(ex)
-      }
-      // handle the status file updation for the update cmd.
-      if (updateModel.isDefined) {
-
-        if (loadStatus == CarbonCommonConstants.STORE_LOADSTATUS_FAILURE) {
-          // updateModel.get.executorErrors.errorMsg = errorMessage
-          if (updateModel.get.executorErrors.failureCauses == FailureCauses.NONE) {
-            updateModel.get.executorErrors.failureCauses = FailureCauses.EXECUTOR_FAILURE
-            if (null != executorMessage && !executorMessage.isEmpty) {
-              updateModel.get.executorErrors.errorMsg = executorMessage
-            } else {
-              updateModel.get.executorErrors.errorMsg = "Update failed as the data load has failed."
+    } catch {
+      case ex: Throwable =>
+        loadStatus = CarbonCommonConstants.STORE_LOADSTATUS_FAILURE
+        ex match {
+          case sparkException: SparkException =>
+            if (sparkException.getCause.isInstanceOf[DataLoadingException] ||
+                sparkException.getCause.isInstanceOf[CarbonDataLoadingException]) {
+              executorMessage = sparkException.getCause.getMessage
+              errorMessage = errorMessage + ": " + executorMessage
             }
-          }
-          return
-        } else if (loadStatus == CarbonCommonConstants.STORE_LOADSTATUS_PARTIAL_SUCCESS &&
-                   updateModel.get.executorErrors.failureCauses == FailureCauses.BAD_RECORDS &&
-                   carbonLoadModel.getBadRecordsAction.split(",")(1) == LoggerAction.FAIL.name) {
-          return
-        } else {
-          // in success case handle updation of the table status file.
-          // success case.
-          val segmentDetails = new util.HashSet[String]()
-
-          var resultSize = 0
-
-          res.foreach(resultOfSeg => {
-            resultSize = resultSize + resultOfSeg.size
-            resultOfSeg.foreach(
-            resultOfBlock => {
-              segmentDetails.add(resultOfBlock._2._1.getLoadName)
+          case _ =>
+            if (ex.getCause != null) {
+              executorMessage = ex.getCause.getMessage
+              errorMessage = errorMessage + ": " + executorMessage
             }
-          )}
-          )
-
-          // this means that the update doesnt have any records to update so no need to do table
-          // status file updation.
-          if (resultSize == 0) {
-            LOGGER.audit("Data update is successful with 0 rows updation for " +
-                         s"${carbonLoadModel.getDatabaseName}.${carbonLoadModel.getTableName}")
-            return
-          }
-
-          if (
-            CarbonUpdateUtil
-              .updateTableMetadataStatus(segmentDetails,
-                carbonTable,
-                updateModel.get.updatedTimeStamp + "",
-                true,
-                new util.ArrayList[String](0))) {
-            LOGGER.audit("Data update is successful for " +
-                         s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
+        }
+        LOGGER.info(errorMessage)
+        LOGGER.error(ex)
+    }
+    // handle the status file updation for the update cmd.
+    if (updateModel.isDefined) {
+      if (loadStatus == CarbonCommonConstants.STORE_LOADSTATUS_FAILURE) {
+        if (updateModel.get.executorErrors.failureCauses == FailureCauses.NONE) {
+          updateModel.get.executorErrors.failureCauses = FailureCauses.EXECUTOR_FAILURE
+          if (null != executorMessage && !executorMessage.isEmpty) {
+            updateModel.get.executorErrors.errorMsg = executorMessage
+          } else {
+            updateModel.get.executorErrors.errorMsg = "Update failed as the data load has failed."
           }
-          else {
-            val errorMessage = "Data update failed due to failure in table status updation."
-            LOGGER.audit("Data update is failed for " +
-                         s"${carbonLoadModel.getDatabaseName}.${carbonLoadModel.getTableName}")
-            LOGGER.error("Data update failed due to failure in table status updation.")
-            updateModel.get.executorErrors.errorMsg = errorMessage
-            updateModel.get.executorErrors.failureCauses = FailureCauses
-              .STATUS_FILE_UPDATION_FAILURE
-            return
+        }
+        return
+      } else if (loadStatus == CarbonCommonConstants.STORE_LOADSTATUS_PARTIAL_SUCCESS &&
+                 updateModel.get.executorErrors.failureCauses == FailureCauses.BAD_RECORDS &&
+                 carbonLoadModel.getBadRecordsAction.split(",")(1) == LoggerAction.FAIL.name) {
+        return
+      } else {
+        // in success case handle updation of the table status file.
+        // success case.
+        val segmentDetails = new util.HashSet[String]()
+        var resultSize = 0
+        res.foreach { resultOfSeg =>
+          resultSize = resultSize + resultOfSeg.size
+          resultOfSeg.foreach { resultOfBlock =>
+            segmentDetails.add(resultOfBlock._2._1.getLoadName)
           }
-
         }
 
-        return
+        // this means that the update doesnt have any records to update so no need to do table
+        // status file updation.
+        if (resultSize == 0) {
+          LOGGER.audit("Data update is successful with 0 rows updation for " +
+                       s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
+          return
+        }
+        if (CarbonUpdateUtil.updateTableMetadataStatus(
+          segmentDetails,
+          carbonTable,
+          updateModel.get.updatedTimeStamp + "",
+          true,
+          new util.ArrayList[String](0))) {
+          LOGGER.audit("Data update is successful for " +
+                       s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
+        } else {
+          val errorMessage = "Data update failed due to failure in table status updation."
+          LOGGER.audit("Data update is failed for " +
+                       s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
+          LOGGER.error("Data update failed due to failure in table status updation.")
+          updateModel.get.executorErrors.errorMsg = errorMessage
+          updateModel.get.executorErrors.failureCauses = FailureCauses
+            .STATUS_FILE_UPDATION_FAILURE
+          return
+        }
       }
-      if (loadStatus == CarbonCommonConstants.STORE_LOADSTATUS_FAILURE) {
+      return
+    }
+    if (loadStatus == CarbonCommonConstants.STORE_LOADSTATUS_FAILURE) {
+      // update the load entry in table status file for changing the status to failure
+      CommonUtil.updateTableStatusForFailure(carbonLoadModel)
+      LOGGER.info("********starting clean up**********")
+      CarbonLoaderUtil.deleteSegment(carbonLoadModel, carbonLoadModel.getSegmentId.toInt)
+      LOGGER.info("********clean up done**********")
+      LOGGER.audit(s"Data load is failed for " +
+                   s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
+      LOGGER.warn("Cannot write load metadata file as data load failed")
+      throw new Exception(errorMessage)
+    } else {
+      // check if data load fails due to bad record and throw data load failure due to
+      // bad record exception
+      if (loadStatus == CarbonCommonConstants.STORE_LOADSTATUS_PARTIAL_SUCCESS &&
+          status(0)._2._2.failureCauses == FailureCauses.BAD_RECORDS &&
+          carbonLoadModel.getBadRecordsAction.split(",")(1) == LoggerAction.FAIL.name) {
         // update the load entry in table status file for changing the status to failure
         CommonUtil.updateTableStatusForFailure(carbonLoadModel)
         LOGGER.info("********starting clean up**********")
         CarbonLoaderUtil.deleteSegment(carbonLoadModel, carbonLoadModel.getSegmentId.toInt)
         LOGGER.info("********clean up done**********")
         LOGGER.audit(s"Data load is failed for " +
-            s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
+                     s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
+        throw new Exception(status(0)._2._2.errorMsg)
+      }
+      // if segment is empty then fail the data load
+      if (!CarbonLoaderUtil.isValidSegment(carbonLoadModel, carbonLoadModel.getSegmentId.toInt)) {
+        // update the load entry in table status file for changing the status to failure
+        CommonUtil.updateTableStatusForFailure(carbonLoadModel)
+        LOGGER.info("********starting clean up**********")
+        CarbonLoaderUtil.deleteSegment(carbonLoadModel, carbonLoadModel.getSegmentId.toInt)
+        LOGGER.info("********clean up done**********")
+        LOGGER.audit(s"Data load is failed for " +
+                     s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }" +
+                     " as there is no data to load")
         LOGGER.warn("Cannot write load metadata file as data load failed")
-        throw new Exception(errorMessage)
+        throw new Exception("No Data to load")
+      }
+      writeDictionary(carbonLoadModel, result, writeAll = false)
+      updateTableStatus(status, carbonLoadModel, loadStatus, overwriteTable)
+
+      if (CarbonCommonConstants.STORE_LOADSTATUS_PARTIAL_SUCCESS.equals(loadStatus)) {
+        LOGGER.audit("Data load is partially successful for " +
+                     s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
       } else {
-        // check if data load fails due to bad record and throw data load failure due to
-        // bad record exception
-        if (loadStatus == CarbonCommonConstants.STORE_LOADSTATUS_PARTIAL_SUCCESS &&
-            status(0)._2._2.failureCauses == FailureCauses.BAD_RECORDS &&
-            carbonLoadModel.getBadRecordsAction.split(",")(1) == LoggerAction.FAIL.name) {
-          // update the load entry in table status file for changing the status to failure
-          CommonUtil.updateTableStatusForFailure(carbonLoadModel)
-          LOGGER.info("********starting clean up**********")
-          CarbonLoaderUtil.deleteSegment(carbonLoadModel, carbonLoadModel.getSegmentId.toInt)
-          LOGGER.info("********clean up done**********")
-          LOGGER.audit(s"Data load is failed for " +
-                       s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
-          throw new Exception(status(0)._2._2.errorMsg)
-        }
-        // if segment is empty then fail the data load
-        if (!CarbonLoaderUtil.isValidSegment(carbonLoadModel, carbonLoadModel.getSegmentId.toInt)) {
-          // update the load entry in table status file for changing the status to failure
-          CommonUtil.updateTableStatusForFailure(carbonLoadModel)
-          LOGGER.info("********starting clean up**********")
-          CarbonLoaderUtil.deleteSegment(carbonLoadModel, carbonLoadModel.getSegmentId.toInt)
-          LOGGER.info("********clean up done**********")
-          LOGGER.audit(s"Data load is failed for " +
-                       s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }" +
-                       " as there is no data to load")
-          LOGGER.warn("Cannot write load metadata file as data load failed")
-          throw new Exception("No Data to load")
-        }
-        writeDictionary(carbonLoadModel, result, false)
-        updateStatus(status, loadStatus)
+        LOGGER.audit("Data load is successful for " +
+                     s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
+      }
+      try {
+        // compaction handling
+        handleSegmentMerging(sqlContext, carbonLoadModel, carbonTable)
+      } catch {
+        case e: Exception =>
+          throw new Exception(
+            "Dataload is success. Auto-Compaction has failed. Please check logs.")
+      }
+    }
+  }
 
-        if (CarbonCommonConstants.STORE_LOADSTATUS_PARTIAL_SUCCESS.equals(loadStatus)) {
-          LOGGER.audit("Data load is partially successful for " +
-                       s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
-        } else {
-          LOGGER.audit("Data load is successful for " +
-                       s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
+  /**
+   * If data load is triggered by UPDATE query, this func will execute the update
+   * TODO: move it to a separate update command
+   */
+  private def loadDataFrameForUpdate(
+      sqlContext: SQLContext,
+      dataFrame: Option[DataFrame],
+      carbonLoadModel: CarbonLoadModel,
+      updateModel: Option[UpdateTableModel],
+      carbonTable: CarbonTable
+  ): Array[List[(String, (LoadMetadataDetails, ExecutionErrors))]] = {
+    val segmentUpdateParallelism = CarbonProperties.getInstance().getParallelismForSegmentUpdate
+
+    val updateRdd = dataFrame.get.rdd
+
+    // return directly if no rows to update
+    val noRowsToUpdate = updateRdd.isEmpty()
+    if (noRowsToUpdate) {
+      Array[List[(String, (LoadMetadataDetails, ExecutionErrors))]]()
+    } else {
+      // splitting as (key, value) i.e., (segment, updatedRows)
+      val keyRDD = updateRdd.map(row =>
+        (row.get(row.size - 1).toString, Row(row.toSeq.slice(0, row.size - 1): _*)))
+
+      val loadMetadataDetails = SegmentStatusManager.readLoadMetadata(
+        carbonTable.getMetaDataFilepath)
+      val segmentIds = loadMetadataDetails.map(_.getLoadName)
+      val segmentIdIndex = segmentIds.zipWithIndex.toMap
+      val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonLoadModel.getStorePath,
+        carbonTable.getCarbonTableIdentifier)
+      val segmentId2maxTaskNo = segmentIds.map { segId =>
+        (segId, CarbonUpdateUtil.getLatestTaskIdForSegment(segId, carbonTablePath))
+      }.toMap
+
+      class SegmentPartitioner(segIdIndex: Map[String, Int], parallelism: Int)
+        extends org.apache.spark.Partitioner {
+        override def numPartitions: Int = segmentIdIndex.size * parallelism
+
+        override def getPartition(key: Any): Int = {
+          val segId = key.asInstanceOf[String]
+          // partitionId
+          segmentIdIndex(segId) * parallelism + Random.nextInt(parallelism)
         }
+      }
+
+      val partitionByRdd = keyRDD.partitionBy(
+        new SegmentPartitioner(segmentIdIndex, segmentUpdateParallelism))
+
+      // because partitionId=segmentIdIndex*parallelism+RandomPart and RandomPart<parallelism,
+      // so segmentIdIndex=partitionId/parallelism, this has been verified.
+      partitionByRdd.map(_._2).mapPartitions { partition =>
+        val partitionId = TaskContext.getPartitionId()
+        val segIdIndex = partitionId / segmentUpdateParallelism
+        val randomPart = partitionId - segIdIndex * segmentUpdateParallelism
+        val segId = segmentIds(segIdIndex)
+        val newTaskNo = segmentId2maxTaskNo(segId) + randomPart + 1
+        List(triggerDataLoadForSegment(carbonLoadModel, updateModel, segId, newTaskNo, partition)
+          .toList).toIterator
+      }.collect()
+    }
+  }
+
+  /**
+   * TODO: move it to a separate update command
+   */
+  private def triggerDataLoadForSegment(
+      carbonLoadModel: CarbonLoadModel,
+      updateModel: Option[UpdateTableModel],
+      key: String,
+      taskNo: Int,
+      iter: Iterator[Row]
+  ): Iterator[(String, (LoadMetadataDetails, ExecutionErrors))] = {
+    val rddResult = new updateResultImpl()
+    val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
+    val resultIter = new Iterator[(String, (LoadMetadataDetails, ExecutionErrors))] {
+      var partitionID = "0"
+      val loadMetadataDetails = new LoadMetadataDetails
+      val executionErrors = ExecutionErrors(FailureCauses.NONE, "")
+      var uniqueLoadStatusId = ""
+      try {
+        val segId = key
+        val index = taskNo
+        uniqueLoadStatusId = carbonLoadModel.getTableName +
+                             CarbonCommonConstants.UNDERSCORE +
+                             (index + "_0")
+
+        loadMetadataDetails.setPartitionCount(partitionID)
+        loadMetadataDetails.setLoadName(segId)
+        loadMetadataDetails.setLoadStatus(CarbonCommonConstants.STORE_LOADSTATUS_FAILURE)
+        carbonLoadModel.setPartitionId(partitionID)
+        carbonLoadModel.setSegmentId(segId)
+        carbonLoadModel.setTaskNo(String.valueOf(index))
+        carbonLoadModel.setFactTimeStamp(updateModel.get.updatedTimeStamp)
+
+        loadMetadataDetails.setLoadStatus(CarbonCommonConstants.STORE_LOADSTATUS_SUCCESS)
+        UpdateDataLoad.DataLoadForUpdate(segId,
+          index,
+          iter,
+          carbonLoadModel,
+          loadMetadataDetails)
+      } catch {
+        case e: NoRetryException =>
+          loadMetadataDetails
+            .setLoadStatus(CarbonCommonConstants.STORE_LOADSTATUS_PARTIAL_SUCCESS)
+          executionErrors.failureCauses = FailureCauses.BAD_RECORDS
+          executionErrors.errorMsg = e.getMessage
+          LOGGER.info("Bad Record Found")
+        case e: Exception =>
+          LOGGER.info("DataLoad failure")
+          LOGGER.error(e)
+          throw e
+      }
+
+      var finished = false
+
+      override def hasNext: Boolean = !finished
+
+      override def next(): (String, (LoadMetadataDetails, ExecutionErrors)) = {
+        finished = true
+        rddResult
+          .getKey(uniqueLoadStatusId,
+            (loadMetadataDetails, executionErrors))
+      }
+    }
+    resultIter
+  }
+
+  /**
+   * Trigger to write dictionary files
+   */
+  private def writeDictionary(carbonLoadModel: CarbonLoadModel,
+      result: Option[DictionaryServer], writeAll: Boolean): Unit = {
+    // write dictionary file
+    val uniqueTableName: String = s"${ carbonLoadModel.getDatabaseName }_${
+      carbonLoadModel.getTableName
+    }"
+    result match {
+      case Some(server) =>
         try {
-          // compaction handling
-          handleSegmentMerging()
+          server.writeTableDictionary(carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
+            .getCarbonTableIdentifier.getTableId)
         } catch {
-          case e: Exception =>
-            throw new Exception(
-              "Dataload is success. Auto-Compaction has failed. Please check logs.")
+          case _: Exception =>
+            LOGGER.error(s"Error while writing dictionary file for $uniqueTableName")
+            throw new Exception("Dataload failed due to error while writing dictionary file!")
+        }
+      case _ =>
+    }
+  }
+
+  /**
+   * Trigger compaction after data load
+   */
+  private def handleSegmentMerging(
+      sqlContext: SQLContext,
+      carbonLoadModel: CarbonLoadModel,
+      carbonTable: CarbonTable
+  ): Unit = {
+    LOGGER.info(s"compaction need status is" +
+                s" ${ CarbonDataMergerUtil.checkIfAutoLoadMergingRequired() }")
+    if (CarbonDataMergerUtil.checkIfAutoLoadMergingRequired()) {
+      LOGGER.audit(s"Compaction request received for table " +
+                   s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
+      val compactionSize = 0
+      val isCompactionTriggerByDDl = false
+      val compactionModel = CompactionModel(compactionSize,
+        CompactionType.MINOR_COMPACTION,
+        carbonTable,
+        isCompactionTriggerByDDl
+      )
+      var storeLocation = ""
+      val configuredStore = Util.getConfiguredLocalDirs(SparkEnv.get.conf)
+      if (null != configuredStore && configuredStore.nonEmpty) {
+        storeLocation = configuredStore(Random.nextInt(configuredStore.length))
+      }
+      if (storeLocation == null) {
+        storeLocation = System.getProperty("java.io.tmpdir")
+      }
+      storeLocation = storeLocation + "/carbonstore/" + System.nanoTime()
+
+      val isConcurrentCompactionAllowed = CarbonProperties.getInstance().getProperty(
+        CarbonCommonConstants.ENABLE_CONCURRENT_COMPACTION,
+        CarbonCommonConstants.DEFAULT_ENABLE_CONCURRENT_COMPACTION
+      ).equalsIgnoreCase("true")
+
+      if (!isConcurrentCompactionAllowed) {
+        handleCompactionForSystemLocking(sqlContext,
+          carbonLoadModel,
+          storeLocation,
+          CompactionType.MINOR_COMPACTION,
+          carbonTable,
+          compactionModel
+        )
+      } else {
+        val lock = CarbonLockFactory.getCarbonLockObj(
+          carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
+          LockUsage.COMPACTION_LOCK)
+
+        if (lock.lockWithRetries()) {
+          LOGGER.info("Acquired the compaction lock.")
+          try {
+            startCompactionThreads(sqlContext,
+              carbonLoadModel,
+              storeLocation,
+              compactionModel,
+              lock
+            )
+          } catch {
+            case e: Exception =>
+              LOGGER.error(s"Exception in start compaction thread. ${ e.getMessage }")
+              lock.unlock()
+              throw e
+          }
+        } else {
+          LOGGER.audit("Not able to acquire the compaction lock for table " +
+                       s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName}")
+          LOGGER.error("Not able to acquire the compaction lock for table " +
+                       s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName}")
         }
       }
     }
+  }
 
+  /**
+   * Update table status file after data loading
+   */
+  private def updateTableStatus(
+      status: Array[(String, (LoadMetadataDetails, ExecutionErrors))],
+      carbonLoadModel: CarbonLoadModel,
+      loadStatus: String,
+      overwriteTable: Boolean
+  ): Unit = {
+    val metadataDetails = if (status != null && status(0) != null) {
+      status(0)._2._1
+    } else {
+      new LoadMetadataDetails
+    }
+    CarbonLoaderUtil.populateNewLoadMetaEntry(
+      metadataDetails,
+      loadStatus,
+      carbonLoadModel.getFactTimeStamp,
+      true)
+    val success = CarbonLoaderUtil.recordLoadMetadata(metadataDetails, carbonLoadModel, false,
+      overwriteTable)
+    if (!success) {
+      val errorMessage = "Dataload failed due to failure in table status updation."
+      LOGGER.audit("Data load is failed for " +
+                   s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
+      LOGGER.error("Dataload failed due to failure in table status updation.")
+      throw new Exception(errorMessage)
+    } else if (!carbonLoadModel.isRetentionRequest) {
+      // TODO : Handle it
+      LOGGER.info("********Database updated**********")
+    }
   }
 
 
   /**
    * repartition the input data for partition table.
-   *
-   * @param sqlContext
-   * @param dataFrame
-   * @param carbonLoadModel
-   * @return
    */
-  private def repartitionInputData(sqlContext: SQLContext,
+  private def repartitionInputData(
+      sqlContext: SQLContext,
       dataFrame: Option[DataFrame],
       carbonLoadModel: CarbonLoadModel): RDD[Row] = {
     val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
@@ -1132,7 +777,7 @@ object CarbonDataRDDFactory {
       throw new DataLoadingException("Partition column not found.")
     }
 
-    val dateFormatMap = CarbonDataProcessorUtil.getDateFormatMap(carbonLoadModel.getDateFormat())
+    val dateFormatMap = CarbonDataProcessorUtil.getDateFormatMap(carbonLoadModel.getDateFormat)
     val specificFormat = Option(dateFormatMap.get(partitionColumn.toLowerCase))
     val timeStampFormat = if (specificFormat.isDefined) {
       new SimpleDateFormat(specificFormat.get)
@@ -1159,7 +804,7 @@ object CarbonDataRDDFactory {
         carbonLoadModel.getSerializationNullFormat.split(CarbonCommonConstants.COMMA, 2)(1)
       dataFrame.get.rdd.map { row =>
         if (null != row && row.length > partitionColumnIndex &&
-          null != row.get(partitionColumnIndex)) {
+            null != row.get(partitionColumnIndex)) {
           (CarbonScalaUtil.getString(row.get(partitionColumnIndex), serializationNullFormat,
             delimiterLevel1, delimiterLevel2, timeStampFormat, dateFormat), row)
         } else {
@@ -1214,23 +859,142 @@ object CarbonDataRDDFactory {
     }
   }
 
-  private def writeDictionary(carbonLoadModel: CarbonLoadModel,
-      result: Option[DictionaryServer], writeAll: Boolean) = {
-    // write dictionary file
-    val uniqueTableName: String = s"${ carbonLoadModel.getDatabaseName }_${
-      carbonLoadModel.getTableName
-    }"
-    result match {
-      case Some(server) =>
-        try {
-          server.writeTableDictionary(carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
-            .getCarbonTableIdentifier.getTableId)
-        } catch {
-          case _: Exception =>
-            LOGGER.error(s"Error while writing dictionary file for $uniqueTableName")
-            throw new Exception("Dataload failed due to error while writing dictionary file!")
+  /**
+   * Execute load process for partition table
+   */
+  private def loadDataForPartitionTable(
+      sqlContext: SQLContext,
+      dataFrame: Option[DataFrame],
+      carbonLoadModel: CarbonLoadModel
+  ): Array[(String, (LoadMetadataDetails, ExecutionErrors))] = {
+    try {
+      val rdd = repartitionInputData(sqlContext, dataFrame, carbonLoadModel)
+      new PartitionTableDataLoaderRDD(
+        sqlContext.sparkContext,
+        new DataLoadResultImpl(),
+        carbonLoadModel,
+        rdd
+      ).collect()
+    } catch {
+      case ex: Exception =>
+        LOGGER.error(ex, "load data failed for partition table")
+        throw ex
+    }
+  }
+
+  /**
+   * Execute load process to load from input dataframe
+   */
+  private def loadDataFrame(
+      sqlContext: SQLContext,
+      dataFrame: Option[DataFrame],
+      carbonLoadModel: CarbonLoadModel
+  ): Array[(String, (LoadMetadataDetails, ExecutionErrors))] = {
+    try {
+      val rdd = dataFrame.get.rdd
+
+      val nodeNumOfData = rdd.partitions.flatMap[String, Array[String]] { p =>
+        DataLoadPartitionCoalescer.getPreferredLocs(rdd, p).map(_.host)
+      }.distinct.length
+      val nodes = DistributionUtil.ensureExecutorsByNumberAndGetNodeList(
+        nodeNumOfData,
+        sqlContext.sparkContext)
+      val newRdd = new DataLoadCoalescedRDD[Row](rdd, nodes.toArray.distinct)
+
+      new NewDataFrameLoaderRDD(
+        sqlContext.sparkContext,
+        new DataLoadResultImpl(),
+        carbonLoadModel,
+        newRdd
+      ).collect()
+    } catch {
+      case ex: Exception =>
+        LOGGER.error(ex, "load data frame failed")
+        throw ex
+    }
+  }
+
+  /**
+   * Execute load process to load from input file path specified in `carbonLoadModel`
+   */
+  private def loadDataFile(
+      sqlContext: SQLContext,
+      carbonLoadModel: CarbonLoadModel
+  ): Array[(String, (LoadMetadataDetails, ExecutionErrors))] = {
+    /*
+     * when data load handle by node partition
+     * 1)clone the hadoop configuration,and set the file path to the configuration
+     * 2)use org.apache.hadoop.mapreduce.lib.input.TextInputFormat to get splits,size info
+     * 3)use CarbonLoaderUtil.nodeBlockMapping to get mapping info of node and block,
+     *   for locally writing carbondata files(one file one block) in nodes
+     * 4)use NewCarbonDataLoadRDD to load data and write to carbondata files
+     */
+    val hadoopConfiguration = new Configuration(sqlContext.sparkContext.hadoopConfiguration)
+    // FileUtils will skip file which is no csv, and return all file path which split by ','
+    val filePaths = carbonLoadModel.getFactFilePath
+    hadoopConfiguration.set(FileInputFormat.INPUT_DIR, filePaths)
+    hadoopConfiguration.set(FileInputFormat.INPUT_DIR_RECURSIVE, "true")
+    hadoopConfiguration.set("io.compression.codecs",
+      """org.apache.hadoop.io.compress.GzipCodec,
+             org.apache.hadoop.io.compress.DefaultCodec,
+             org.apache.hadoop.io.compress.BZip2Codec""".stripMargin)
+
+    CommonUtil.configSplitMaxSize(sqlContext.sparkContext, filePaths, hadoopConfiguration)
+
+    val inputFormat = new org.apache.hadoop.mapreduce.lib.input.TextInputFormat
+    val jobContext = new Job(hadoopConfiguration)
+    val rawSplits = inputFormat.getSplits(jobContext).toArray
+    val blockList = rawSplits.map { inputSplit =>
+      val fileSplit = inputSplit.asInstanceOf[FileSplit]
+      new TableBlockInfo(fileSplit.getPath.toString,
+        fileSplit.getStart, "1",
+        fileSplit.getLocations, fileSplit.getLength, ColumnarFormatVersion.V1, null
+      ).asInstanceOf[Distributable]
+    }
+    // group blocks to nodes, tasks
+    val startTime = System.currentTimeMillis
+    val activeNodes = DistributionUtil
+      .ensureExecutorsAndGetNodeList(blockList, sqlContext.sparkContext)
+    val nodeBlockMapping =
+      CarbonLoaderUtil
+        .nodeBlockMapping(blockList.toSeq.asJava, -1, activeNodes.toList.asJava).asScala
+        .toSeq
+    val timeElapsed: Long = System.currentTimeMillis - startTime
+    LOGGER.info("Total Time taken in block allocation: " + timeElapsed)
+    LOGGER.info(s"Total no of blocks: ${ blockList.length }, " +
+                s"No.of Nodes: ${nodeBlockMapping.size}")
+    var str = ""
+    nodeBlockMapping.foreach { entry =>
+      val tableBlock = entry._2
+      str = str + "#Node: " + entry._1 + " no.of.blocks: " + tableBlock.size()
+      tableBlock.asScala.foreach(tableBlockInfo =>
+        if (!tableBlockInfo.getLocations.exists(hostentry =>
+          hostentry.equalsIgnoreCase(entry._1)
+        )) {
+          str = str + " , mismatch locations: " + tableBlockInfo.getLocations
+            .foldLeft("")((a, b) => a + "," + b)
         }
-      case _ =>
+      )
+      str = str + "\n"
     }
+    LOGGER.info(str)
+    val blocksGroupBy: Array[(String, Array[BlockDetails])] = nodeBlockMapping.map { entry =>
+      val blockDetailsList =
+        entry._2.asScala.map(distributable => {
+          val tableBlock = distributable.asInstanceOf[TableBlockInfo]
+          new BlockDetails(new Path(tableBlock.getFilePath),
+            tableBlock.getBlockOffset, tableBlock.getBlockLength, tableBlock.getLocations
+          )
+        }).toArray
+      (entry._1, blockDetailsList)
+    }.toArray
+
+    new NewCarbonDataLoadRDD(
+      sqlContext.sparkContext,
+      new DataLoadResultImpl(),
+      carbonLoadModel,
+      blocksGroupBy
+    ).collect()
   }
+
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/0578ba0f/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala
index 6815629..bf86aca 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala
@@ -17,7 +17,6 @@
 
 package org.apache.spark.sql
 
-import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.logical.{UnaryNode, _}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/0578ba0f/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/AlterTableCompactionCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/AlterTableCompactionCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/AlterTableCompactionCommand.scala
index 52df2a4..f87e734 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/AlterTableCompactionCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/AlterTableCompactionCommand.scala
@@ -17,15 +17,21 @@
 
 package org.apache.spark.sql.execution.command.management
 
-import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
-import org.apache.spark.sql.execution.command.{AlterTableModel, DataProcessCommand, RunnableCommand}
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.{CarbonEnv, Row, SparkSession, SQLContext}
+import org.apache.spark.sql.execution.command.{AlterTableModel, CompactionModel, DataProcessCommand, RunnableCommand}
 import org.apache.spark.sql.hive.CarbonRelation
 
 import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
 import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.locks.{CarbonLockFactory, LockUsage}
+import org.apache.carbondata.core.mutate.CarbonUpdateUtil
 import org.apache.carbondata.core.util.CarbonProperties
 import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, CarbonLoadModel}
+import org.apache.carbondata.processing.merger.{CarbonDataMergerUtil, CompactionType}
 import org.apache.carbondata.spark.rdd.CarbonDataRDDFactory
+import org.apache.carbondata.spark.util.CommonUtil
 
 /**
  * Command for the compaction in alter table command
@@ -34,12 +40,14 @@ case class AlterTableCompactionCommand(
     alterTableModel: AlterTableModel)
   extends RunnableCommand with DataProcessCommand {
 
+  private val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getName)
+
   override def run(sparkSession: SparkSession): Seq[Row] = {
     processData(sparkSession)
   }
 
   override def processData(sparkSession: SparkSession): Seq[Row] = {
-    val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getName)
+
     val tableName = alterTableModel.tableName.toLowerCase
     val databaseName = alterTableModel.dbName.getOrElse(sparkSession.catalog.currentDatabase)
     val relation =
@@ -71,8 +79,7 @@ case class AlterTableCompactionCommand(
       )
     storeLocation = storeLocation + "/carbonstore/" + System.nanoTime()
     try {
-      CarbonDataRDDFactory
-        .alterTableForCompaction(sparkSession.sqlContext,
+      alterTableForCompaction(sparkSession.sqlContext,
           alterTableModel,
           carbonLoadModel,
           storeLocation
@@ -87,4 +94,111 @@ case class AlterTableCompactionCommand(
     }
     Seq.empty
   }
+
+  private def alterTableForCompaction(sqlContext: SQLContext,
+      alterTableModel: AlterTableModel,
+      carbonLoadModel: CarbonLoadModel,
+      storeLocation: String): Unit = {
+    var compactionSize: Long = 0
+    var compactionType: CompactionType = CompactionType.MINOR_COMPACTION
+    if (alterTableModel.compactionType.equalsIgnoreCase("major")) {
+      compactionSize = CarbonDataMergerUtil.getCompactionSize(CompactionType.MAJOR_COMPACTION)
+      compactionType = CompactionType.MAJOR_COMPACTION
+    } else if (alterTableModel.compactionType.equalsIgnoreCase(
+      CompactionType.IUD_UPDDEL_DELTA_COMPACTION.toString)) {
+      compactionType = CompactionType.IUD_UPDDEL_DELTA_COMPACTION
+      if (alterTableModel.segmentUpdateStatusManager.isDefined) {
+        carbonLoadModel.setSegmentUpdateStatusManager(
+          alterTableModel.segmentUpdateStatusManager.get)
+        carbonLoadModel.setLoadMetadataDetails(
+          alterTableModel.segmentUpdateStatusManager.get.getLoadMetadataDetails.toList.asJava)
+      }
+    } else if (alterTableModel.compactionType.equalsIgnoreCase(
+      CompactionType.SEGMENT_INDEX_COMPACTION.toString)) {
+      compactionType = CompactionType.SEGMENT_INDEX_COMPACTION
+    } else {
+      compactionType = CompactionType.MINOR_COMPACTION
+    }
+
+    LOGGER.audit(s"Compaction request received for table " +
+                 s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
+    val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
+
+    if (null == carbonLoadModel.getLoadMetadataDetails) {
+      CommonUtil.readLoadMetadataDetails(carbonLoadModel)
+    }
+    if (compactionType == CompactionType.SEGMENT_INDEX_COMPACTION) {
+      // Just launch job to merge index and return
+      CommonUtil.mergeIndexFiles(sqlContext.sparkContext,
+        carbonLoadModel.getLoadMetadataDetails.asScala.map(_.getLoadName),
+        carbonLoadModel.getStorePath,
+        carbonTable)
+      return
+    }
+    // reading the start time of data load.
+    val loadStartTime : Long =
+      if (alterTableModel.factTimeStamp.isEmpty) {
+        CarbonUpdateUtil.readCurrentTime
+      } else {
+        alterTableModel.factTimeStamp.get
+      }
+    carbonLoadModel.setFactTimeStamp(loadStartTime)
+
+    val isCompactionTriggerByDDl = true
+    val compactionModel = CompactionModel(compactionSize,
+      compactionType,
+      carbonTable,
+      isCompactionTriggerByDDl
+    )
+
+    val isConcurrentCompactionAllowed = CarbonProperties.getInstance()
+      .getProperty(CarbonCommonConstants.ENABLE_CONCURRENT_COMPACTION,
+        CarbonCommonConstants.DEFAULT_ENABLE_CONCURRENT_COMPACTION
+      )
+      .equalsIgnoreCase("true")
+
+    // if system level compaction is enabled then only one compaction can run in the system
+    // if any other request comes at this time then it will create a compaction request file.
+    // so that this will be taken up by the compaction process which is executing.
+    if (!isConcurrentCompactionAllowed) {
+      LOGGER.info("System level compaction lock is enabled.")
+      CarbonDataRDDFactory.handleCompactionForSystemLocking(sqlContext,
+        carbonLoadModel,
+        storeLocation,
+        compactionType,
+        carbonTable,
+        compactionModel
+      )
+    } else {
+      // normal flow of compaction
+      val lock = CarbonLockFactory
+        .getCarbonLockObj(carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
+          LockUsage.COMPACTION_LOCK
+        )
+
+      if (lock.lockWithRetries()) {
+        LOGGER.info("Acquired the compaction lock for table" +
+                    s" ${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
+        try {
+          CarbonDataRDDFactory.startCompactionThreads(sqlContext,
+            carbonLoadModel,
+            storeLocation,
+            compactionModel,
+            lock
+          )
+        } catch {
+          case e: Exception =>
+            LOGGER.error(s"Exception in start compaction thread. ${ e.getMessage }")
+            lock.unlock()
+            throw e
+        }
+      } else {
+        LOGGER.audit("Not able to acquire the compaction lock for table " +
+                     s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
+        LOGGER.error(s"Not able to acquire the compaction lock for table" +
+                     s" ${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
+        sys.error("Table is already locked for compaction. Please try after some time.")
+      }
+    }
+  }
 }


[49/49] carbondata git commit: [CARBONDATA-1480]Min Max Index Example for DataMap

Posted by ra...@apache.org.
[CARBONDATA-1480]Min Max Index Example for DataMap

Datamap Example. Implementation of Min Max Index through Datamap. And Using the Index while prunning.

This closes #1359


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

Branch: refs/heads/fgdatamap
Commit: 7f6172212619e4c950d78653ca0e2eca7ded6b31
Parents: e4f2843
Author: sounakr <so...@gmail.com>
Authored: Thu Sep 28 16:21:05 2017 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Tue Nov 14 03:40:48 2017 +0530

----------------------------------------------------------------------
 .../core/datamap/DataMapStoreManager.java       |  16 +-
 .../carbondata/core/datamap/TableDataMap.java   |  15 +-
 .../carbondata/core/datamap/dev/DataMap.java    |   3 +-
 .../core/datamap/dev/DataMapWriter.java         |   3 +-
 .../indexstore/SegmentPropertiesFetcher.java    |  36 +++
 .../blockletindex/BlockletDataMap.java          |   2 +-
 .../blockletindex/BlockletDataMapFactory.java   |  33 ++-
 datamap/examples/pom.xml                        | 111 ++++++++++
 .../datamap/examples/BlockletMinMax.java        |  41 ++++
 .../datamap/examples/MinMaxDataMap.java         | 143 ++++++++++++
 .../datamap/examples/MinMaxDataMapFactory.java  | 114 ++++++++++
 .../datamap/examples/MinMaxDataWriter.java      | 221 +++++++++++++++++++
 .../examples/MinMaxIndexBlockDetails.java       |  77 +++++++
 .../MinMaxDataMapExample.scala                  |  77 +++++++
 .../testsuite/datamap/DataMapWriterSuite.scala  |   2 +-
 pom.xml                                         |   2 +
 .../datamap/DataMapWriterListener.java          |   4 +-
 .../store/writer/AbstractFactDataWriter.java    |   7 +-
 .../writer/v3/CarbonFactDataWriterImplV3.java   |   3 +
 19 files changed, 892 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/7f617221/core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java b/core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java
index 2b5d5cd..3851d19 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java
@@ -26,6 +26,7 @@ import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.datamap.dev.DataMapFactory;
 import org.apache.carbondata.core.indexstore.BlockletDetailsFetcher;
+import org.apache.carbondata.core.indexstore.SegmentPropertiesFetcher;
 import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataMap;
 import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataMapFactory;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
@@ -103,7 +104,7 @@ public final class DataMapStoreManager {
       tableDataMaps = new ArrayList<>();
     }
     TableDataMap dataMap = getTableDataMap(dataMapName, tableDataMaps);
-    if (dataMap != null) {
+    if (dataMap != null && dataMap.getDataMapName().equalsIgnoreCase(dataMapName)) {
       throw new RuntimeException("Already datamap exists in that path with type " + dataMapName);
     }
 
@@ -113,12 +114,15 @@ public final class DataMapStoreManager {
       DataMapFactory dataMapFactory = factoryClass.newInstance();
       dataMapFactory.init(identifier, dataMapName);
       BlockletDetailsFetcher blockletDetailsFetcher;
+      SegmentPropertiesFetcher segmentPropertiesFetcher = null;
       if (dataMapFactory instanceof BlockletDetailsFetcher) {
         blockletDetailsFetcher = (BlockletDetailsFetcher) dataMapFactory;
       } else {
         blockletDetailsFetcher = getBlockletDetailsFetcher(identifier);
       }
-      dataMap = new TableDataMap(identifier, dataMapName, dataMapFactory, blockletDetailsFetcher);
+      segmentPropertiesFetcher = (SegmentPropertiesFetcher) blockletDetailsFetcher;
+      dataMap = new TableDataMap(identifier, dataMapName, dataMapFactory, blockletDetailsFetcher,
+          segmentPropertiesFetcher);
     } catch (Exception e) {
       LOGGER.error(e);
       throw new RuntimeException(e);
@@ -128,11 +132,11 @@ public final class DataMapStoreManager {
     return dataMap;
   }
 
-  private TableDataMap getTableDataMap(String dataMapName,
-      List<TableDataMap> tableDataMaps) {
+  private TableDataMap getTableDataMap(String dataMapName, List<TableDataMap> tableDataMaps) {
     TableDataMap dataMap = null;
-    for (TableDataMap tableDataMap: tableDataMaps) {
-      if (tableDataMap.getDataMapName().equals(dataMapName)) {
+    for (TableDataMap tableDataMap : tableDataMaps) {
+      if (tableDataMap.getDataMapName().equals(dataMapName) || (!tableDataMap.getDataMapName()
+          .equals(""))) {
         dataMap = tableDataMap;
         break;
       }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7f617221/core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java b/core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java
index 705a9fd..07d8eb7 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java
@@ -22,9 +22,11 @@ import java.util.List;
 
 import org.apache.carbondata.core.datamap.dev.DataMap;
 import org.apache.carbondata.core.datamap.dev.DataMapFactory;
+import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.indexstore.Blocklet;
 import org.apache.carbondata.core.indexstore.BlockletDetailsFetcher;
 import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
+import org.apache.carbondata.core.indexstore.SegmentPropertiesFetcher;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
 import org.apache.carbondata.events.Event;
@@ -45,15 +47,19 @@ public final class TableDataMap implements OperationEventListener {
 
   private BlockletDetailsFetcher blockletDetailsFetcher;
 
+  private SegmentPropertiesFetcher segmentPropertiesFetcher;
+
   /**
    * It is called to initialize and load the required table datamap metadata.
    */
   public TableDataMap(AbsoluteTableIdentifier identifier, String dataMapName,
-      DataMapFactory dataMapFactory, BlockletDetailsFetcher blockletDetailsFetcher) {
+      DataMapFactory dataMapFactory, BlockletDetailsFetcher blockletDetailsFetcher,
+      SegmentPropertiesFetcher segmentPropertiesFetcher) {
     this.identifier = identifier;
     this.dataMapName = dataMapName;
     this.dataMapFactory = dataMapFactory;
     this.blockletDetailsFetcher = blockletDetailsFetcher;
+    this.segmentPropertiesFetcher = segmentPropertiesFetcher;
   }
 
   /**
@@ -66,11 +72,13 @@ public final class TableDataMap implements OperationEventListener {
   public List<ExtendedBlocklet> prune(List<String> segmentIds, FilterResolverIntf filterExp)
       throws IOException {
     List<ExtendedBlocklet> blocklets = new ArrayList<>();
+    SegmentProperties segmentProperties;
     for (String segmentId : segmentIds) {
       List<Blocklet> pruneBlocklets = new ArrayList<>();
       List<DataMap> dataMaps = dataMapFactory.getDataMaps(segmentId);
+      segmentProperties = segmentPropertiesFetcher.getSegmentProperties(segmentId);
       for (DataMap dataMap : dataMaps) {
-        pruneBlocklets.addAll(dataMap.prune(filterExp));
+        pruneBlocklets.addAll(dataMap.prune(filterExp, segmentProperties));
       }
       blocklets.addAll(addSegmentId(blockletDetailsFetcher
           .getExtendedBlocklets(pruneBlocklets, segmentId), segmentId));
@@ -122,7 +130,8 @@ public final class TableDataMap implements OperationEventListener {
     List<Blocklet> blocklets = new ArrayList<>();
     List<DataMap> dataMaps = dataMapFactory.getDataMaps(distributable);
     for (DataMap dataMap : dataMaps) {
-      blocklets.addAll(dataMap.prune(filterExp));
+      blocklets.addAll(dataMap.prune(filterExp,
+          segmentPropertiesFetcher.getSegmentProperties(distributable.getSegmentId())));
     }
     for (Blocklet blocklet: blocklets) {
       ExtendedBlocklet detailedBlocklet =

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7f617221/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java
index ada23ad..3731922 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java
@@ -19,6 +19,7 @@ package org.apache.carbondata.core.datamap.dev;
 import java.io.IOException;
 import java.util.List;
 
+import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.indexstore.Blocklet;
 import org.apache.carbondata.core.memory.MemoryException;
 import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
@@ -40,7 +41,7 @@ public interface DataMap {
    * @param filterExp
    * @return
    */
-  List<Blocklet> prune(FilterResolverIntf filterExp);
+  List<Blocklet> prune(FilterResolverIntf filterExp, SegmentProperties segmentProperties);
 
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7f617221/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapWriter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapWriter.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapWriter.java
index 28163d7..413eaa5 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapWriter.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapWriter.java
@@ -27,7 +27,7 @@ public interface DataMapWriter {
    *  Start of new block notification.
    *  @param blockId file name of the carbondata file
    */
-  void onBlockStart(String blockId);
+  void onBlockStart(String blockId, String blockPath);
 
   /**
    * End of block notification
@@ -45,7 +45,6 @@ public interface DataMapWriter {
    * @param blockletId sequence number of blocklet in the block
    */
   void onBlockletEnd(int blockletId);
-
   /**
    * Add the column pages row to the datamap, order of pages is same as `indexColumns` in
    * DataMapMeta returned in DataMapFactory.

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7f617221/core/src/main/java/org/apache/carbondata/core/indexstore/SegmentPropertiesFetcher.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/SegmentPropertiesFetcher.java b/core/src/main/java/org/apache/carbondata/core/indexstore/SegmentPropertiesFetcher.java
new file mode 100644
index 0000000..ec2ae93
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/SegmentPropertiesFetcher.java
@@ -0,0 +1,36 @@
+/*
+ * 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.indexstore;
+
+import java.io.IOException;
+
+import org.apache.carbondata.core.datastore.block.SegmentProperties;
+
+/**
+ * Fetches the detailed segmentProperties which has more information to execute the query
+ */
+public interface SegmentPropertiesFetcher {
+
+  /**
+   * get the Segment properties based on the SegmentID.
+   * @param segmentId
+   * @return
+   * @throws IOException
+   */
+  SegmentProperties getSegmentProperties(String segmentId) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7f617221/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 43e265d..54016ad 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
@@ -315,7 +315,7 @@ public class BlockletDataMap implements DataMap, Cacheable {
   }
 
   @Override
-  public List<Blocklet> prune(FilterResolverIntf filterExp) {
+  public List<Blocklet> prune(FilterResolverIntf filterExp, SegmentProperties segmentProperties) {
 
     // getting the start and end index key based on filter for hitting the
     // selected block reference nodes based on filter resolver tree.

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7f617221/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java
index e58c911..abdad6f 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java
@@ -30,13 +30,18 @@ import org.apache.carbondata.core.datamap.DataMapMeta;
 import org.apache.carbondata.core.datamap.dev.DataMap;
 import org.apache.carbondata.core.datamap.dev.DataMapFactory;
 import org.apache.carbondata.core.datamap.dev.DataMapWriter;
+import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.indexstore.Blocklet;
 import org.apache.carbondata.core.indexstore.BlockletDetailsFetcher;
 import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
+import org.apache.carbondata.core.indexstore.SegmentPropertiesFetcher;
 import org.apache.carbondata.core.indexstore.TableBlockIndexUniqueIdentifier;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
+import org.apache.carbondata.core.metadata.blocklet.DataFileFooter;
+import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
+import org.apache.carbondata.core.util.DataFileFooterConverter;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.events.Event;
 
@@ -48,13 +53,17 @@ import org.apache.hadoop.fs.RemoteIterator;
 /**
  * Table map for blocklet
  */
-public class BlockletDataMapFactory implements DataMapFactory, BlockletDetailsFetcher {
+public class BlockletDataMapFactory implements DataMapFactory, BlockletDetailsFetcher,
+    SegmentPropertiesFetcher {
 
   private AbsoluteTableIdentifier identifier;
 
   // segmentId -> list of index file
   private Map<String, List<TableBlockIndexUniqueIdentifier>> segmentMap = new HashMap<>();
 
+  // segmentId -> SegmentProperties.
+  private Map<String, SegmentProperties> segmentPropertiesMap = new HashMap<>();
+
   private Cache<TableBlockIndexUniqueIdentifier, DataMap> cache;
 
   @Override
@@ -170,6 +179,7 @@ public class BlockletDataMapFactory implements DataMapFactory, BlockletDetailsFe
 
   @Override
   public void clear(String segmentId) {
+    segmentPropertiesMap.remove(segmentId);
     List<TableBlockIndexUniqueIdentifier> blockIndexes = segmentMap.remove(segmentId);
     if (blockIndexes != null) {
       for (TableBlockIndexUniqueIdentifier blockIndex : blockIndexes) {
@@ -221,4 +231,25 @@ public class BlockletDataMapFactory implements DataMapFactory, BlockletDetailsFe
     // TODO: pass SORT_COLUMNS into this class
     return null;
   }
+
+  @Override public SegmentProperties getSegmentProperties(String segmentId) throws IOException {
+    SegmentProperties segmentProperties = segmentPropertiesMap.get(segmentId);
+    if (segmentProperties == null) {
+      int[] columnCardinality;
+      List<TableBlockIndexUniqueIdentifier> tableBlockIndexUniqueIdentifiers =
+          getTableBlockIndexUniqueIdentifiers(segmentId);
+      DataFileFooterConverter fileFooterConverter = new DataFileFooterConverter();
+      List<DataFileFooter> indexInfo =
+          fileFooterConverter.getIndexInfo(tableBlockIndexUniqueIdentifiers.get(0).getFilePath());
+      for (DataFileFooter fileFooter : indexInfo) {
+        List<ColumnSchema> columnInTable = fileFooter.getColumnInTable();
+        if (segmentProperties == null) {
+          columnCardinality = fileFooter.getSegmentInfo().getColumnCardinality();
+          segmentProperties = new SegmentProperties(columnInTable, columnCardinality);
+        }
+      }
+      segmentPropertiesMap.put(segmentId, segmentProperties);
+    }
+    return segmentProperties;
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7f617221/datamap/examples/pom.xml
----------------------------------------------------------------------
diff --git a/datamap/examples/pom.xml b/datamap/examples/pom.xml
new file mode 100644
index 0000000..6832e62
--- /dev/null
+++ b/datamap/examples/pom.xml
@@ -0,0 +1,111 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+    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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.carbondata</groupId>
+        <artifactId>carbondata-parent</artifactId>
+        <version>1.3.0-SNAPSHOT</version>
+        <relativePath>../../pom.xml</relativePath>
+    </parent>
+
+    <artifactId>carbondata-datamap-examples</artifactId>
+    <name>Apache CarbonData :: Datamap Examples</name>
+
+    <properties>
+        <dev.path>${basedir}/../../dev</dev.path>
+    </properties>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.carbondata</groupId>
+            <artifactId>carbondata-spark2</artifactId>
+            <version>${project.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.apache.spark</groupId>
+                    <artifactId>spark-hive-thriftserver_2.10</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.spark</groupId>
+                    <artifactId>spark-repl_2.10</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.spark</groupId>
+                    <artifactId>spark-sql_2.10</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-sql_${scala.binary.version}</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-hive-thriftserver_${scala.binary.version}</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-repl_${scala.binary.version}</artifactId>
+        </dependency>
+    </dependencies>
+
+    <build>
+        <sourceDirectory>src/minmaxdatamap/main/java</sourceDirectory>
+        <resources>
+            <resource>
+                <directory>.</directory>
+                <includes>
+                    <include>CARBON_EXAMPLESLogResource.properties</include>
+                </includes>
+            </resource>
+        </resources>
+        <plugins>
+            <plugin>
+                <groupId>org.scala-tools</groupId>
+                <artifactId>maven-scala-plugin</artifactId>
+                <version>2.15.2</version>
+                <executions>
+                    <execution>
+                        <id>compile</id>
+                        <goals>
+                            <goal>compile</goal>
+                        </goals>
+                        <phase>compile</phase>
+                    </execution>
+                    <execution>
+                        <phase>process-resources</phase>
+                        <goals>
+                            <goal>compile</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <artifactId>maven-compiler-plugin</artifactId>
+                <configuration>
+                    <source>1.7</source>
+                    <target>1.7</target>
+                </configuration>
+            </plugin>
+        </plugins>
+    </build>
+
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7f617221/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/BlockletMinMax.java
----------------------------------------------------------------------
diff --git a/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/BlockletMinMax.java b/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/BlockletMinMax.java
new file mode 100644
index 0000000..e6968fe
--- /dev/null
+++ b/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/BlockletMinMax.java
@@ -0,0 +1,41 @@
+/*
+ * 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.datamap.examples;
+
+
+public class BlockletMinMax {
+  private byte[][] Min;
+
+  private byte[][] Max;
+
+  public byte[][] getMin() {
+    return Min;
+  }
+
+  public void setMin(byte[][] min) {
+    Min = min;
+  }
+
+  public byte[][] getMax() {
+    return Max;
+  }
+
+  public void setMax(byte[][] max) {
+    Max = max;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7f617221/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataMap.java
----------------------------------------------------------------------
diff --git a/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataMap.java b/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataMap.java
new file mode 100644
index 0000000..2ad6327
--- /dev/null
+++ b/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataMap.java
@@ -0,0 +1,143 @@
+/*
+ * 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.datamap.examples;
+
+import java.io.BufferedReader;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.List;
+
+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.datamap.dev.DataMap;
+import org.apache.carbondata.core.datastore.block.SegmentProperties;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFileFilter;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.fileoperations.AtomicFileOperations;
+import org.apache.carbondata.core.fileoperations.AtomicFileOperationsImpl;
+import org.apache.carbondata.core.indexstore.Blocklet;
+import org.apache.carbondata.core.memory.MemoryException;
+import org.apache.carbondata.core.scan.filter.FilterUtil;
+import org.apache.carbondata.core.scan.filter.executer.FilterExecuter;
+import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
+import org.apache.carbondata.core.util.CarbonUtil;
+
+import com.google.gson.Gson;
+
+/**
+ * Datamap implementation for min max blocklet.
+ */
+public class MinMaxDataMap implements DataMap {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(MinMaxDataMap.class.getName());
+
+  public static final String NAME = "clustered.minmax.btree.blocklet";
+
+  private String filePath;
+
+  private MinMaxIndexBlockDetails[] readMinMaxDataMap;
+
+  @Override public void init(String filePath) throws MemoryException, IOException {
+    this.filePath = filePath;
+    CarbonFile[] listFiles = getCarbonMinMaxIndexFiles(filePath, "0");
+    for (int i = 0; i < listFiles.length; i++) {
+      readMinMaxDataMap = readJson(listFiles[i].getPath());
+    }
+  }
+
+  private CarbonFile[] getCarbonMinMaxIndexFiles(String filePath, String segmentId) {
+    String path = filePath.substring(0, filePath.lastIndexOf("/") + 1);
+    CarbonFile carbonFile = FileFactory.getCarbonFile(path);
+    return carbonFile.listFiles(new CarbonFileFilter() {
+      @Override public boolean accept(CarbonFile file) {
+        return file.getName().endsWith(".minmaxindex");
+      }
+    });
+  }
+
+  public MinMaxIndexBlockDetails[] readJson(String filePath) throws IOException {
+    Gson gsonObjectToRead = new Gson();
+    DataInputStream dataInputStream = null;
+    BufferedReader buffReader = null;
+    InputStreamReader inStream = null;
+    MinMaxIndexBlockDetails[] readMinMax = null;
+    AtomicFileOperations fileOperation =
+        new AtomicFileOperationsImpl(filePath, FileFactory.getFileType(filePath));
+
+    try {
+      if (!FileFactory.isFileExist(filePath, FileFactory.getFileType(filePath))) {
+        return null;
+      }
+      dataInputStream = fileOperation.openForRead();
+      inStream = new InputStreamReader(dataInputStream,
+          CarbonCommonConstants.CARBON_DEFAULT_STREAM_ENCODEFORMAT);
+      buffReader = new BufferedReader(inStream);
+      readMinMax = gsonObjectToRead.fromJson(buffReader, MinMaxIndexBlockDetails[].class);
+    } catch (IOException e) {
+      return null;
+    } finally {
+      CarbonUtil.closeStreams(buffReader, inStream, dataInputStream);
+    }
+    return readMinMax;
+  }
+
+  /**
+   * Block Prunning logic for Min Max DataMap.
+   *
+   * @param filterExp
+   * @param segmentProperties
+   * @return
+   */
+  @Override public List<Blocklet> prune(FilterResolverIntf filterExp,
+      SegmentProperties segmentProperties) {
+    List<Blocklet> blocklets = new ArrayList<>();
+
+    if (filterExp == null) {
+      for (int i = 0; i < readMinMaxDataMap.length; i++) {
+        blocklets.add(new Blocklet(readMinMaxDataMap[i].getFilePath(),
+            String.valueOf(readMinMaxDataMap[i].getBlockletId())));
+      }
+    } else {
+      FilterExecuter filterExecuter =
+          FilterUtil.getFilterExecuterTree(filterExp, segmentProperties, null);
+      int startIndex = 0;
+      while (startIndex < readMinMaxDataMap.length) {
+        BitSet bitSet = filterExecuter.isScanRequired(readMinMaxDataMap[startIndex].getMaxValues(),
+            readMinMaxDataMap[startIndex].getMinValues());
+        if (!bitSet.isEmpty()) {
+          blocklets.add(new Blocklet(readMinMaxDataMap[startIndex].getFilePath(),
+              String.valueOf(readMinMaxDataMap[startIndex].getBlockletId())));
+        }
+        startIndex++;
+      }
+    }
+    return blocklets;
+  }
+
+  @Override
+  public void clear() {
+    readMinMaxDataMap = null;
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7f617221/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataMapFactory.java
----------------------------------------------------------------------
diff --git a/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataMapFactory.java b/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataMapFactory.java
new file mode 100644
index 0000000..b196d0d
--- /dev/null
+++ b/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataMapFactory.java
@@ -0,0 +1,114 @@
+/*
+ * 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.datamap.examples;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.carbondata.core.datamap.DataMapDistributable;
+import org.apache.carbondata.core.datamap.DataMapMeta;
+import org.apache.carbondata.core.datamap.dev.DataMap;
+import org.apache.carbondata.core.datamap.dev.DataMapFactory;
+import org.apache.carbondata.core.datamap.dev.DataMapWriter;
+import org.apache.carbondata.core.events.ChangeEvent;
+import org.apache.carbondata.core.indexstore.schema.FilterType;
+import org.apache.carbondata.core.memory.MemoryException;
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
+
+
+/**
+ * Min Max DataMap Factory
+ */
+public class MinMaxDataMapFactory implements DataMapFactory {
+
+  private AbsoluteTableIdentifier identifier;
+
+  @Override
+  public void init(AbsoluteTableIdentifier identifier, String dataMapName) {
+    this.identifier = identifier;
+  }
+
+  /**
+   * createWriter will return the MinMaxDataWriter.
+   * @param segmentId
+   * @return
+   */
+  @Override
+  public DataMapWriter createWriter(String segmentId) {
+    return new MinMaxDataWriter();
+  }
+
+  /**
+   * getDataMaps Factory method Initializes the Min Max Data Map and returns.
+   * @param segmentId
+   * @return
+   * @throws IOException
+   */
+  @Override public List<DataMap> getDataMaps(String segmentId) throws IOException {
+    List<DataMap> dataMapList = new ArrayList<>();
+    // Form a dataMap of Type MinMaxDataMap.
+    MinMaxDataMap dataMap = new MinMaxDataMap();
+    try {
+      dataMap.init(identifier.getTablePath() + "/Fact/Part0/Segment_" + segmentId + File.separator);
+    } catch (MemoryException ex) {
+
+    }
+    dataMapList.add(dataMap);
+    return dataMapList;
+  }
+
+  /**
+   *
+   * @param segmentId
+   * @return
+   */
+  @Override public List<DataMapDistributable> toDistributable(String segmentId) {
+    return null;
+  }
+
+  /**
+   * Clear the DataMap.
+   * @param segmentId
+   */
+  @Override public void clear(String segmentId) {
+  }
+
+  /**
+   * Clearing the data map.
+   */
+  @Override
+  public void clear() {
+  }
+
+  @Override public DataMap getDataMap(DataMapDistributable distributable) {
+    return null;
+  }
+
+  @Override
+  public void fireEvent(ChangeEvent event) {
+
+  }
+
+  @Override
+  public DataMapMeta getMeta() {
+    return new DataMapMeta(new ArrayList<String>(Arrays.asList("c2")), FilterType.EQUALTO);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7f617221/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataWriter.java
----------------------------------------------------------------------
diff --git a/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataWriter.java b/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataWriter.java
new file mode 100644
index 0000000..78544d3
--- /dev/null
+++ b/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataWriter.java
@@ -0,0 +1,221 @@
+/*
+ * 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.datamap.examples;
+
+import java.io.BufferedWriter;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+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.datamap.dev.DataMapWriter;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.datastore.page.ColumnPage;
+import org.apache.carbondata.core.metadata.schema.table.TableInfo;
+import org.apache.carbondata.core.util.ByteUtil;
+import org.apache.carbondata.core.util.CarbonUtil;
+
+import com.google.gson.Gson;
+
+public class MinMaxDataWriter implements DataMapWriter {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(TableInfo.class.getName());
+
+  private byte[][] pageLevelMin, pageLevelMax;
+
+  private byte[][] blockletLevelMin, blockletLevelMax;
+
+  private Map<Integer, BlockletMinMax> blockMinMaxMap;
+
+  private String blockPath;
+
+
+  @Override public void onBlockStart(String blockId, String blockPath) {
+    pageLevelMax = null;
+    pageLevelMin = null;
+    blockletLevelMax = null;
+    blockletLevelMin = null;
+    blockMinMaxMap = null;
+    blockMinMaxMap = new HashMap<Integer, BlockletMinMax>();
+    this.blockPath = blockPath;
+  }
+
+  @Override public void onBlockEnd(String blockId) {
+    updateMinMaxIndex(blockId);
+  }
+
+  @Override public void onBlockletStart(int blockletId) {
+  }
+
+  @Override public void onBlockletEnd(int blockletId) {
+    updateBlockletMinMax(blockletId);
+  }
+
+  @Override
+  public void onPageAdded(int blockletId, int pageId, ColumnPage[] pages) {
+    // Calculate Min and Max value within this page.
+
+    // As part of example we are extracting Min Max values Manually. The same can be done from
+    // retrieving the page statistics. For e.g.
+
+    // if (pageLevelMin == null && pageLevelMax == null) {
+    //    pageLevelMin[1] = CarbonUtil.getValueAsBytes(pages[0].getStatistics().getDataType(),
+    //        pages[0].getStatistics().getMin());
+    //    pageLevelMax[1] = CarbonUtil.getValueAsBytes(pages[0].getStatistics().getDataType(),
+    //        pages[0].getStatistics().getMax());
+    //  } else {
+    //    if (ByteUtil.UnsafeComparer.INSTANCE.compareTo(pageLevelMin[1], CarbonUtil
+    //        .getValueAsBytes(pages[0].getStatistics().getDataType(),
+    //            pages[0].getStatistics().getMin())) > 0) {
+    //      pageLevelMin[1] = CarbonUtil.getValueAsBytes(pages[0].getStatistics().getDataType(),
+    //          pages[0].getStatistics().getMin());
+    //    }
+    //    if (ByteUtil.UnsafeComparer.INSTANCE.compareTo(pageLevelMax[1], CarbonUtil
+    //        .getValueAsBytes(pages[0].getStatistics().getDataType(),
+    //            pages[0].getStatistics().getMax())) < 0) {
+    //      pageLevelMax[1] = CarbonUtil.getValueAsBytes(pages[0].getStatistics().getDataType(),
+    //          pages[0].getStatistics().getMax());
+    //    }
+
+    byte[] value = new byte[pages[0].getBytes(0).length - 2];
+    if (pageLevelMin == null && pageLevelMax == null) {
+      pageLevelMin = new byte[2][];
+      pageLevelMax = new byte[2][];
+
+      System.arraycopy(pages[0].getBytes(0), 2, value, 0, value.length);
+      pageLevelMin[1] = value;
+      pageLevelMax[1] = value;
+
+    } else {
+      for (int rowIndex = 0; rowIndex < pages[0].getPageSize(); rowIndex++) {
+        System.arraycopy(pages[0].getBytes(rowIndex), 2, value, 0, value.length);
+        if (ByteUtil.UnsafeComparer.INSTANCE.compareTo(pageLevelMin[1], value) > 0) {
+          pageLevelMin[1] = value;
+        }
+        if (ByteUtil.UnsafeComparer.INSTANCE.compareTo(pageLevelMax[1], value) < 0) {
+          pageLevelMax[1] = value;
+        }
+      }
+    }
+  }
+
+  private void updateBlockletMinMax(int blockletId) {
+    if (blockletLevelMax == null || blockletLevelMin == null) {
+      blockletLevelMax = new byte[2][];
+      blockletLevelMin = new byte[2][];
+      if (pageLevelMax != null || pageLevelMin != null) {
+        blockletLevelMin = pageLevelMin;
+        blockletLevelMax = pageLevelMax;
+      }
+    } else {
+      if (ByteUtil.UnsafeComparer.INSTANCE.compareTo(blockletLevelMin[1], pageLevelMin[1]) > 0) {
+        blockletLevelMin = pageLevelMin;
+      }
+
+      if (ByteUtil.UnsafeComparer.INSTANCE.compareTo(blockletLevelMax[1], pageLevelMax[1]) > 0) {
+        blockletLevelMax = pageLevelMax;
+      }
+    }
+    BlockletMinMax blockletMinMax = new BlockletMinMax();
+    blockletMinMax.setMax(blockletLevelMax);
+    blockletMinMax.setMin(blockletLevelMin);
+    blockMinMaxMap.put(blockletId, blockletMinMax);
+  }
+
+
+  public void updateMinMaxIndex(String blockId) {
+    constructMinMaxIndex(blockId);
+  }
+
+
+
+  /**
+   * Construct the Min Max Index.
+   * @param blockId
+   */
+  public void constructMinMaxIndex(String blockId) {
+    // construct Min and Max values of each Blocklets present inside a block.
+    List<MinMaxIndexBlockDetails> tempMinMaxIndexBlockDetails = null;
+    tempMinMaxIndexBlockDetails = loadBlockDetails();
+    try {
+      writeMinMaxIndexFile(tempMinMaxIndexBlockDetails, blockPath, blockId);
+    } catch (IOException ex) {
+      LOGGER.info(" Unable to write the file");
+    }
+  }
+
+  /**
+   * loadBlockDetails into the MinMaxIndexBlockDetails class.
+   */
+  private List<MinMaxIndexBlockDetails> loadBlockDetails() {
+    List<MinMaxIndexBlockDetails> minMaxIndexBlockDetails = new ArrayList<MinMaxIndexBlockDetails>();
+    MinMaxIndexBlockDetails tmpminMaxIndexBlockDetails = new MinMaxIndexBlockDetails();
+
+    for (int index = 0; index < blockMinMaxMap.size(); index++) {
+      tmpminMaxIndexBlockDetails.setMinValues(blockMinMaxMap.get(index).getMin());
+      tmpminMaxIndexBlockDetails.setMaxValues(blockMinMaxMap.get(index).getMax());
+      tmpminMaxIndexBlockDetails.setBlockletId(index);
+      tmpminMaxIndexBlockDetails.setFilePath(this.blockPath);
+      minMaxIndexBlockDetails.add(tmpminMaxIndexBlockDetails);
+    }
+    return minMaxIndexBlockDetails;
+  }
+
+  /**
+   * Write the data to a file. This is JSON format file.
+   * @param minMaxIndexBlockDetails
+   * @param blockPath
+   * @param blockId
+   * @throws IOException
+   */
+  public void writeMinMaxIndexFile(List<MinMaxIndexBlockDetails> minMaxIndexBlockDetails,
+      String blockPath, String blockId) throws IOException {
+    String filePath = blockPath.substring(0, blockPath.lastIndexOf(File.separator) + 1) + blockId
+        + ".minmaxindex";
+    BufferedWriter brWriter = null;
+    DataOutputStream dataOutStream = null;
+    try {
+      FileFactory.createNewFile(filePath, FileFactory.getFileType(filePath));
+      dataOutStream = FileFactory.getDataOutputStream(filePath, FileFactory.getFileType(filePath));
+      Gson gsonObjectToWrite = new Gson();
+      brWriter = new BufferedWriter(new OutputStreamWriter(dataOutStream,
+          CarbonCommonConstants.CARBON_DEFAULT_STREAM_ENCODEFORMAT));
+      String minmaxIndexData = gsonObjectToWrite.toJson(minMaxIndexBlockDetails);
+      brWriter.write(minmaxIndexData);
+    } catch (IOException ioe) {
+      LOGGER.info("Error in writing minMaxindex file");
+    } finally {
+      if (null != brWriter) {
+        brWriter.flush();
+      }
+      if (null != dataOutStream) {
+        dataOutStream.flush();
+      }
+      CarbonUtil.closeStreams(brWriter, dataOutStream);
+    }
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7f617221/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxIndexBlockDetails.java
----------------------------------------------------------------------
diff --git a/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxIndexBlockDetails.java b/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxIndexBlockDetails.java
new file mode 100644
index 0000000..0596db5
--- /dev/null
+++ b/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxIndexBlockDetails.java
@@ -0,0 +1,77 @@
+/*
+ * 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.datamap.examples;
+
+import java.io.Serializable;
+
+public class MinMaxIndexBlockDetails implements Serializable {
+  private static final long serialVersionUID = 1206104914911491724L;
+
+  /**
+   * Min value of a column of one blocklet Bit-Packed
+   */
+  private byte[][] minValues;
+
+  /**
+   * Max value of a columns of one blocklet Bit-Packed
+   */
+  private byte[][] maxValues;
+
+  /**
+   * filePath pointing to the block.
+   */
+  private String filePath;
+
+  /**
+   * BlockletID of the block.
+   */
+  private Integer BlockletId;
+
+
+  public byte[][] getMinValues() {
+    return minValues;
+  }
+
+  public void setMinValues(byte[][] minValues) {
+    this.minValues = minValues;
+  }
+
+  public byte[][] getMaxValues() {
+    return maxValues;
+  }
+
+  public void setMaxValues(byte[][] maxValues) {
+    this.maxValues = maxValues;
+  }
+
+  public String getFilePath() {
+    return filePath;
+  }
+
+  public void setFilePath(String filePath) {
+    this.filePath = filePath;
+  }
+
+  public Integer getBlockletId() {
+    return BlockletId;
+  }
+
+  public void setBlockletId(Integer blockletId) {
+    BlockletId = blockletId;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7f617221/datamap/examples/src/minmaxdatamap/test/scala/minmaxdatamaptestcase/MinMaxDataMapExample.scala
----------------------------------------------------------------------
diff --git a/datamap/examples/src/minmaxdatamap/test/scala/minmaxdatamaptestcase/MinMaxDataMapExample.scala b/datamap/examples/src/minmaxdatamap/test/scala/minmaxdatamaptestcase/MinMaxDataMapExample.scala
new file mode 100644
index 0000000..0cfe410
--- /dev/null
+++ b/datamap/examples/src/minmaxdatamap/test/scala/minmaxdatamaptestcase/MinMaxDataMapExample.scala
@@ -0,0 +1,77 @@
+/*
+ * 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.datamap.examples
+
+import java.io.File
+
+import org.apache.spark.sql.{DataFrame, SaveMode, SparkSession}
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datamap.DataMapStoreManager
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
+import org.apache.carbondata.core.util.CarbonProperties
+
+object MinMaxDataMapExample {
+  def main(args: Array[String]): Unit = {
+
+    val rootPath = new File(this.getClass.getResource("/").getPath
+                            + "").getCanonicalPath
+    val storeLocation = s"$rootPath/dataMap/examples/target/store"
+    val warehouse = s"$rootPath/datamap/examples/target/warehouse"
+    val metastoredb = s"$rootPath/datamap/examples/target"
+
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/MM/dd")
+
+    import org.apache.spark.sql.CarbonSession._
+
+    val spark = SparkSession
+      .builder()
+      .master("local")
+      .appName("CarbonDataMapExample")
+      .config("spark.sql.warehouse.dir", warehouse)
+      .getOrCreateCarbonSession(storeLocation)
+
+    spark.sparkContext.setLogLevel("ERROR")
+    import spark.implicits._
+
+    // register datamap writer
+    DataMapStoreManager.getInstance().createAndRegisterDataMap(
+      AbsoluteTableIdentifier.from(storeLocation, "default", "carbonminmax"),
+      classOf[MinMaxDataMapFactory].getName,
+      MinMaxDataMap.NAME)
+
+    spark.sql("DROP TABLE IF EXISTS carbonminmax")
+
+    val df = spark.sparkContext.parallelize(1 to 33000)
+      .map(x => ("a", "b", x))
+      .toDF("c1", "c2", "c3")
+
+    // save dataframe to carbon file
+    df.write
+      .format("carbondata")
+      .option("tableName", "carbonminmax")
+      .mode(SaveMode.Overwrite)
+      .save()
+
+    // Query the table.
+    spark.sql("select c2 from carbonminmax").show(20, false)
+    spark.sql("select c2 from carbonminmax where c2 = 'b'").show(20, false)
+    spark.sql("DROP TABLE IF EXISTS carbonminmax")
+
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7f617221/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala
index 348d9a7..18c3d33 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala
@@ -182,7 +182,7 @@ object DataMapWriterSuite {
       callbackSeq :+= s"blocklet start $blockletId"
     }
 
-    override def onBlockStart(blockId: String): Unit = {
+    override def onBlockStart(blockId: String, blockPath: String): Unit = {
       callbackSeq :+= s"block start $blockId"
     }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7f617221/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 342cb2b..18fd17c 100644
--- a/pom.xml
+++ b/pom.xml
@@ -435,6 +435,7 @@
         <module>format</module>
         <module>integration/spark2</module>
         <module>examples/spark2</module>
+        <module>datamap/examples</module>
         <module>integration/hive</module>
         <module>integration/presto</module>
         <module>examples/flink</module>
@@ -474,6 +475,7 @@
         <module>integration/presto</module>
         <module>streaming</module>
         <module>examples/spark2</module>
+        <module>datamap/examples</module>
       </modules>
       <build>
         <plugins>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7f617221/processing/src/main/java/org/apache/carbondata/processing/datamap/DataMapWriterListener.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/datamap/DataMapWriterListener.java b/processing/src/main/java/org/apache/carbondata/processing/datamap/DataMapWriterListener.java
index 4b0113c..8e350d9 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/datamap/DataMapWriterListener.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/datamap/DataMapWriterListener.java
@@ -82,10 +82,10 @@ public class DataMapWriterListener {
     LOG.info("DataMapWriter " + writer + " added");
   }
 
-  public void onBlockStart(String blockId) {
+  public void onBlockStart(String blockId, String blockPath) {
     for (List<DataMapWriter> writers : registry.values()) {
       for (DataMapWriter writer : writers) {
-        writer.onBlockStart(blockId);
+        writer.onBlockStart(blockId, blockPath);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7f617221/processing/src/main/java/org/apache/carbondata/processing/store/writer/AbstractFactDataWriter.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/writer/AbstractFactDataWriter.java b/processing/src/main/java/org/apache/carbondata/processing/store/writer/AbstractFactDataWriter.java
index 1b6ba72..fd50502 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/store/writer/AbstractFactDataWriter.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/store/writer/AbstractFactDataWriter.java
@@ -269,7 +269,7 @@ public abstract class AbstractFactDataWriter implements CarbonFactDataWriter {
 
   private void notifyDataMapBlockStart() {
     if (listener != null) {
-      listener.onBlockStart(carbonDataFileName);
+      listener.onBlockStart(carbonDataFileName, constructFactFileFullPath());
     }
   }
 
@@ -280,6 +280,11 @@ public abstract class AbstractFactDataWriter implements CarbonFactDataWriter {
     blockletId = 0;
   }
 
+  private String constructFactFileFullPath() {
+    String factFilePath =
+        this.dataWriterVo.getCarbonDataDirectoryPath() + File.separator + this.carbonDataFileName;
+    return factFilePath;
+  }
   /**
    * Finish writing current file. It will flush stream, copy and rename temp file to final file
    * @param copyInCurrentThread set to false if want to do data copy in a new thread

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7f617221/processing/src/main/java/org/apache/carbondata/processing/store/writer/v3/CarbonFactDataWriterImplV3.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/writer/v3/CarbonFactDataWriterImplV3.java b/processing/src/main/java/org/apache/carbondata/processing/store/writer/v3/CarbonFactDataWriterImplV3.java
index d8ae8ff..c366f63 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/store/writer/v3/CarbonFactDataWriterImplV3.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/store/writer/v3/CarbonFactDataWriterImplV3.java
@@ -43,6 +43,7 @@ import org.apache.carbondata.processing.store.TablePage;
 import org.apache.carbondata.processing.store.writer.AbstractFactDataWriter;
 import org.apache.carbondata.processing.store.writer.CarbonDataWriterVo;
 
+
 /**
  * Below class will be used to write the data in V3 format
  * <Column1 Data ChunkV3><Column1<Page1><Page2><Page3><Page4>>
@@ -157,6 +158,8 @@ public class CarbonFactDataWriterImplV3 extends AbstractFactDataWriter {
     }
   }
 
+
+
   /**
    * Write the collect blocklet data (blockletDataHolder) to file
    */


[45/49] carbondata git commit: [CARBONDATA-1520] [PreAgg] Support pre-aggregate table load

Posted by ra...@apache.org.
[CARBONDATA-1520] [PreAgg] Support pre-aggregate table load

This closes #1446


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

Branch: refs/heads/fgdatamap
Commit: cc0e6f1e77b39d712de0e6101b2d24e57c5b47cb
Parents: f7f516e
Author: kunal642 <ku...@gmail.com>
Authored: Thu Oct 26 17:09:54 2017 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Tue Nov 14 00:48:16 2017 +0530

----------------------------------------------------------------------
 .../core/constants/CarbonCommonConstants.java   |   6 +
 .../core/indexstore/UnsafeMemoryDMStore.java    |  14 +-
 .../blockletindex/BlockletDataMap.java          |  41 +++--
 .../core/indexstore/row/DataMapRow.java         |   6 +-
 .../core/indexstore/row/DataMapRowImpl.java     |   4 +-
 .../core/indexstore/row/UnsafeDataMapRow.java   |   8 +-
 .../core/indexstore/schema/CarbonRowSchema.java | 124 +++++++++++++
 .../core/indexstore/schema/DataMapSchema.java   | 124 -------------
 .../metadata/converter/SchemaConverter.java     |   2 +-
 .../core/metadata/schema/table/CarbonTable.java |   5 +
 .../metadata/schema/table/DataMapSchema.java    |   4 +-
 .../schema/table/RelationIdentifier.java        |   4 +
 .../core/metadata/schema/table/TableInfo.java   |   2 +-
 .../core/scan/executor/util/QueryUtil.java      |  44 +++--
 .../carbondata/core/util/SessionParams.java     |  13 ++
 .../core/writer/CarbonIndexFileMergeWriter.java |   2 +-
 .../carbondata/events/OperationListenerBus.java |   3 +-
 .../hadoop/api/CarbonTableInputFormat.java      |  19 +-
 .../src/test/resources/sample.csv               |   2 +
 .../dataload/TestLoadDataGeneral.scala          |  10 +-
 .../TestLoadDataWithAutoLoadMerge.scala         |   2 +-
 .../TestLoadDataWithYarnLocalDirs.scala         |   2 +-
 .../preaggregate/TestPreAggregateLoad.scala     | 172 +++++++++++++++++++
 .../deleteTable/TestDeleteTableNewDDL.scala     |   2 +
 .../carbondata/spark/rdd/CarbonScanRDD.scala    |  13 +-
 .../spark/rdd/CarbonDataRDDFactory.scala        |   8 +-
 .../spark/sql/CarbonDictionaryDecoder.scala     |  24 ++-
 .../command/management/LoadTableCommand.scala   |   4 +
 .../CreatePreAggregateTableCommand.scala        |   5 +
 .../DropPreAggregateTablePostListener.scala     |  49 ------
 .../preaaggregate/PreAggregateListeners.scala   |  81 +++++++++
 .../preaaggregate/PreAggregateUtil.scala        |  23 ++-
 .../spark/sql/hive/CarbonAnalysisRules.scala    |  57 +++++-
 .../spark/sql/hive/CarbonSessionState.scala     |   7 +-
 .../execution/command/CarbonHiveCommands.scala  |   3 +
 .../impl/DictionaryFieldConverterImpl.java      |   8 +-
 .../converter/impl/FieldEncoderFactory.java     |  38 +++-
 .../loading/model/CarbonLoadModel.java          |  13 ++
 38 files changed, 685 insertions(+), 263 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
index 014478f..e27e5bd 100644
--- a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
+++ b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
@@ -57,6 +57,12 @@ public final class CarbonCommonConstants {
   public static final String CARBON_INPUT_SEGMENTS = "carbon.input.segments.";
 
   /**
+   * Fetch and validate the segments.
+   * Used for aggregate table load as segment validation is not required.
+   */
+  public static final String VALIDATE_CARBON_INPUT_SEGMENTS = "validate.carbon.input.segments.";
+
+  /**
    * location of the carbon member, hierarchy and fact files
    */
   @CarbonProperty

http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/core/src/main/java/org/apache/carbondata/core/indexstore/UnsafeMemoryDMStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/UnsafeMemoryDMStore.java b/core/src/main/java/org/apache/carbondata/core/indexstore/UnsafeMemoryDMStore.java
index bf1678a..450796a 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/UnsafeMemoryDMStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/UnsafeMemoryDMStore.java
@@ -18,7 +18,7 @@ package org.apache.carbondata.core.indexstore;
 
 import org.apache.carbondata.core.indexstore.row.DataMapRow;
 import org.apache.carbondata.core.indexstore.row.UnsafeDataMapRow;
-import org.apache.carbondata.core.indexstore.schema.DataMapSchema;
+import org.apache.carbondata.core.indexstore.schema.CarbonRowSchema;
 import org.apache.carbondata.core.memory.MemoryBlock;
 import org.apache.carbondata.core.memory.MemoryException;
 import org.apache.carbondata.core.memory.UnsafeMemoryManager;
@@ -44,7 +44,7 @@ public class UnsafeMemoryDMStore {
 
   private boolean isMemoryFreed;
 
-  private DataMapSchema[] schema;
+  private CarbonRowSchema[] schema;
 
   private int[] pointers;
 
@@ -52,7 +52,7 @@ public class UnsafeMemoryDMStore {
 
   private final long taskId = ThreadLocalTaskInfo.getCarbonTaskInfo().getTaskId();
 
-  public UnsafeMemoryDMStore(DataMapSchema[] schema) throws MemoryException {
+  public UnsafeMemoryDMStore(CarbonRowSchema[] schema) throws MemoryException {
     this.schema = schema;
     this.allocatedSize = capacity;
     this.memoryBlock = UnsafeMemoryManager.allocateMemoryWithRetry(taskId, allocatedSize);
@@ -101,7 +101,7 @@ public class UnsafeMemoryDMStore {
     pointers[rowCount++] = pointer;
   }
 
-  private void addToUnsafe(DataMapSchema schema, DataMapRow row, int index) {
+  private void addToUnsafe(CarbonRowSchema schema, DataMapRow row, int index) {
     switch (schema.getSchemaType()) {
       case FIXED:
         DataType dataType = schema.getDataType();
@@ -154,8 +154,8 @@ public class UnsafeMemoryDMStore {
         runningLength += data.length;
         break;
       case STRUCT:
-        DataMapSchema[] childSchemas =
-            ((DataMapSchema.StructDataMapSchema) schema).getChildSchemas();
+        CarbonRowSchema[] childSchemas =
+            ((CarbonRowSchema.StructCarbonRowSchema) schema).getChildSchemas();
         DataMapRow struct = row.getRow(index);
         for (int i = 0; i < childSchemas.length; i++) {
           addToUnsafe(childSchemas[i], struct, i);
@@ -200,7 +200,7 @@ public class UnsafeMemoryDMStore {
     return runningLength;
   }
 
-  public DataMapSchema[] getSchema() {
+  public CarbonRowSchema[] getSchema() {
     return schema;
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/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 7829034..43e265d 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
@@ -45,7 +45,7 @@ import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
 import org.apache.carbondata.core.indexstore.UnsafeMemoryDMStore;
 import org.apache.carbondata.core.indexstore.row.DataMapRow;
 import org.apache.carbondata.core.indexstore.row.DataMapRowImpl;
-import org.apache.carbondata.core.indexstore.schema.DataMapSchema;
+import org.apache.carbondata.core.indexstore.schema.CarbonRowSchema;
 import org.apache.carbondata.core.keygenerator.KeyGenException;
 import org.apache.carbondata.core.memory.MemoryException;
 import org.apache.carbondata.core.metadata.blocklet.BlockletInfo;
@@ -132,7 +132,7 @@ public class BlockletDataMap implements DataMap, Cacheable {
       String filePath) {
     int[] minMaxLen = segmentProperties.getColumnsValueSize();
     List<BlockletInfo> blockletList = fileFooter.getBlockletList();
-    DataMapSchema[] schema = unsafeMemoryDMStore.getSchema();
+    CarbonRowSchema[] schema = unsafeMemoryDMStore.getSchema();
     for (int index = 0; index < blockletList.size(); index++) {
       DataMapRow row = new DataMapRowImpl(schema);
       int ordinal = 0;
@@ -256,9 +256,10 @@ public class BlockletDataMap implements DataMap, Cacheable {
     return updatedValues;
   }
 
-  private DataMapRow addMinMax(int[] minMaxLen, DataMapSchema dataMapSchema, byte[][] minValues) {
-    DataMapSchema[] minSchemas =
-        ((DataMapSchema.StructDataMapSchema) dataMapSchema).getChildSchemas();
+  private DataMapRow addMinMax(int[] minMaxLen, CarbonRowSchema carbonRowSchema,
+      byte[][] minValues) {
+    CarbonRowSchema[] minSchemas =
+        ((CarbonRowSchema.StructCarbonRowSchema) carbonRowSchema).getChildSchemas();
     DataMapRow minRow = new DataMapRowImpl(minSchemas);
     int minOrdinal = 0;
     // min value adding
@@ -269,46 +270,48 @@ public class BlockletDataMap implements DataMap, Cacheable {
   }
 
   private void createSchema(SegmentProperties segmentProperties) throws MemoryException {
-    List<DataMapSchema> indexSchemas = new ArrayList<>();
+    List<CarbonRowSchema> indexSchemas = new ArrayList<>();
 
     // Index key
-    indexSchemas.add(new DataMapSchema.VariableDataMapSchema(DataTypes.BYTE_ARRAY));
+    indexSchemas.add(new CarbonRowSchema.VariableCarbonRowSchema(DataTypes.BYTE_ARRAY));
     int[] minMaxLen = segmentProperties.getColumnsValueSize();
     // do it 2 times, one for min and one for max.
     for (int k = 0; k < 2; k++) {
-      DataMapSchema[] mapSchemas = new DataMapSchema[minMaxLen.length];
+      CarbonRowSchema[] mapSchemas = new CarbonRowSchema[minMaxLen.length];
       for (int i = 0; i < minMaxLen.length; i++) {
         if (minMaxLen[i] <= 0) {
-          mapSchemas[i] = new DataMapSchema.VariableDataMapSchema(DataTypes.BYTE_ARRAY);
+          mapSchemas[i] = new CarbonRowSchema.VariableCarbonRowSchema(DataTypes.BYTE_ARRAY);
         } else {
-          mapSchemas[i] = new DataMapSchema.FixedDataMapSchema(DataTypes.BYTE_ARRAY, minMaxLen[i]);
+          mapSchemas[i] =
+              new CarbonRowSchema.FixedCarbonRowSchema(DataTypes.BYTE_ARRAY, minMaxLen[i]);
         }
       }
-      DataMapSchema mapSchema = new DataMapSchema.StructDataMapSchema(
-          DataTypes.createDefaultStructType(), mapSchemas);
+      CarbonRowSchema mapSchema =
+          new CarbonRowSchema.StructCarbonRowSchema(DataTypes.createDefaultStructType(),
+              mapSchemas);
       indexSchemas.add(mapSchema);
     }
 
     // for number of rows.
-    indexSchemas.add(new DataMapSchema.FixedDataMapSchema(DataTypes.INT));
+    indexSchemas.add(new CarbonRowSchema.FixedCarbonRowSchema(DataTypes.INT));
 
     // for table block path
-    indexSchemas.add(new DataMapSchema.VariableDataMapSchema(DataTypes.BYTE_ARRAY));
+    indexSchemas.add(new CarbonRowSchema.VariableCarbonRowSchema(DataTypes.BYTE_ARRAY));
 
     // for number of pages.
-    indexSchemas.add(new DataMapSchema.FixedDataMapSchema(DataTypes.SHORT));
+    indexSchemas.add(new CarbonRowSchema.FixedCarbonRowSchema(DataTypes.SHORT));
 
     // for version number.
-    indexSchemas.add(new DataMapSchema.FixedDataMapSchema(DataTypes.SHORT));
+    indexSchemas.add(new CarbonRowSchema.FixedCarbonRowSchema(DataTypes.SHORT));
 
     // for schema updated time.
-    indexSchemas.add(new DataMapSchema.FixedDataMapSchema(DataTypes.LONG));
+    indexSchemas.add(new CarbonRowSchema.FixedCarbonRowSchema(DataTypes.LONG));
 
     //for blocklet info
-    indexSchemas.add(new DataMapSchema.VariableDataMapSchema(DataTypes.BYTE_ARRAY));
+    indexSchemas.add(new CarbonRowSchema.VariableCarbonRowSchema(DataTypes.BYTE_ARRAY));
 
     unsafeMemoryDMStore =
-        new UnsafeMemoryDMStore(indexSchemas.toArray(new DataMapSchema[indexSchemas.size()]));
+        new UnsafeMemoryDMStore(indexSchemas.toArray(new CarbonRowSchema[indexSchemas.size()]));
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/core/src/main/java/org/apache/carbondata/core/indexstore/row/DataMapRow.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/row/DataMapRow.java b/core/src/main/java/org/apache/carbondata/core/indexstore/row/DataMapRow.java
index 631e0ad..b764bdf 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/row/DataMapRow.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/row/DataMapRow.java
@@ -16,7 +16,7 @@
  */
 package org.apache.carbondata.core.indexstore.row;
 
-import org.apache.carbondata.core.indexstore.schema.DataMapSchema;
+import org.apache.carbondata.core.indexstore.schema.CarbonRowSchema;
 
 /**
  * It is just a normal row to store data. Implementation classes could be safe and unsafe.
@@ -24,9 +24,9 @@ import org.apache.carbondata.core.indexstore.schema.DataMapSchema;
  */
 public abstract class DataMapRow {
 
-  protected DataMapSchema[] schemas;
+  protected CarbonRowSchema[] schemas;
 
-  public DataMapRow(DataMapSchema[] schemas) {
+  public DataMapRow(CarbonRowSchema[] schemas) {
     this.schemas = schemas;
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/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 032b29e..0bb4a5c 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
@@ -16,7 +16,7 @@
  */
 package org.apache.carbondata.core.indexstore.row;
 
-import org.apache.carbondata.core.indexstore.schema.DataMapSchema;
+import org.apache.carbondata.core.indexstore.schema.CarbonRowSchema;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
 
 /**
@@ -26,7 +26,7 @@ public class DataMapRowImpl extends DataMapRow {
 
   private Object[] data;
 
-  public DataMapRowImpl(DataMapSchema[] schemas) {
+  public DataMapRowImpl(CarbonRowSchema[] schemas) {
     super(schemas);
     this.data = new Object[schemas.length];
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/core/src/main/java/org/apache/carbondata/core/indexstore/row/UnsafeDataMapRow.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/row/UnsafeDataMapRow.java b/core/src/main/java/org/apache/carbondata/core/indexstore/row/UnsafeDataMapRow.java
index 2c76990..932865d 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/row/UnsafeDataMapRow.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/row/UnsafeDataMapRow.java
@@ -17,7 +17,7 @@
 
 package org.apache.carbondata.core.indexstore.row;
 
-import org.apache.carbondata.core.indexstore.schema.DataMapSchema;
+import org.apache.carbondata.core.indexstore.schema.CarbonRowSchema;
 import org.apache.carbondata.core.memory.MemoryBlock;
 
 import static org.apache.carbondata.core.memory.CarbonUnsafe.BYTE_ARRAY_OFFSET;
@@ -32,7 +32,7 @@ public class UnsafeDataMapRow extends DataMapRow {
 
   private int pointer;
 
-  public UnsafeDataMapRow(DataMapSchema[] schemas, MemoryBlock block, int pointer) {
+  public UnsafeDataMapRow(CarbonRowSchema[] schemas, MemoryBlock block, int pointer) {
     super(schemas);
     this.block = block;
     this.pointer = pointer;
@@ -84,8 +84,8 @@ public class UnsafeDataMapRow extends DataMapRow {
   }
 
   @Override public DataMapRow getRow(int ordinal) {
-    DataMapSchema[] childSchemas =
-        ((DataMapSchema.StructDataMapSchema) schemas[ordinal]).getChildSchemas();
+    CarbonRowSchema[] childSchemas =
+        ((CarbonRowSchema.StructCarbonRowSchema) schemas[ordinal]).getChildSchemas();
     return new UnsafeDataMapRow(childSchemas, block, pointer + getPosition(ordinal));
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/core/src/main/java/org/apache/carbondata/core/indexstore/schema/CarbonRowSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/schema/CarbonRowSchema.java b/core/src/main/java/org/apache/carbondata/core/indexstore/schema/CarbonRowSchema.java
new file mode 100644
index 0000000..813be4a
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/schema/CarbonRowSchema.java
@@ -0,0 +1,124 @@
+/*
+ * 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.indexstore.schema;
+
+import org.apache.carbondata.core.metadata.datatype.DataType;
+
+/**
+ * It just have 2 types right now, either fixed or variable.
+ */
+public abstract class CarbonRowSchema {
+
+  protected DataType dataType;
+
+  public CarbonRowSchema(DataType dataType) {
+    this.dataType = dataType;
+  }
+
+  /**
+   * Either fixed or variable length.
+   *
+   * @return
+   */
+  public DataType getDataType() {
+    return dataType;
+  }
+
+  /**
+   * Gives length in case of fixed schema other wise returns length
+   *
+   * @return
+   */
+  public abstract int getLength();
+
+  /**
+   * schema type
+   * @return
+   */
+  public abstract DataMapSchemaType getSchemaType();
+
+  /*
+ * It has always fixed length, length cannot be updated later.
+ * Usage examples : all primitive types like short, int etc
+ */
+  public static class FixedCarbonRowSchema extends CarbonRowSchema {
+
+    private int length;
+
+    public FixedCarbonRowSchema(DataType dataType) {
+      super(dataType);
+    }
+
+    public FixedCarbonRowSchema(DataType dataType, int length) {
+      super(dataType);
+      this.length = length;
+    }
+
+    @Override public int getLength() {
+      if (length == 0) {
+        return dataType.getSizeInBytes();
+      } else {
+        return length;
+      }
+    }
+
+    @Override public DataMapSchemaType getSchemaType() {
+      return DataMapSchemaType.FIXED;
+    }
+  }
+
+  public static class VariableCarbonRowSchema extends CarbonRowSchema {
+
+    public VariableCarbonRowSchema(DataType dataType) {
+      super(dataType);
+    }
+
+    @Override public int getLength() {
+      return dataType.getSizeInBytes();
+    }
+
+    @Override public DataMapSchemaType getSchemaType() {
+      return DataMapSchemaType.VARIABLE;
+    }
+  }
+
+  public static class StructCarbonRowSchema extends CarbonRowSchema {
+
+    private CarbonRowSchema[] childSchemas;
+
+    public StructCarbonRowSchema(DataType dataType, CarbonRowSchema[] childSchemas) {
+      super(dataType);
+      this.childSchemas = childSchemas;
+    }
+
+    @Override public int getLength() {
+      return dataType.getSizeInBytes();
+    }
+
+    public CarbonRowSchema[] getChildSchemas() {
+      return childSchemas;
+    }
+
+    @Override public DataMapSchemaType getSchemaType() {
+      return DataMapSchemaType.STRUCT;
+    }
+  }
+
+  public enum DataMapSchemaType {
+    FIXED, VARIABLE, STRUCT
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/core/src/main/java/org/apache/carbondata/core/indexstore/schema/DataMapSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/schema/DataMapSchema.java b/core/src/main/java/org/apache/carbondata/core/indexstore/schema/DataMapSchema.java
deleted file mode 100644
index 80c68ac..0000000
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/schema/DataMapSchema.java
+++ /dev/null
@@ -1,124 +0,0 @@
-/*
- * 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.indexstore.schema;
-
-import org.apache.carbondata.core.metadata.datatype.DataType;
-
-/**
- * It just have 2 types right now, either fixed or variable.
- */
-public abstract class DataMapSchema {
-
-  protected DataType dataType;
-
-  public DataMapSchema(DataType dataType) {
-    this.dataType = dataType;
-  }
-
-  /**
-   * Either fixed or variable length.
-   *
-   * @return
-   */
-  public DataType getDataType() {
-    return dataType;
-  }
-
-  /**
-   * Gives length in case of fixed schema other wise returns length
-   *
-   * @return
-   */
-  public abstract int getLength();
-
-  /**
-   * schema type
-   * @return
-   */
-  public abstract DataMapSchemaType getSchemaType();
-
-  /*
- * It has always fixed length, length cannot be updated later.
- * Usage examples : all primitive types like short, int etc
- */
-  public static class FixedDataMapSchema extends DataMapSchema {
-
-    private int length;
-
-    public FixedDataMapSchema(DataType dataType) {
-      super(dataType);
-    }
-
-    public FixedDataMapSchema(DataType dataType, int length) {
-      super(dataType);
-      this.length = length;
-    }
-
-    @Override public int getLength() {
-      if (length == 0) {
-        return dataType.getSizeInBytes();
-      } else {
-        return length;
-      }
-    }
-
-    @Override public DataMapSchemaType getSchemaType() {
-      return DataMapSchemaType.FIXED;
-    }
-  }
-
-  public static class VariableDataMapSchema extends DataMapSchema {
-
-    public VariableDataMapSchema(DataType dataType) {
-      super(dataType);
-    }
-
-    @Override public int getLength() {
-      return dataType.getSizeInBytes();
-    }
-
-    @Override public DataMapSchemaType getSchemaType() {
-      return DataMapSchemaType.VARIABLE;
-    }
-  }
-
-  public static class StructDataMapSchema extends DataMapSchema {
-
-    private DataMapSchema[] childSchemas;
-
-    public StructDataMapSchema(DataType dataType, DataMapSchema[] childSchemas) {
-      super(dataType);
-      this.childSchemas = childSchemas;
-    }
-
-    @Override public int getLength() {
-      return dataType.getSizeInBytes();
-    }
-
-    public DataMapSchema[] getChildSchemas() {
-      return childSchemas;
-    }
-
-    @Override public DataMapSchemaType getSchemaType() {
-      return DataMapSchemaType.STRUCT;
-    }
-  }
-
-  public enum DataMapSchemaType {
-    FIXED, VARIABLE, STRUCT
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/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 bfbb6f7..af86253 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
@@ -106,7 +106,7 @@ public interface SchemaConverter {
    * method to convert thrift datamap schema object to wrapper
    * data map object
    * @param thriftchildSchema
-   * @return DataMapSchema
+   * @return CarbonRowSchema
    */
   DataMapSchema fromExternalToWrapperDataMapSchema(
       org.apache.carbondata.format.DataMapSchema thriftchildSchema);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/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 e63f4e3..4a6fb8b 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
@@ -702,4 +702,9 @@ public class CarbonTable implements Serializable {
     this.dimensionOrdinalMax = dimensionOrdinalMax;
   }
 
+  public boolean isPreAggregateTable() {
+    return tableInfo.getParentRelationIdentifiers() != null && !tableInfo
+        .getParentRelationIdentifiers().isEmpty();
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/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
index 5f92ec8..9c71e37 100644
--- 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
@@ -28,6 +28,8 @@ import java.util.Map;
  */
 public class DataMapSchema implements Serializable, Writable {
 
+  private static final long serialVersionUID = 6577149126264181553L;
+
   private String className;
 
   private RelationIdentifier relationIdentifier;
@@ -100,7 +102,7 @@ public class DataMapSchema implements Serializable, Writable {
     this.className = in.readUTF();
     boolean isRelationIdnentifierExists = in.readBoolean();
     if (isRelationIdnentifierExists) {
-      this.relationIdentifier = new RelationIdentifier(null, null, null);
+      this.relationIdentifier = new RelationIdentifier();
       this.relationIdentifier.readFields(in);
     }
     boolean isChildSchemaExists = in.readBoolean();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/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
index 9a70b8b..2a2d937 100644
--- 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
@@ -38,6 +38,10 @@ public class RelationIdentifier implements Serializable, Writable {
     this.tableId = tableId;
   }
 
+  public RelationIdentifier() {
+    this(null, null, null);
+  }
+
   public String getDatabaseName() {
     return databaseName;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/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 3acd6d6..44d8126 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
@@ -261,7 +261,7 @@ public class TableInfo implements Serializable, Writable {
     out.writeUTF(metaDataFilepath);
     out.writeUTF(storePath);
     boolean isChildSchemaExists =
-        null != dataMapSchemaList && dataMapSchemaList.size() > 0 ? true : false;
+        null != dataMapSchemaList && dataMapSchemaList.size() > 0;
     out.writeBoolean(isChildSchemaExists);
     if (isChildSchemaExists) {
       out.writeShort(dataMapSchemaList.size());

http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/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 b090e59..4d13462 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
@@ -42,6 +42,7 @@ import org.apache.carbondata.core.keygenerator.KeyGenerator;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.CarbonMetadata;
 import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
+import org.apache.carbondata.core.metadata.ColumnIdentifier;
 import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
@@ -353,14 +354,15 @@ public class QueryUtil {
     CacheProvider cacheProvider = CacheProvider.getInstance();
     Cache<DictionaryColumnUniqueIdentifier, Dictionary> forwardDictionaryCache = cacheProvider
         .createCache(CacheType.FORWARD_DICTIONARY, absoluteTableIdentifier.getStorePath());
-
     List<Dictionary> columnDictionaryList =
         forwardDictionaryCache.getAll(dictionaryColumnUniqueIdentifiers);
     Map<String, Dictionary> columnDictionaryMap = new HashMap<>(columnDictionaryList.size());
     for (int i = 0; i < dictionaryColumnUniqueIdentifiers.size(); i++) {
       // TODO: null check for column dictionary, if cache size is less it
       // might return null here, in that case throw exception
-      columnDictionaryMap.put(dictionaryColumnIdList.get(i), columnDictionaryList.get(i));
+      columnDictionaryMap
+          .put(dictionaryColumnUniqueIdentifiers.get(i).getColumnIdentifier().getColumnId(),
+              columnDictionaryList.get(i));
     }
     return columnDictionaryMap;
   }
@@ -376,27 +378,47 @@ public class QueryUtil {
       List<String> dictionaryColumnIdList, CarbonTableIdentifier carbonTableIdentifier,
       TableProvider tableProvider) throws IOException {
     CarbonTable carbonTable = tableProvider.getCarbonTable(carbonTableIdentifier);
-    CarbonTablePath carbonTablePath =
-        CarbonStorePath.getCarbonTablePath(carbonTable.getStorePath(), carbonTableIdentifier);
     List<DictionaryColumnUniqueIdentifier> dictionaryColumnUniqueIdentifiers =
         new ArrayList<>(dictionaryColumnIdList.size());
     for (String columnId : dictionaryColumnIdList) {
       CarbonDimension dimension = CarbonMetadata.getInstance()
           .getCarbonDimensionBasedOnColIdentifier(carbonTable, columnId);
       if (dimension != null) {
+        CarbonTableIdentifier newCarbonTableIdentifier;
+        ColumnIdentifier columnIdentifier;
+        if (null != dimension.getColumnSchema().getParentColumnTableRelations() && !dimension
+            .getColumnSchema().getParentColumnTableRelations().isEmpty()) {
+          newCarbonTableIdentifier = getTableIdentifierForColumn(dimension);
+          columnIdentifier = new ColumnIdentifier(
+              dimension.getColumnSchema().getParentColumnTableRelations().get(0).getColumnId(),
+              dimension.getColumnProperties(), dimension.getDataType());
+        } else {
+          newCarbonTableIdentifier = carbonTableIdentifier;
+          columnIdentifier = dimension.getColumnIdentifier();
+        }
+        CarbonTablePath newCarbonTablePath = CarbonStorePath
+            .getCarbonTablePath(carbonTable.getStorePath(), newCarbonTableIdentifier);
+
         dictionaryColumnUniqueIdentifiers.add(
-            new DictionaryColumnUniqueIdentifier(
-                carbonTableIdentifier,
-                dimension.getColumnIdentifier(),
-                dimension.getDataType(),
-                carbonTablePath
-            )
-        );
+            new DictionaryColumnUniqueIdentifier(newCarbonTableIdentifier, columnIdentifier,
+                dimension.getDataType(), newCarbonTablePath));
       }
     }
     return dictionaryColumnUniqueIdentifiers;
   }
 
+  public static CarbonTableIdentifier getTableIdentifierForColumn(CarbonDimension carbonDimension) {
+    String parentTableName =
+        carbonDimension.getColumnSchema().getParentColumnTableRelations().get(0)
+            .getRelationIdentifier().getTableName();
+    String parentDatabaseName =
+        carbonDimension.getColumnSchema().getParentColumnTableRelations().get(0)
+            .getRelationIdentifier().getDatabaseName();
+    String parentTableId = carbonDimension.getColumnSchema().getParentColumnTableRelations().get(0)
+        .getRelationIdentifier().getTableId();
+    return new CarbonTableIdentifier(parentDatabaseName, parentTableName, parentTableId);
+  }
+
   /**
    * Below method will used to get the method will be used to get the measure
    * block indexes to be read from the file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/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 1a91272..5dda9e4 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
@@ -66,6 +66,13 @@ public class SessionParams implements Serializable {
     return sProps.get(key);
   }
 
+  public String getProperty(String key, String defaultValue) {
+    if (!sProps.containsKey(key)) {
+      return defaultValue;
+    }
+    return sProps.get(key);
+  }
+
   /**
    * This method will be used to add a new property
    *
@@ -172,6 +179,8 @@ public class SessionParams implements Serializable {
           if (!isValid) {
             throw new InvalidConfigurationException("Invalid CARBON_INPUT_SEGMENT_IDs");
           }
+        } else if (key.startsWith(CarbonCommonConstants.VALIDATE_CARBON_INPUT_SEGMENTS)) {
+          isValid = true;
         } else {
           throw new InvalidConfigurationException(
               "The key " + key + " not supported for dynamic configuration.");
@@ -180,6 +189,10 @@ public class SessionParams implements Serializable {
     return isValid;
   }
 
+  public void removeProperty(String property) {
+    sProps.remove(property);
+  }
+
   /**
    * clear the set properties
    */

http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/core/src/main/java/org/apache/carbondata/core/writer/CarbonIndexFileMergeWriter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/writer/CarbonIndexFileMergeWriter.java b/core/src/main/java/org/apache/carbondata/core/writer/CarbonIndexFileMergeWriter.java
index 8d9bddc..e19ab24 100644
--- a/core/src/main/java/org/apache/carbondata/core/writer/CarbonIndexFileMergeWriter.java
+++ b/core/src/main/java/org/apache/carbondata/core/writer/CarbonIndexFileMergeWriter.java
@@ -49,7 +49,7 @@ public class CarbonIndexFileMergeWriter {
         return file.getName().endsWith(CarbonTablePath.INDEX_FILE_EXT);
       }
     });
-    if (indexFiles.length > 0) {
+    if (indexFiles != null && indexFiles.length > 0) {
       SegmentIndexFileStore fileStore = new SegmentIndexFileStore();
       fileStore.readAllIIndexOfSegment(segmentPath);
       openThriftWriter(segmentPath + "/" +

http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/core/src/main/java/org/apache/carbondata/events/OperationListenerBus.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/events/OperationListenerBus.java b/core/src/main/java/org/apache/carbondata/events/OperationListenerBus.java
index 01ecb04..321ddd5 100644
--- a/core/src/main/java/org/apache/carbondata/events/OperationListenerBus.java
+++ b/core/src/main/java/org/apache/carbondata/events/OperationListenerBus.java
@@ -53,7 +53,7 @@ public class OperationListenerBus {
    * @param eventClass
    * @param operationEventListener
    */
-  public void addListener(Class<? extends Event> eventClass,
+  public OperationListenerBus addListener(Class<? extends Event> eventClass,
       OperationEventListener operationEventListener) {
 
     String eventType = eventClass.getName();
@@ -63,6 +63,7 @@ public class OperationListenerBus {
       eventMap.put(eventType, operationEventListeners);
     }
     operationEventListeners.add(operationEventListener);
+    return INSTANCE;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
index 57359fc..92ef6da 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
@@ -263,6 +263,14 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
     configuration.set(CarbonTableInputFormat.VALIDATE_INPUT_SEGMENT_IDs, validate.toString());
   }
 
+  public static void setAggeragateTableSegments(Configuration configuration, String segments) {
+    configuration.set(CarbonCommonConstants.CARBON_INPUT_SEGMENTS, segments);
+  }
+
+  private static String getAggeragateTableSegments(Configuration configuration) {
+    return configuration.get(CarbonCommonConstants.CARBON_INPUT_SEGMENTS);
+  }
+
   /**
    * get list of segment to access
    */
@@ -300,6 +308,11 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
   @Override public List<InputSplit> getSplits(JobContext job) throws IOException {
     AbsoluteTableIdentifier identifier = getAbsoluteTableIdentifier(job.getConfiguration());
     SegmentUpdateStatusManager updateStatusManager = new SegmentUpdateStatusManager(identifier);
+    CarbonTable carbonTable = getOrCreateCarbonTable(job.getConfiguration());
+    if (null == carbonTable) {
+      throw new IOException("Missing/Corrupt schema file for table.");
+    }
+    String aggregateTableSegments = getAggeragateTableSegments(job.getConfiguration());
     TableDataMap blockletMap =
         DataMapStoreManager.getInstance().getDataMap(identifier, BlockletDataMap.NAME,
             BlockletDataMapFactory.class.getName());
@@ -352,6 +365,8 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
       if (invalidSegments.size() > 0) {
         blockletMap.clear(invalidSegments);
       }
+    } else {
+      filteredSegmentToAccess = Arrays.asList(aggregateTableSegments.split(","));
     }
 
     // Clean the updated segments from memory if the update happens on segments
@@ -376,12 +391,8 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
 
     // process and resolve the expression
     Expression filter = getFilterPredicates(job.getConfiguration());
-    CarbonTable carbonTable = getOrCreateCarbonTable(job.getConfiguration());
     TableProvider tableProvider = new SingleTableProvider(carbonTable);
     // this will be null in case of corrupt schema file.
-    if (null == carbonTable) {
-      throw new IOException("Missing/Corrupt schema file for table.");
-    }
     PartitionInfo partitionInfo = carbonTable.getPartitionInfo(carbonTable.getFactTableName());
     CarbonInputFormatUtil.processFilterExpression(filter, carbonTable, null, null);
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/integration/spark-common-test/src/test/resources/sample.csv
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/resources/sample.csv b/integration/spark-common-test/src/test/resources/sample.csv
index 7c57de7..06985e8 100644
--- a/integration/spark-common-test/src/test/resources/sample.csv
+++ b/integration/spark-common-test/src/test/resources/sample.csv
@@ -3,3 +3,5 @@ id,name,city,age
 2,eason,shenzhen,27
 3,jarry,wuhan,35
 3,jarry,Bangalore,35
+4,kunal,Delhi,26
+4,vishal,Bangalore,29

http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
index b90a5ea..23d1292 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
@@ -65,7 +65,7 @@ class TestLoadDataGeneral extends QueryTest with BeforeAndAfterAll {
     sql(s"LOAD DATA LOCAL INPATH '$testData' into table loadtest")
     checkAnswer(
       sql("SELECT COUNT(*) FROM loadtest"),
-      Seq(Row(4))
+      Seq(Row(6))
     )
   }
 
@@ -74,7 +74,7 @@ class TestLoadDataGeneral extends QueryTest with BeforeAndAfterAll {
     sql(s"LOAD DATA LOCAL INPATH '$testData' into table loadtest")
     checkAnswer(
       sql("SELECT COUNT(*) FROM loadtest"),
-      Seq(Row(8))
+      Seq(Row(10))
     )
   }
 
@@ -83,7 +83,7 @@ class TestLoadDataGeneral extends QueryTest with BeforeAndAfterAll {
     sql(s"LOAD DATA LOCAL INPATH '$testData' into table loadtest")
     checkAnswer(
       sql("SELECT COUNT(*) FROM loadtest"),
-      Seq(Row(12))
+      Seq(Row(14))
     )
   }
 
@@ -92,7 +92,7 @@ class TestLoadDataGeneral extends QueryTest with BeforeAndAfterAll {
     sql(s"LOAD DATA LOCAL INPATH '$testData' into table loadtest")
     checkAnswer(
       sql("SELECT COUNT(*) FROM loadtest"),
-      Seq(Row(16))
+      Seq(Row(18))
     )
   }
 
@@ -101,7 +101,7 @@ class TestLoadDataGeneral extends QueryTest with BeforeAndAfterAll {
     sql(s"LOAD DATA LOCAL INPATH '$testData' into table loadtest options ('delimiter'='\\017')")
     checkAnswer(
       sql("SELECT COUNT(*) FROM loadtest"),
-      Seq(Row(20))
+      Seq(Row(22))
     )
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataWithAutoLoadMerge.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataWithAutoLoadMerge.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataWithAutoLoadMerge.scala
index 5211e9e..51e84d4 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataWithAutoLoadMerge.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataWithAutoLoadMerge.scala
@@ -44,7 +44,7 @@ class TestLoadDataWithAutoLoadMerge extends QueryTest with BeforeAndAfterAll {
     sql(s"LOAD DATA LOCAL INPATH '$testData' into table automerge")
     checkAnswer(
       sql("SELECT COUNT(*) FROM automerge"),
-      Seq(Row(4))
+      Seq(Row(6))
     )
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataWithYarnLocalDirs.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataWithYarnLocalDirs.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataWithYarnLocalDirs.scala
index e92a7fd..ff415ae 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataWithYarnLocalDirs.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataWithYarnLocalDirs.scala
@@ -87,7 +87,7 @@ class TestLoadDataWithYarnLocalDirs extends QueryTest with BeforeAndAfterAll {
     disableMultipleDir
 
     checkAnswer(sql("select id from carbontable_yarnLocalDirs"),
-      Seq(Row(1), Row(2), Row(3), Row(3)))
+      Seq(Row(1), Row(2), Row(3), Row(3), Row(4), Row(4)))
 
     cleanUpYarnLocalDir
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateLoad.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateLoad.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateLoad.scala
new file mode 100644
index 0000000..0c65577
--- /dev/null
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateLoad.scala
@@ -0,0 +1,172 @@
+/*
+ * 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.Row
+import org.apache.spark.sql.test.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+class TestPreAggregateLoad extends QueryTest with BeforeAndAfterAll {
+  
+  val testData = s"$resourcesPath/sample.csv"
+  
+  override def beforeAll(): Unit = {
+    sql("DROP TABLE IF EXISTS maintable")
+  }
+
+  private def createAllAggregateTables(parentTableName: String): Unit = {
+    sql(
+      s"""create table ${ parentTableName }_preagg_sum stored BY 'carbondata' tblproperties
+         |('parent'='$parentTableName') as select id,sum(age) from $parentTableName group by id"""
+        .stripMargin)
+    sql(
+      s"""create table ${ parentTableName }_preagg_avg stored BY 'carbondata' tblproperties
+         |('parent'='$parentTableName') as select id,avg(age) from $parentTableName group by id"""
+        .stripMargin)
+    sql(
+      s"""create table ${ parentTableName }_preagg_count stored BY 'carbondata' tblproperties
+         |('parent'='$parentTableName') as select id,count(age) from $parentTableName group by id"""
+        .stripMargin)
+    sql(
+      s"""create table ${ parentTableName }_preagg_min stored BY 'carbondata' tblproperties
+         |('parent'='$parentTableName') as select id,min(age) from $parentTableName group by id"""
+        .stripMargin)
+    sql(
+      s"""create table ${ parentTableName }_preagg_max stored BY 'carbondata' tblproperties
+         |('parent'='$parentTableName') as select id,max(age) from $parentTableName group by id"""
+        .stripMargin)
+  }
+
+  test("test load into main table with pre-aggregate table") {
+    sql("DROP TABLE IF EXISTS maintable")
+    sql(
+      """
+        | CREATE TABLE maintable(id int, name string, city string, age int)
+        | STORED BY 'org.apache.carbondata.format'
+      """.stripMargin)
+    createAllAggregateTables("maintable")
+    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
+    checkAnswer(sql(s"select * from maintable_preagg_sum"),
+      Seq(Row(1, 31), Row(2, 27), Row(3, 70), Row(4, 55)))
+    checkAnswer(sql(s"select * from maintable_preagg_avg"),
+      Seq(Row(1, 31, 1), Row(2, 27, 1), Row(3, 70, 2), Row(4, 55, 2)))
+    checkAnswer(sql(s"select * from maintable_preagg_count"),
+      Seq(Row(1, 1), Row(2, 1), Row(3, 2), Row(4, 2)))
+    checkAnswer(sql(s"select * from maintable_preagg_min"),
+      Seq(Row(1, 31), Row(2, 27), Row(3, 35), Row(4, 26)))
+    checkAnswer(sql(s"select * from maintable_preagg_max"),
+      Seq(Row(1, 31), Row(2, 27), Row(3, 35), Row(4, 29)))
+    sql("drop table if exists maintable")
+  }
+
+  test("test load into main table with pre-aggregate table with dictionary_include") {
+    sql("drop table if exists maintable")
+    sql(
+      """
+        | CREATE TABLE maintable(id int, name string, city string, age int)
+        | STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('dictionary_include'='id')
+      """.stripMargin)
+    createAllAggregateTables("maintable")
+    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
+    checkAnswer(sql(s"select * from maintable_preagg_sum"),
+      Seq(Row(1, 31), Row(2, 27), Row(3, 70), Row(4, 55)))
+    checkAnswer(sql(s"select * from maintable_preagg_avg"),
+      Seq(Row(1, 31, 1), Row(2, 27, 1), Row(3, 70, 2), Row(4, 55,2)))
+    checkAnswer(sql(s"select * from maintable_preagg_count"),
+      Seq(Row(1, 1), Row(2, 1), Row(3, 2), Row(4, 2)))
+    checkAnswer(sql(s"select * from maintable_preagg_min"),
+      Seq(Row(1, 31), Row(2, 27), Row(3, 35), Row(4, 26)))
+    checkAnswer(sql(s"select * from maintable_preagg_max"),
+      Seq(Row(1, 31), Row(2, 27), Row(3, 35), Row(4, 29)))
+    sql("drop table if exists maintable")
+  }
+
+  test("test load into main table with pre-aggregate table with single_pass") {
+    sql("drop table if exists maintable")
+    sql(
+      """
+        | CREATE TABLE maintable(id int, name string, city string, age int)
+        | STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('dictionary_include'='id')
+      """.stripMargin)
+    createAllAggregateTables("maintable")
+    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable options('single_pass'='true')")
+    checkAnswer(sql(s"select * from maintable_preagg_sum"),
+      Seq(Row(1, 31), Row(2, 27), Row(3, 70), Row(4, 55)))
+    checkAnswer(sql(s"select * from maintable_preagg_avg"),
+      Seq(Row(1, 31, 1), Row(2, 27, 1), Row(3, 70, 2), Row(4, 55,2)))
+    checkAnswer(sql(s"select * from maintable_preagg_count"),
+      Seq(Row(1, 1), Row(2, 1), Row(3, 2), Row(4, 2)))
+    checkAnswer(sql(s"select * from maintable_preagg_min"),
+      Seq(Row(1, 31), Row(2, 27), Row(3, 35), Row(4, 26)))
+    checkAnswer(sql(s"select * from maintable_preagg_max"),
+      Seq(Row(1, 31), Row(2, 27), Row(3, 35), Row(4, 29)))
+    sql("drop table if exists maintable")
+  }
+
+  test("test load into main table with incremental load") {
+    sql("drop table if exists maintable")
+    sql(
+      """
+        | CREATE TABLE maintable(id int, name string, city string, age int)
+        | STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('dictionary_include'='id')
+      """.stripMargin)
+    createAllAggregateTables("maintable")
+    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
+    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
+    checkAnswer(sql(s"select * from maintable_preagg_sum"),
+      Seq(Row(1, 31),
+        Row(2, 27),
+        Row(3, 70),
+        Row(4, 55),
+        Row(1, 31),
+        Row(2, 27),
+        Row(3, 70),
+        Row(4, 55)))
+    checkAnswer(sql(s"select * from maintable_preagg_avg"),
+      Seq(Row(1, 31, 1),
+        Row(2, 27, 1),
+        Row(3, 70, 2),
+        Row(4, 55, 2),
+        Row(1, 31, 1),
+        Row(2, 27, 1),
+        Row(3, 70, 2),
+        Row(4, 55, 2)))
+    checkAnswer(sql(s"select * from maintable_preagg_count"),
+      Seq(Row(1, 1), Row(2, 1), Row(3, 2), Row(4, 2), Row(1, 1), Row(2, 1), Row(3, 2), Row(4, 2)))
+    checkAnswer(sql(s"select * from maintable_preagg_min"),
+      Seq(Row(1, 31),
+        Row(2, 27),
+        Row(3, 35),
+        Row(4, 26),
+        Row(1, 31),
+        Row(2, 27),
+        Row(3, 35),
+        Row(4, 26)))
+    checkAnswer(sql(s"select * from maintable_preagg_max"),
+      Seq(Row(1, 31),
+        Row(2, 27),
+        Row(3, 35),
+        Row(4, 29),
+        Row(1, 31),
+        Row(2, 27),
+        Row(3, 35),
+        Row(4, 29)))
+    sql("drop table if exists maintable")
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/deleteTable/TestDeleteTableNewDDL.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/deleteTable/TestDeleteTableNewDDL.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/deleteTable/TestDeleteTableNewDDL.scala
index 485b94b..b82a0af 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/deleteTable/TestDeleteTableNewDDL.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/deleteTable/TestDeleteTableNewDDL.scala
@@ -182,6 +182,7 @@ class TestDeleteTableNewDDL extends QueryTest with BeforeAndAfterAll {
 
   test("drop table and create table with dictionary exclude string scenario") {
     try {
+
       sql("create database test")
       sql(
         "CREATE table test.dropTableTest3 (ID int, date String, country String, name " +
@@ -242,6 +243,7 @@ class TestDeleteTableNewDDL extends QueryTest with BeforeAndAfterAll {
     sql("drop table if exists dropTableTest4")
     sql("drop table if exists table1")
     sql("drop table if exists table2")
+    sql("drop database if exists test cascade")
   }
 
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
index 5e46417..52a31a9 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
@@ -41,7 +41,7 @@ import org.apache.carbondata.core.scan.expression.Expression
 import org.apache.carbondata.core.scan.model.QueryModel
 import org.apache.carbondata.core.stats.{QueryStatistic, QueryStatisticsConstants, QueryStatisticsRecorder}
 import org.apache.carbondata.core.statusmanager.FileFormat
-import org.apache.carbondata.core.util.{CarbonProperties, CarbonTimeStatisticsFactory, DataTypeUtil, TaskMetricsMap}
+import org.apache.carbondata.core.util._
 import org.apache.carbondata.hadoop._
 import org.apache.carbondata.hadoop.api.CarbonTableInputFormat
 import org.apache.carbondata.hadoop.streaming.{CarbonStreamInputFormat, CarbonStreamRecordReader}
@@ -357,6 +357,17 @@ class CarbonScanRDD(
       CarbonTableInputFormat
         .setSegmentsToAccess(conf, segmentNumbersFromProperty.split(",").toList.asJava)
     }
+    val carbonSessionInfo = ThreadLocalSessionInfo.getCarbonSessionInfo
+    if (carbonSessionInfo != null) {
+      CarbonTableInputFormat.setAggeragateTableSegments(conf, carbonSessionInfo.getSessionParams
+        .getProperty(CarbonCommonConstants.CARBON_INPUT_SEGMENTS +
+                     identifier.getCarbonTableIdentifier.getDatabaseName + "." +
+                     identifier.getCarbonTableIdentifier.getTableName, ""))
+      CarbonTableInputFormat.setValidateSegmentsToAccess(conf, carbonSessionInfo.getSessionParams
+          .getProperty(CarbonCommonConstants.VALIDATE_CARBON_INPUT_SEGMENTS +
+                       identifier.getCarbonTableIdentifier.getDatabaseName + "." +
+                       identifier.getCarbonTableIdentifier.getTableName, "true").toBoolean)
+    }
     format
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
index 1083669..9899be1 100644
--- a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
@@ -265,8 +265,7 @@ object CarbonDataRDDFactory {
       result: Option[DictionaryServer],
       overwriteTable: Boolean,
       dataFrame: Option[DataFrame] = None,
-      updateModel: Option[UpdateTableModel] = None
-  ): Unit = {
+      updateModel: Option[UpdateTableModel] = None): Unit = {
     val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
     val operationContext = new OperationContext
     // for handling of the segment Merging.
@@ -350,7 +349,7 @@ object CarbonDataRDDFactory {
               if (value == CarbonCommonConstants.STORE_LOADSTATUS_FAILURE) {
                 loadStatus = CarbonCommonConstants.STORE_LOADSTATUS_FAILURE
               } else if (value == CarbonCommonConstants.STORE_LOADSTATUS_PARTIAL_SUCCESS &&
-                         !loadStatus.equals(CarbonCommonConstants.STORE_LOADSTATUS_FAILURE)) {
+                  !loadStatus.equals(CarbonCommonConstants.STORE_LOADSTATUS_FAILURE)) {
                 loadStatus = CarbonCommonConstants.STORE_LOADSTATUS_PARTIAL_SUCCESS
               }
           }
@@ -464,7 +463,8 @@ object CarbonDataRDDFactory {
         throw new Exception(status(0)._2._2.errorMsg)
       }
       // if segment is empty then fail the data load
-      if (!CarbonLoaderUtil.isValidSegment(carbonLoadModel, carbonLoadModel.getSegmentId.toInt)) {
+      if (!carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable.isPreAggregateTable &&
+          !CarbonLoaderUtil.isValidSegment(carbonLoadModel, carbonLoadModel.getSegmentId.toInt)) {
         // update the load entry in table status file for changing the status to failure
         CommonUtil.updateTableStatusForFailure(carbonLoadModel)
         LOGGER.info("********starting clean up**********")

http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/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 18f76d1..2671aad 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
@@ -34,10 +34,11 @@ import org.apache.spark.sql.types._
 
 import org.apache.carbondata.core.cache.{Cache, CacheProvider, CacheType}
 import org.apache.carbondata.core.cache.dictionary.{Dictionary, DictionaryColumnUniqueIdentifier}
-import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, ColumnIdentifier}
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier, ColumnIdentifier}
 import org.apache.carbondata.core.metadata.datatype.{DataTypes => CarbonDataTypes}
 import org.apache.carbondata.core.metadata.encoder.Encoding
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension
+import org.apache.carbondata.core.scan.executor.util.QueryUtil
 import org.apache.carbondata.core.util.DataTypeUtil
 import org.apache.carbondata.core.util.path.CarbonStorePath
 import org.apache.carbondata.spark.CarbonAliasDecoderRelation
@@ -271,11 +272,24 @@ case class CarbonDictionaryDecoder(
       case (tableName, columnIdentifier, carbonDimension) =>
         if (columnIdentifier != null) {
           try {
+            val (newCarbonTableIdentifier, newColumnIdentifier) =
+              if (null != carbonDimension.getColumnSchema.getParentColumnTableRelations &&
+                  !carbonDimension
+                    .getColumnSchema.getParentColumnTableRelations.isEmpty) {
+                (QueryUtil.getTableIdentifierForColumn(carbonDimension),
+                  new ColumnIdentifier(carbonDimension.getColumnSchema
+                    .getParentColumnTableRelations.get(0).getColumnId,
+                    carbonDimension.getColumnProperties,
+                    carbonDimension.getDataType))
+              } else {
+                (atiMap(tableName).getCarbonTableIdentifier, columnIdentifier)
+              }
             val dictionaryColumnUniqueIdentifier = new DictionaryColumnUniqueIdentifier(
-              atiMap(tableName).getCarbonTableIdentifier,
-              columnIdentifier, carbonDimension.getDataType,
-              CarbonStorePath.getCarbonTablePath(atiMap(tableName)))
-            allDictIdentifiers += dictionaryColumnUniqueIdentifier;
+              newCarbonTableIdentifier,
+              newColumnIdentifier, carbonDimension.getDataType,
+              CarbonStorePath
+                .getCarbonTablePath(atiMap(tableName).getStorePath, newCarbonTableIdentifier))
+            allDictIdentifiers += dictionaryColumnUniqueIdentifier
             new ForwardDictionaryWrapper(
               storePath,
               dictionaryColumnUniqueIdentifier)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/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 222c30d..b28ec10 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
@@ -152,6 +152,10 @@ case class LoadTableCommand(
           LOGGER.audit(s"Cannot use single_pass=true for $dbName.$tableName during the first load")
           carbonLoadModel.setUseOnePass(false)
         }
+        // if table is an aggregate table then disable single pass.
+        if (carbonLoadModel.isAggLoadRequest) {
+          carbonLoadModel.setUseOnePass(false)
+        }
         // Create table and metadata folders if not exist
         val carbonTablePath = CarbonStorePath
           .getCarbonTablePath(storePath, table.getCarbonTableIdentifier)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/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 e42e933..b952285 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
@@ -115,6 +115,11 @@ case class CreatePreAggregateTableCommand(
             .buildChildSchema("", tableInfo.getDatabaseName, queryString, "AGGREGATION")
           // upadting the parent table about child table
           PreAggregateUtil.updateMainTable(parentDbName, parentTableName, childSchema, sparkSession)
+          val loadAvailable = PreAggregateUtil
+            .checkMainTableLoad(parentTable)
+          if (loadAvailable) {
+            sparkSession.sql(s"insert into ${ cm.databaseName }.${ cm.tableName } $queryString")
+          }
         } catch {
           case e: Exception =>
             val identifier: TableIdentifier = TableIdentifier(tbName, Some(dbName))

http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/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
deleted file mode 100644
index 7127c46..0000000
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/DropPreAggregateTablePostListener.scala
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * 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/cc0e6f1e/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateListeners.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateListeners.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateListeners.scala
new file mode 100644
index 0000000..b507856
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateListeners.scala
@@ -0,0 +1,81 @@
+/*
+ * 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.CarbonSession
+import org.apache.spark.sql.execution.command.CarbonDropTableCommand
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.metadata.schema.table.DataMapSchema
+import org.apache.carbondata.events.{DropTablePostEvent, Event, LoadTablePostExecutionEvent, OperationContext, OperationEventListener}
+
+object 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)
+      }
+    }
+
+  }
+}
+
+object LoadPostAggregateListener extends OperationEventListener {
+  /**
+   * Called on a specified event occurrence
+   *
+   * @param event
+   */
+  override def onEvent(event: Event, operationContext: OperationContext): Unit = {
+    val loadEvent = event.asInstanceOf[LoadTablePostExecutionEvent]
+    val sparkSession = loadEvent.sparkSession
+    val carbonLoadModel = loadEvent.carbonLoadModel
+    val table = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
+    if (!table.getTableInfo.getDataMapSchemaList.isEmpty) {
+      for (dataMapSchema: DataMapSchema <- table.getTableInfo.getDataMapSchemaList.asScala) {
+        CarbonSession
+          .threadSet(CarbonCommonConstants.CARBON_INPUT_SEGMENTS +
+                     carbonLoadModel.getDatabaseName + "." +
+                     carbonLoadModel.getTableName,
+            carbonLoadModel.getSegmentId)
+        CarbonSession.threadSet(CarbonCommonConstants.VALIDATE_CARBON_INPUT_SEGMENTS +
+                                carbonLoadModel.getDatabaseName + "." +
+                                carbonLoadModel.getTableName, "false")
+        val childTableName = dataMapSchema.getRelationIdentifier.getTableName
+        val childDatabaseName = dataMapSchema.getRelationIdentifier.getDatabaseName
+        val selectQuery = dataMapSchema.getProperties.get("CHILD_SELECT QUERY")
+        sparkSession.sql(s"insert into $childDatabaseName.$childTableName $selectQuery")
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/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 fd0e543..b35b525 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
@@ -26,7 +26,7 @@ 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.CarbonRelation
 import org.apache.spark.sql.hive.HiveExternalCatalog.{DATASOURCE_SCHEMA_NUMPARTS, DATASOURCE_SCHEMA_PART_PREFIX}
 import org.apache.spark.sql.types.DataType
 
@@ -35,6 +35,7 @@ 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.statusmanager.SegmentStatusManager
 import org.apache.carbondata.core.util.path.CarbonStorePath
 import org.apache.carbondata.format.TableInfo
 import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
@@ -428,4 +429,24 @@ object PreAggregateUtil {
           thriftTable, carbonTable.getAbsoluteTableIdentifier.getTablePath)(sparkSession)
     }
   }
+
+  def getChildCarbonTable(databaseName: String, tableName: String)
+    (sparkSession: SparkSession): Option[CarbonTable] = {
+    val metaStore = CarbonEnv.getInstance(sparkSession).carbonMetastore
+    metaStore.getTableFromMetadataCache(databaseName, tableName) match {
+      case Some(tableMeta) => Some(tableMeta.carbonTable)
+      case None => try {
+        Some(metaStore.lookupRelation(Some(databaseName), tableName)(sparkSession)
+          .asInstanceOf[CarbonRelation].metaData.carbonTable)
+      } catch {
+        case _: Exception =>
+          None
+      }
+    }
+  }
+
+  def checkMainTableLoad(carbonTable: CarbonTable): Boolean = {
+    SegmentStatusManager.readLoadMetadata(
+      carbonTable.getMetaDataFilepath).nonEmpty
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
index f61ab84..ba7e1eb 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
@@ -20,7 +20,8 @@ package org.apache.spark.sql.hive
 import org.apache.spark.sql._
 import org.apache.spark.sql.catalyst.CarbonTableIdentifierImplicit
 import org.apache.spark.sql.catalyst.analysis.{UnresolvedAlias, UnresolvedAttribute, UnresolvedFunction, UnresolvedRelation, UnresolvedStar}
-import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, NamedExpression}
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, Cast, ExprId, NamedExpression}
+import org.apache.spark.sql.catalyst.expressions.aggregate.{DeclarativeAggregate, _}
 import org.apache.spark.sql.catalyst.plans.Inner
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.rules._
@@ -57,8 +58,16 @@ object CarbonPreInsertionCasts extends Rule[LogicalPlan] {
           .DEFAULT_MAX_NUMBER_OF_COLUMNS
         )
     }
-    if (child.output.size >= relation.carbonRelation.output.size) {
-      val newChildOutput = child.output.zipWithIndex.map { columnWithIndex =>
+    val isAggregateTable = !relation.carbonRelation.tableMeta.carbonTable.getTableInfo
+      .getParentRelationIdentifiers.isEmpty
+    // transform logical plan if the load is for aggregate table.
+    val childPlan = if (isAggregateTable) {
+      transformAggregatePlan(child)
+    } else {
+      child
+    }
+    if (childPlan.output.size >= relation.carbonRelation.output.size) {
+      val newChildOutput = childPlan.output.zipWithIndex.map { columnWithIndex =>
         columnWithIndex._1 match {
           case attr: Alias =>
             Alias(attr.child, s"col${ columnWithIndex._2 }")(attr.exprId)
@@ -67,16 +76,54 @@ object CarbonPreInsertionCasts extends Rule[LogicalPlan] {
           case attr => attr
         }
       }
-      val newChild: LogicalPlan = if (newChildOutput == child.output) {
+      val newChild: LogicalPlan = if (newChildOutput == childPlan.output) {
         p.child
       } else {
-        Project(newChildOutput, child)
+        Project(newChildOutput, childPlan)
       }
       InsertIntoCarbonTable(relation, p.partition, newChild, p.overwrite, p.ifNotExists)
     } else {
       sys.error("Cannot insert into target table because column number are different")
     }
   }
+
+  /**
+   * Transform the logical plan with average(col1) aggregation type to sum(col1) and count(col1).
+   *
+   * @param logicalPlan
+   * @return
+   */
+  private def transformAggregatePlan(logicalPlan: LogicalPlan): LogicalPlan = {
+    logicalPlan transform {
+      case aggregate@Aggregate(_, aExp, _) =>
+        val newExpressions = aExp flatMap {
+          case alias@Alias(attrExpression: AggregateExpression, _) =>
+            attrExpression.aggregateFunction flatMap {
+              case Average(attr: AttributeReference) =>
+                Seq(Alias(attrExpression
+                  .copy(aggregateFunction = Sum(attr.withName(attr.name)),
+                    resultId = NamedExpression.newExprId),
+                  attr.name)(),
+                  Alias(attrExpression
+                    .copy(aggregateFunction = Count(attr.withName(attr.name)),
+                      resultId = NamedExpression.newExprId), attr.name)())
+              case Average(Cast(attr: AttributeReference, _)) =>
+                Seq(Alias(attrExpression
+                  .copy(aggregateFunction = Sum(attr.withName(attr.name)),
+                    resultId = NamedExpression.newExprId),
+                  attr.name)(),
+                  Alias(attrExpression
+                    .copy(aggregateFunction = Count(attr.withName(attr.name)),
+                      resultId = NamedExpression.newExprId), attr.name)())
+              case _: DeclarativeAggregate => Seq(alias)
+              case _ => Nil
+            }
+          case namedExpr: NamedExpression => Seq(namedExpr)
+        }
+        aggregate.copy(aggregateExpressions = newExpressions)
+      case plan: LogicalPlan => plan
+    }
+  }
 }
 
 case class CarbonIUDAnalysisRule(sparkSession: SparkSession) extends Rule[LogicalPlan] {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/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 97ea7f8..d17dd11 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,7 +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.command.preaaggregate.{DropPreAggregateTablePostListener, LoadPostAggregateListener}
 import org.apache.spark.sql.execution.datasources._
 import org.apache.spark.sql.execution.strategy.{CarbonLateDecodeStrategy, DDLStrategy, StreamingTableStrategy}
 import org.apache.spark.sql.internal.SQLConf
@@ -35,7 +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}
+import org.apache.carbondata.events.{DropTablePostEvent, LoadTablePostExecutionEvent, OperationListenerBus}
 
 /**
  * This class will have carbon catalog and refresh the relation from cache if the carbontable in
@@ -130,7 +130,8 @@ object CarbonSessionState {
 
   def init(): Unit = {
     OperationListenerBus.getInstance()
-      .addListener(classOf[DropTablePostEvent], new DropPreAggregateTablePostListener)
+      .addListener(classOf[DropTablePostEvent], DropPreAggregateTablePostListener)
+      .addListener(classOf[LoadTablePostExecutionEvent], LoadPostAggregateListener)
   }
 
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/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 f2c8a0a..7d25efd 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
@@ -24,6 +24,7 @@ import org.apache.spark.sql.execution.command._
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil, SessionParams}
+import org.apache.carbondata.hadoop.api.CarbonTableInputFormat
 import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 
 case class CarbonDropDatabaseCommand(command: DropDatabaseCommand)
@@ -83,6 +84,8 @@ object CarbonSetCommand {
           "property should be in \" carbon.input.segments.<database_name>" +
           ".<table_name>=<seg_id list> \" format.")
       }
+    } else if (key.startsWith(CarbonCommonConstants.VALIDATE_CARBON_INPUT_SEGMENTS)) {
+      sessionParams.addProperty(key.toLowerCase(), value)
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/DictionaryFieldConverterImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/DictionaryFieldConverterImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/DictionaryFieldConverterImpl.java
index 2671393..7045101 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/DictionaryFieldConverterImpl.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/DictionaryFieldConverterImpl.java
@@ -37,7 +37,6 @@ import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.core.util.DataTypeUtil;
-import org.apache.carbondata.core.util.path.CarbonStorePath;
 import org.apache.carbondata.processing.loading.DataField;
 import org.apache.carbondata.processing.loading.converter.BadRecordLogHolder;
 import org.apache.carbondata.processing.loading.dictionary.DictionaryServerClientDictionary;
@@ -68,15 +67,12 @@ public class DictionaryFieldConverterImpl extends AbstractDictionaryFieldConvert
       Cache<DictionaryColumnUniqueIdentifier, Dictionary> cache,
       CarbonTableIdentifier carbonTableIdentifier, String nullFormat, int index,
       DictionaryClient client, boolean useOnePass, String storePath,
-      Map<Object, Integer> localCache, boolean isEmptyBadRecord) throws IOException {
+      Map<Object, Integer> localCache, boolean isEmptyBadRecord,
+      DictionaryColumnUniqueIdentifier identifier) throws IOException {
     this.index = index;
     this.carbonDimension = (CarbonDimension) dataField.getColumn();
     this.nullFormat = nullFormat;
     this.isEmptyBadRecord = isEmptyBadRecord;
-    DictionaryColumnUniqueIdentifier identifier =
-        new DictionaryColumnUniqueIdentifier(carbonTableIdentifier,
-            dataField.getColumn().getColumnIdentifier(), dataField.getColumn().getDataType(),
-            CarbonStorePath.getCarbonTablePath(storePath, carbonTableIdentifier));
 
     // if use one pass, use DictionaryServerClientDictionary
     if (useOnePass) {


[12/49] carbondata git commit: [CARBONDATA-1537] Added back Adaptive delta encoding for floating type for backward compatibility

Posted by ra...@apache.org.
[CARBONDATA-1537] Added back Adaptive delta encoding for floating type for backward compatibility

Currently, backward compatibility is broken because of missing adaptive delta floating encoding which was present in older versions but it is removed in the latest version.
Added back this encoding to keep the compatibility

This closes #1400


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

Branch: refs/heads/fgdatamap
Commit: 11661eb69d8a484747cba30abac88eb4c0cb2125
Parents: d6967bf
Author: Ravindra Pesala <ra...@gmail.com>
Authored: Sun Oct 1 22:53:12 2017 +0530
Committer: Jacky Li <ja...@qq.com>
Committed: Tue Nov 7 15:17:50 2017 +0800

----------------------------------------------------------------------
 .../page/encoding/DefaultEncodingFactory.java   |   8 +-
 .../page/encoding/EncodingFactory.java          |  19 +-
 .../adaptive/AdaptiveDeltaFloatingCodec.java    | 216 +++++++++++++++++++
 format/src/main/thrift/schema.thrift            |   1 +
 .../CarbonV1toV3CompatabilityTestCase.scala     |   6 +
 5 files changed, 242 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/11661eb6/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/DefaultEncodingFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/DefaultEncodingFactory.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/DefaultEncodingFactory.java
index 518573d..54467b2 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/DefaultEncodingFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/DefaultEncodingFactory.java
@@ -24,6 +24,7 @@ import org.apache.carbondata.core.datastore.compression.Compressor;
 import org.apache.carbondata.core.datastore.compression.CompressorFactory;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
 import org.apache.carbondata.core.datastore.page.DecimalColumnPage;
+import org.apache.carbondata.core.datastore.page.encoding.adaptive.AdaptiveDeltaFloatingCodec;
 import org.apache.carbondata.core.datastore.page.encoding.adaptive.AdaptiveDeltaIntegralCodec;
 import org.apache.carbondata.core.datastore.page.encoding.adaptive.AdaptiveFloatingCodec;
 import org.apache.carbondata.core.datastore.page.encoding.adaptive.AdaptiveIntegralCodec;
@@ -281,7 +282,6 @@ public class DefaultEncodingFactory extends EncodingFactory {
     //Here we should use the Max abs as max to getDatatype, let's say -1 and -10000000, -1 is max,
     //but we can't use -1 to getDatatype, we should use -10000000.
     double absMaxValue = Math.max(Math.abs(maxValue), Math.abs(minValue));
-
     if (decimalCount == 0) {
       // short, int, long
       return selectCodecByAlgorithmForIntegral(stats);
@@ -291,7 +291,11 @@ public class DefaultEncodingFactory extends EncodingFactory {
       // double
       long max = (long) (Math.pow(10, decimalCount) * absMaxValue);
       DataType adaptiveDataType = fitLongMinMax(max, 0);
-      if (adaptiveDataType.getSizeInBytes() < DataTypes.DOUBLE.getSizeInBytes()) {
+      DataType deltaDataType = compareMinMaxAndSelectDataType(
+          (long) (Math.pow(10, decimalCount) * (maxValue - minValue)));
+      if (adaptiveDataType.getSizeInBytes() > deltaDataType.getSizeInBytes()) {
+        return new AdaptiveDeltaFloatingCodec(srcDataType, deltaDataType, stats);
+      } else if (adaptiveDataType.getSizeInBytes() < DataTypes.DOUBLE.getSizeInBytes()) {
         return new AdaptiveFloatingCodec(srcDataType, adaptiveDataType, stats);
       } else {
         return new DirectCompressCodec(DataTypes.DOUBLE);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/11661eb6/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/EncodingFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/EncodingFactory.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/EncodingFactory.java
index 0f45abb..4a674e3 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/EncodingFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/EncodingFactory.java
@@ -26,6 +26,7 @@ import java.util.List;
 import org.apache.carbondata.core.datastore.ColumnType;
 import org.apache.carbondata.core.datastore.TableSpec;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
+import org.apache.carbondata.core.datastore.page.encoding.adaptive.AdaptiveDeltaFloatingCodec;
 import org.apache.carbondata.core.datastore.page.encoding.adaptive.AdaptiveDeltaIntegralCodec;
 import org.apache.carbondata.core.datastore.page.encoding.adaptive.AdaptiveFloatingCodec;
 import org.apache.carbondata.core.datastore.page.encoding.adaptive.AdaptiveIntegralCodec;
@@ -40,12 +41,7 @@ import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.format.Encoding;
 
-import static org.apache.carbondata.format.Encoding.ADAPTIVE_DELTA_INTEGRAL;
-import static org.apache.carbondata.format.Encoding.ADAPTIVE_FLOATING;
-import static org.apache.carbondata.format.Encoding.ADAPTIVE_INTEGRAL;
-import static org.apache.carbondata.format.Encoding.BOOL_BYTE;
-import static org.apache.carbondata.format.Encoding.DIRECT_COMPRESS;
-import static org.apache.carbondata.format.Encoding.RLE_INTEGRAL;
+import static org.apache.carbondata.format.Encoding.*;
 
 /**
  * Base class for encoding factory implementation.
@@ -91,6 +87,12 @@ public abstract class EncodingFactory {
       SimpleStatsResult stats = PrimitivePageStatsCollector.newInstance(metadata);
       return new AdaptiveFloatingCodec(metadata.getSchemaDataType(), metadata.getStoreDataType(),
           stats).createDecoder(metadata);
+    } else if (encoding == ADAPTIVE_DELTA_FLOATING) {
+      ColumnPageEncoderMeta metadata = new ColumnPageEncoderMeta();
+      metadata.readFields(in);
+      SimpleStatsResult stats = PrimitivePageStatsCollector.newInstance(metadata);
+      return new AdaptiveDeltaFloatingCodec(metadata.getSchemaDataType(),
+          metadata.getStoreDataType(), stats).createDecoder(metadata);
     } else if (encoding == RLE_INTEGRAL) {
       RLEEncoderMeta metadata = new RLEEncoderMeta();
       metadata.readFields(in);
@@ -152,6 +154,11 @@ public abstract class EncodingFactory {
             new ColumnPageEncoderMeta(spec, DataType.getDataType(metadata.getType()), stats,
                 compressor);
         return codec.createDecoder(meta);
+      } else if (codec instanceof AdaptiveDeltaFloatingCodec) {
+        AdaptiveDeltaFloatingCodec adaptiveCodec = (AdaptiveDeltaFloatingCodec) codec;
+        ColumnPageEncoderMeta meta =
+            new ColumnPageEncoderMeta(spec, adaptiveCodec.getTargetDataType(), stats, compressor);
+        return codec.createDecoder(meta);
       } else {
         throw new RuntimeException("internal error");
       }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/11661eb6/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveDeltaFloatingCodec.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveDeltaFloatingCodec.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveDeltaFloatingCodec.java
new file mode 100644
index 0000000..1f00feb
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveDeltaFloatingCodec.java
@@ -0,0 +1,216 @@
+/*
+ * 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.datastore.page.encoding.adaptive;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.carbondata.core.datastore.compression.Compressor;
+import org.apache.carbondata.core.datastore.compression.CompressorFactory;
+import org.apache.carbondata.core.datastore.page.ColumnPage;
+import org.apache.carbondata.core.datastore.page.ColumnPageValueConverter;
+import org.apache.carbondata.core.datastore.page.LazyColumnPage;
+import org.apache.carbondata.core.datastore.page.encoding.ColumnPageCodec;
+import org.apache.carbondata.core.datastore.page.encoding.ColumnPageDecoder;
+import org.apache.carbondata.core.datastore.page.encoding.ColumnPageEncoder;
+import org.apache.carbondata.core.datastore.page.encoding.ColumnPageEncoderMeta;
+import org.apache.carbondata.core.datastore.page.statistics.SimpleStatsResult;
+import org.apache.carbondata.core.memory.MemoryException;
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
+import org.apache.carbondata.format.Encoding;
+
+/**
+ * Codec for floating point (float, double) data type page.
+ * This codec will calculate delta of page max value and page value and converts to Integer value,
+ * and do type casting of the diff to make storage minimum.
+ */
+public class AdaptiveDeltaFloatingCodec extends AdaptiveCodec {
+
+  private ColumnPage encodedPage;
+  private Double factor;
+  private long max;
+
+  public static ColumnPageCodec newInstance(DataType srcDataType, DataType targetDataType,
+      SimpleStatsResult stats) {
+    return new AdaptiveDeltaFloatingCodec(srcDataType, targetDataType, stats);
+  }
+
+  public AdaptiveDeltaFloatingCodec(DataType srcDataType, DataType targetDataType,
+      SimpleStatsResult stats) {
+    super(srcDataType, targetDataType, stats);
+    this.factor = Math.pow(10, stats.getDecimalCount());
+    this.max = (long) (Math.pow(10, stats.getDecimalCount()) * (double) stats.getMax());
+  }
+
+  @Override
+  public String getName() {
+    return "AdaptiveDeltaFloatingCodec";
+  }
+
+  @Override
+  public ColumnPageEncoder createEncoder(Map<String, String> parameter) {
+    final Compressor compressor = CompressorFactory.getInstance().getCompressor();
+    return new ColumnPageEncoder() {
+      @Override
+      protected byte[] encodeData(ColumnPage input) throws MemoryException, IOException {
+        if (encodedPage != null) {
+          throw new IllegalStateException("already encoded");
+        }
+        encodedPage = ColumnPage.newPage(input.getColumnSpec(), targetDataType,
+            input.getPageSize());
+        input.convertValue(converter);
+        byte[] result = encodedPage.compress(compressor);
+        encodedPage.freeMemory();
+        return result;
+      }
+
+      @Override
+      protected List<Encoding> getEncodingList() {
+        List<Encoding> encodings = new ArrayList<Encoding>();
+        encodings.add(Encoding.ADAPTIVE_DELTA_FLOATING);
+        return encodings;
+      }
+
+      @Override
+      protected ColumnPageEncoderMeta getEncoderMeta(ColumnPage inputPage) {
+        return new ColumnPageEncoderMeta(inputPage.getColumnSpec(), targetDataType, stats,
+            compressor.getName());
+      }
+
+    };
+  }
+
+  @Override
+  public ColumnPageDecoder createDecoder(final ColumnPageEncoderMeta meta) {
+    return new ColumnPageDecoder() {
+      @Override
+      public ColumnPage decode(byte[] input, int offset, int length)
+          throws MemoryException, IOException {
+        ColumnPage page = ColumnPage.decompress(meta, input, offset, length);
+        return LazyColumnPage.newPage(page, converter);
+      }
+    };
+  }
+
+  // encoded value = (10 power of decimal) * (page value)
+  private ColumnPageValueConverter converter = new ColumnPageValueConverter() {
+    @Override
+    public void encode(int rowId, byte value) {
+      // this codec is for floating point type only
+      throw new RuntimeException("internal error: " + debugInfo());
+    }
+
+    @Override
+    public void encode(int rowId, short value) {
+      // this codec is for floating point type only
+      throw new RuntimeException("internal error: " + debugInfo());
+    }
+
+    @Override
+    public void encode(int rowId, int value) {
+      // this codec is for floating point type only
+      throw new RuntimeException("internal error: " + debugInfo());
+    }
+
+    @Override
+    public void encode(int rowId, long value) {
+      // this codec is for floating point type only
+      throw new RuntimeException("internal error: " + debugInfo());
+    }
+
+    @Override
+    public void encode(int rowId, float value) {
+      if (targetDataType.equals(DataTypes.BYTE)) {
+        encodedPage.putByte(rowId, (byte) (max - (value * factor)));
+      } else if (targetDataType.equals(DataTypes.SHORT)) {
+        encodedPage.putShort(rowId, (short) (max - (value * factor)));
+      } else if (targetDataType.equals(DataTypes.SHORT_INT)) {
+        encodedPage.putShortInt(rowId, (int) (max - (value * factor)));
+      } else if (targetDataType.equals(DataTypes.INT)) {
+        encodedPage.putInt(rowId, (int) (max - (value * factor)));
+      } else {
+        throw new RuntimeException("internal error: " + debugInfo());
+      }
+    }
+
+    @Override
+    public void encode(int rowId, double value) {
+      if (targetDataType.equals(DataTypes.BYTE)) {
+        encodedPage.putByte(rowId, (byte) (max - (value * factor)));
+      } else if (targetDataType.equals(DataTypes.SHORT)) {
+        encodedPage.putShort(rowId, (short) (max - (value * factor)));
+      } else if (targetDataType.equals(DataTypes.SHORT_INT)) {
+        encodedPage.putShortInt(rowId, (int) (max - (value * factor)));
+      } else if (targetDataType.equals(DataTypes.INT)) {
+        encodedPage.putInt(rowId, (int) (max - (value * factor)));
+      } else if (targetDataType.equals(DataTypes.DOUBLE)) {
+        encodedPage.putDouble(rowId, value);
+      } else {
+        throw new RuntimeException("internal error: " + debugInfo());
+      }
+    }
+
+    @Override
+    public long decodeLong(byte value) {
+      throw new RuntimeException("internal error: " + debugInfo());
+    }
+
+    @Override
+    public long decodeLong(short value) {
+      throw new RuntimeException("internal error: " + debugInfo());
+    }
+
+    @Override
+    public long decodeLong(int value) {
+      throw new RuntimeException("internal error: " + debugInfo());
+    }
+
+    @Override
+    public double decodeDouble(byte value) {
+      return (max - value) / factor;
+    }
+
+    @Override
+    public double decodeDouble(short value) {
+      return (max - value) / factor;
+    }
+
+    @Override
+    public double decodeDouble(int value) {
+      return (max - value) / factor;
+    }
+
+    @Override
+    public double decodeDouble(long value) {
+      return (max - value) / factor;
+    }
+
+    @Override
+    public double decodeDouble(float value) {
+      throw new RuntimeException("internal error: " + debugInfo());
+    }
+
+    @Override
+    public double decodeDouble(double value) {
+      throw new RuntimeException("internal error: " + debugInfo());
+    }
+  };
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/11661eb6/format/src/main/thrift/schema.thrift
----------------------------------------------------------------------
diff --git a/format/src/main/thrift/schema.thrift b/format/src/main/thrift/schema.thrift
index 7869378..216d91f 100644
--- a/format/src/main/thrift/schema.thrift
+++ b/format/src/main/thrift/schema.thrift
@@ -55,6 +55,7 @@ enum Encoding{
 	DIRECT_STRING = 10;   // Stores string value and string length separately in page data
 	ADAPTIVE_FLOATING = 11; // Identifies that a column is encoded using AdaptiveFloatingCodec
 	BOOL_BYTE = 12;   // Identifies that a column is encoded using BooleanPageCodec
+	ADAPTIVE_DELTA_FLOATING = 13; // Identifies that a column is encoded using AdaptiveDeltaFloatingCodec
 }
 
 enum PartitionType{

http://git-wip-us.apache.org/repos/asf/carbondata/blob/11661eb6/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/CarbonV1toV3CompatabilityTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/CarbonV1toV3CompatabilityTestCase.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/CarbonV1toV3CompatabilityTestCase.scala
index d737092..8115e27 100644
--- a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/CarbonV1toV3CompatabilityTestCase.scala
+++ b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/CarbonV1toV3CompatabilityTestCase.scala
@@ -81,6 +81,12 @@ class CarbonV1toV3CompatabilityTestCase extends QueryTest with BeforeAndAfterAll
     checkAnswer(dataFrame, Seq(Row(9281064)))
   }
 
+  test("test v1 to v3 compatabilty filter on measure with double dimension") {
+    val dataFrame = localspark
+      .sql(s"SELECT sum(salary1) FROM t3 where salary1 > 15408")
+    checkAnswer(dataFrame, Seq(Row(9281064)))
+  }
+
   override def afterAll {
     localspark.stop()
   }


[03/49] carbondata git commit: [CARBONDATA-1617] Merging carbonindex files within segment

Posted by ra...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/0586146a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergeFilesRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergeFilesRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergeFilesRDD.scala
new file mode 100644
index 0000000..6e8b000
--- /dev/null
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergeFilesRDD.scala
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.spark.rdd
+
+import org.apache.spark.{Partition, SparkContext, TaskContext}
+
+import org.apache.carbondata.core.util.path.CarbonTablePath
+import org.apache.carbondata.core.writer.CarbonIndexFileMergeWriter
+
+case class CarbonMergeFilePartition(rddId: Int, val idx: Int, segmentPath: String)
+  extends Partition {
+
+  override val index: Int = idx
+
+  override def hashCode(): Int = 41 * (41 + rddId) + idx
+}
+
+/**
+ * RDD to merge all carbonindex files of each segment to carbonindex file into the same segment.
+ * @param sc
+ * @param tablePath
+ * @param segments segments to be merged
+ */
+class CarbonMergeFilesRDD(
+    sc: SparkContext,
+    tablePath: String,
+    segments: Seq[String])
+  extends CarbonRDD[String](sc, Nil) {
+
+  override def getPartitions: Array[Partition] = {
+    segments.zipWithIndex.map {s =>
+      CarbonMergeFilePartition(id, s._2, CarbonTablePath.getSegmentPath(tablePath, s._1))
+    }.toArray
+  }
+
+  override def internalCompute(theSplit: Partition, context: TaskContext): Iterator[String] = {
+    val iter = new Iterator[String] {
+      val split = theSplit.asInstanceOf[CarbonMergeFilePartition]
+      logInfo("Merging carbon index files of segment : " + split.segmentPath)
+
+      new CarbonIndexFileMergeWriter().mergeCarbonIndexFilesOfSegment(split.segmentPath)
+
+      var havePair = false
+      var finished = false
+
+      override def hasNext: Boolean = {
+        if (!finished && !havePair) {
+          finished = true
+          havePair = !finished
+        }
+        !finished
+      }
+
+      override def next(): String = {
+        if (!hasNext) {
+          throw new java.util.NoSuchElementException("End of stream")
+        }
+        havePair = false
+        ""
+      }
+
+    }
+    iter
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/carbondata/blob/0586146a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/Compactor.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/Compactor.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/Compactor.scala
index fb610c1..cb25756 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/Compactor.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/Compactor.scala
@@ -22,10 +22,12 @@ import scala.collection.JavaConverters._
 import org.apache.spark.sql.execution.command.{CarbonMergerMapping, CompactionCallableModel}
 
 import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.mutate.CarbonUpdateUtil
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager
 import org.apache.carbondata.processing.merger.{CarbonDataMergerUtil, CompactionType}
 import org.apache.carbondata.spark.MergeResultImpl
+import org.apache.carbondata.spark.util.CommonUtil
 
 /**
  * Compactor class which handled the compaction cases.
@@ -106,6 +108,8 @@ object Compactor {
     }
 
     if (finalMergeStatus) {
+      val mergedLoadNumber = CarbonDataMergerUtil.getLoadNumberFromLoadName(mergedLoadName)
+      CommonUtil.mergeIndexFiles(sc.sparkContext, Seq(mergedLoadNumber), storePath, carbonTable)
       val endTime = System.nanoTime()
       logger.info(s"time taken to merge $mergedLoadName is ${ endTime - startTime }")
       val statusFileUpdation =
@@ -116,7 +120,7 @@ object Compactor {
               carbonLoadModel))) ||
          (CarbonDataMergerUtil
            .updateLoadMetadataWithMergeStatus(loadsToMerge, carbonTable.getMetaDataFilepath(),
-             mergedLoadName, carbonLoadModel, mergeLoadStartTime, compactionType))
+             mergedLoadNumber, carbonLoadModel, mergeLoadStartTime, compactionType))
           )
 
       if (!statusFileUpdation) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/0586146a/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 bc24c12..27ebf42 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
@@ -28,7 +28,7 @@ import org.apache.commons.lang3.StringUtils
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat
 import org.apache.spark.SparkContext
-import org.apache.spark.sql.{Row, RowFactory}
+import org.apache.spark.sql.{Row, RowFactory, SQLContext}
 import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
 import org.apache.spark.sql.execution.command.{ColumnProperty, Field, PartitionerField}
 import org.apache.spark.sql.types.{MetadataBuilder, StringType}
@@ -44,6 +44,7 @@ import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
 import org.apache.carbondata.core.metadata.datatype.{DataType, DataTypes}
 import org.apache.carbondata.core.metadata.schema.PartitionInfo
 import org.apache.carbondata.core.metadata.schema.partition.PartitionType
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.mutate.CarbonUpdateUtil
 import org.apache.carbondata.core.scan.partition.PartitionUtil
 import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
@@ -55,6 +56,7 @@ import org.apache.carbondata.processing.loading.exception.CarbonDataLoadingExcep
 import org.apache.carbondata.processing.loading.model.CarbonLoadModel
 import org.apache.carbondata.processing.util.{CarbonDataProcessorUtil, CarbonLoaderUtil}
 import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
+import org.apache.carbondata.spark.rdd.CarbonMergeFilesRDD
 
 object CommonUtil {
   private val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
@@ -831,4 +833,20 @@ object CommonUtil {
         LOGGER.error(s)
     }
   }
+
+  /**
+   * Merge the carbonindex files with in the segment to carbonindexmerge file inside same segment
+   */
+  def mergeIndexFiles(sparkContext: SparkContext,
+      segmentIds: Seq[String],
+      storePath: String,
+      carbonTable: CarbonTable): Unit = {
+    if (CarbonProperties.getInstance().getProperty(
+      CarbonCommonConstants.CARBON_MERGE_INDEX_IN_SEGMENT,
+      CarbonCommonConstants.CARBON_MERGE_INDEX_IN_SEGMENT_DEFAULT).toBoolean) {
+      new CarbonMergeFilesRDD(sparkContext, AbsoluteTableIdentifier.from(storePath,
+        carbonTable.getDatabaseName, carbonTable.getFactTableName).getTablePath,
+        segmentIds).collect()
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/0586146a/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 4649082..1163b3f 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
@@ -35,12 +35,11 @@ import org.apache.spark.util.PartitionUtils
 import org.apache.carbondata.common.constants.LoggerAction
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.metadata.datatype.{DataType, DataTypes}
+import org.apache.carbondata.core.metadata.datatype.DataTypes
 import org.apache.carbondata.core.metadata.schema.PartitionInfo
 import org.apache.carbondata.core.metadata.schema.partition.PartitionType
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema
 import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil, DataTypeUtil}
-import org.apache.carbondata.processing.loading.sort.SortScopeOptions
 import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 import org.apache.carbondata.spark.util.{CommonUtil, DataTypeConverterUtil}
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/0586146a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
index 87de8ae..628d444 100644
--- a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
@@ -98,8 +98,10 @@ object CarbonDataRDDFactory {
           .setLoadMetadataDetails(alterTableModel.segmentUpdateStatusManager.get
             .getLoadMetadataDetails.toList.asJava)
       }
-    }
-    else {
+    } else if (alterTableModel.compactionType.
+      equalsIgnoreCase(CompactionType.SEGMENT_INDEX_COMPACTION.toString)) {
+      compactionType = CompactionType.SEGMENT_INDEX_COMPACTION
+    } else {
       compactionType = CompactionType.MINOR_COMPACTION
     }
 
@@ -110,6 +112,14 @@ object CarbonDataRDDFactory {
     if (null == carbonLoadModel.getLoadMetadataDetails) {
       CommonUtil.readLoadMetadataDetails(carbonLoadModel)
     }
+    if (compactionType == CompactionType.SEGMENT_INDEX_COMPACTION) {
+      // Just launch job to merge index and return
+      CommonUtil.mergeIndexFiles(sqlContext.sparkContext,
+        carbonLoadModel.getLoadMetadataDetails.asScala.map(_.getLoadName),
+        carbonLoadModel.getStorePath,
+        carbonTable)
+      return
+    }
     // reading the start time of data load.
     val loadStartTime : Long =
     if (alterTableModel.factTimeStamp.isEmpty) {
@@ -959,9 +969,10 @@ object CarbonDataRDDFactory {
             }
           ))
 
-        }
-        else {
-        val newStatusMap = scala.collection.mutable.Map.empty[String, String]
+        } else {
+          CommonUtil.mergeIndexFiles(sqlContext.sparkContext,
+            Seq(carbonLoadModel.getSegmentId), storePath, carbonTable)
+          val newStatusMap = scala.collection.mutable.Map.empty[String, String]
           if (status.nonEmpty) {
             status.foreach { eachLoadStatus =>
               val state = newStatusMap.get(eachLoadStatus._1)
@@ -1142,8 +1153,10 @@ object CarbonDataRDDFactory {
 
   }
 
+
   /**
    * repartition the input data for partition table.
+   *
    * @param sqlContext
    * @param dataFrame
    * @param carbonLoadModel

http://git-wip-us.apache.org/repos/asf/carbondata/blob/0586146a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala
index 715af1d..bdfaa5a 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala
@@ -87,7 +87,8 @@ class DDLStrategy(sparkSession: SparkSession) extends SparkStrategy {
             altertablemodel.dbName))(sparkSession)
         if (isCarbonTable) {
           if (altertablemodel.compactionType.equalsIgnoreCase("minor") ||
-              altertablemodel.compactionType.equalsIgnoreCase("major")) {
+              altertablemodel.compactionType.equalsIgnoreCase("major") ||
+              altertablemodel.compactionType.equalsIgnoreCase("SEGMENT_INDEX_COMPACTION")) {
             ExecutedCommandExec(alterTable) :: Nil
           } else {
             throw new MalformedCarbonCommandException(

http://git-wip-us.apache.org/repos/asf/carbondata/blob/0586146a/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java b/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
index 13972c8..53add22 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
@@ -131,24 +131,6 @@ public final class CarbonDataMergerUtil {
 
 
   /**
-   * Form the Name of the New Merge Folder
-   *
-   * @param segmentToBeMerged
-   * @return
-   */
-  public static String getMergedLoadName(final String segmentToBeMerged) {
-    if (segmentToBeMerged.contains(".")) {
-      String beforeDecimal = segmentToBeMerged.substring(0, segmentToBeMerged.indexOf("."));
-      String afterDecimal = segmentToBeMerged.substring(segmentToBeMerged.indexOf(".") + 1);
-      int fraction = Integer.parseInt(afterDecimal) + 1;
-      return beforeDecimal + "." + fraction;
-    } else {
-      return segmentToBeMerged + "." + 1;
-    }
-
-  }
-
-  /**
    * Update Both Segment Update Status and Table Status for the case of IUD Delete
    * delta compaction.
    *
@@ -294,13 +276,13 @@ public final class CarbonDataMergerUtil {
    * method to update table status in case of IUD Update Delta Compaction.
    * @param loadsToMerge
    * @param metaDataFilepath
-   * @param MergedLoadName
+   * @param mergedLoadNumber
    * @param carbonLoadModel
    * @param compactionType
    * @return
    */
   public static boolean updateLoadMetadataWithMergeStatus(List<LoadMetadataDetails> loadsToMerge,
-      String metaDataFilepath, String MergedLoadName, CarbonLoadModel carbonLoadModel,
+      String metaDataFilepath, String mergedLoadNumber, CarbonLoadModel carbonLoadModel,
       long mergeLoadStartTime, CompactionType compactionType) {
 
     boolean tableStatusUpdationStatus = false;
@@ -323,10 +305,6 @@ public final class CarbonDataMergerUtil {
 
         LoadMetadataDetails[] loadDetails = SegmentStatusManager.readLoadMetadata(metaDataFilepath);
 
-        String mergedLoadNumber = MergedLoadName.substring(
-            MergedLoadName.lastIndexOf(CarbonCommonConstants.LOAD_FOLDER)
-                + CarbonCommonConstants.LOAD_FOLDER.length(), MergedLoadName.length());
-
         long modificationOrDeletionTimeStamp = CarbonUpdateUtil.readCurrentTime();
         for (LoadMetadataDetails loadDetail : loadDetails) {
           // check if this segment is merged.
@@ -391,6 +369,17 @@ public final class CarbonDataMergerUtil {
   }
 
   /**
+   * Get the load number from load name.
+   * @param loadName
+   * @return
+   */
+  public static String getLoadNumberFromLoadName(String loadName) {
+    return loadName.substring(
+        loadName.lastIndexOf(CarbonCommonConstants.LOAD_FOLDER) + CarbonCommonConstants.LOAD_FOLDER
+            .length(), loadName.length());
+  }
+
+  /**
    * To identify which all segments can be merged.
    *
    * @param carbonLoadModel

http://git-wip-us.apache.org/repos/asf/carbondata/blob/0586146a/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionType.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionType.java b/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionType.java
index 6b9c80a..863257c 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionType.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionType.java
@@ -27,5 +27,6 @@ public enum CompactionType {
     MAJOR_COMPACTION,
     IUD_UPDDEL_DELTA_COMPACTION,
     IUD_DELETE_DELTA_COMPACTION,
+    SEGMENT_INDEX_COMPACTION,
     NONE
 }


[36/49] 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/fgdatamap
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


[29/49] carbondata git commit: [CARBONDATA-1398] Support query from specified segments

Posted by ra...@apache.org.
[CARBONDATA-1398] Support query from specified segments

A new property will introduce to set the segment no.
User will set property(carbon.input.segments. <database_name> .<table_name>) to specify segment no.
During CarbonScan data will be read from from specified segments only.
If property is not set, all segments will be caonsidered as default behavior.

This closes #1367


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

Branch: refs/heads/fgdatamap
Commit: fd0bdf6f6c9ebe124789d4041ed98a9a790f38c1
Parents: 2de2406
Author: rahulforallp <ra...@knoldus.in>
Authored: Thu Sep 14 18:44:09 2017 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Fri Nov 10 10:15:26 2017 +0530

----------------------------------------------------------------------
 .../core/constants/CarbonCommonConstants.java   |   6 +
 .../apache/carbondata/core/util/CarbonUtil.java |  25 ++
 .../carbondata/core/util/SessionParams.java     |  12 +-
 .../hadoop/api/CarbonTableInputFormat.java      |  61 +++-
 .../src/test/resources/data1.csv                |  11 +
 .../org/apache/carbondata/api/CarbonStore.scala |   8 +-
 .../carbondata/spark/rdd/CarbonScanRDD.scala    |   8 +
 .../spark/sql/CarbonCatalystOperators.scala     |   0
 .../spark/sql/CarbonCatalystOperators.scala     |   3 +-
 .../command/mutation/IUDCommonUtil.scala        |  78 ++++
 .../mutation/ProjectForDeleteCommand.scala      |   1 +
 .../mutation/ProjectForUpdateCommand.scala      |   1 +
 .../execution/command/CarbonHiveCommands.scala  |  12 +
 .../spark/sql/internal/CarbonSqlConf.scala      |   5 +
 .../org/apache/spark/util/ShowSegments.scala    |  10 +-
 .../segmentreading/TestSegmentReading.scala     | 352 +++++++++++++++++++
 16 files changed, 578 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/fd0bdf6f/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
index 17936d9..5f63cc1 100644
--- a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
+++ b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
@@ -50,6 +50,12 @@ public final class CarbonCommonConstants {
    * measure meta data file name
    */
   public static final String MEASURE_METADATA_FILE_NAME = "/msrMetaData_";
+
+  /**
+   * set the segment ids to query from the table
+   */
+  public static final String CARBON_INPUT_SEGMENTS = "carbon.input.segments.";
+
   /**
    * location of the carbon member, hierarchy and fact files
    */

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fd0bdf6f/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
index 016b8b3..51b6f06 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
@@ -54,6 +54,7 @@ import org.apache.carbondata.core.datastore.columnar.ColumnGroupModel;
 import org.apache.carbondata.core.datastore.columnar.UnBlockIndexer;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.exception.InvalidConfigurationException;
 import org.apache.carbondata.core.indexstore.BlockletDetailInfo;
 import org.apache.carbondata.core.keygenerator.mdkey.NumberCompressor;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
@@ -2011,6 +2012,30 @@ public final class CarbonUtil {
     }
   }
 
+  public static boolean validateRangeOfSegmentList(String segmentId)
+      throws InvalidConfigurationException {
+    String[] values = segmentId.split(",");
+    try {
+      if (values.length == 0) {
+        throw new InvalidConfigurationException(
+            "carbon.input.segments.<database_name>.<table_name> value can't be empty.");
+      }
+      for (String value : values) {
+        if (!value.equalsIgnoreCase("*")) {
+          Float aFloatValue = Float.parseFloat(value);
+          if (aFloatValue < 0 || aFloatValue > Float.MAX_VALUE) {
+            throw new InvalidConfigurationException(
+                "carbon.input.segments.<database_name>.<table_name> value range should be greater "
+                    + "than 0 and less than " + Float.MAX_VALUE);
+          }
+        }
+      }
+    } catch (NumberFormatException nfe) {
+      throw new InvalidConfigurationException(
+          "carbon.input.segments.<database_name>.<table_name> value range is not valid");
+    }
+    return true;
+  }
   /**
    * Below method will be used to check whether bitset applied on previous filter
    * can be used to apply on next column filter

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fd0bdf6f/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 6d8c900..1878416 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
@@ -25,6 +25,7 @@ import org.apache.carbondata.common.constants.LoggerAction;
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.cache.CacheProvider;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.exception.InvalidConfigurationException;
 
 import static org.apache.carbondata.core.constants.CarbonCommonConstants.CARBON_CUSTOM_BLOCK_DISTRIBUTION;
@@ -144,8 +145,15 @@ public class SessionParams implements Serializable {
         isValid = true;
         break;
       default:
-        throw new InvalidConfigurationException(
-            "The key " + key + " not supported for dynamic configuration.");
+        if (key.startsWith(CarbonCommonConstants.CARBON_INPUT_SEGMENTS)) {
+          isValid = CarbonUtil.validateRangeOfSegmentList(value);
+          if (!isValid) {
+            throw new InvalidConfigurationException("Invalid CARBON_INPUT_SEGMENT_IDs");
+          }
+        } else {
+          throw new InvalidConfigurationException(
+              "The key " + key + " not supported for dynamic configuration.");
+        }
     }
     return isValid;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fd0bdf6f/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
index e22a5c6..57359fc 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
@@ -27,9 +27,11 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.BitSet;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datamap.DataMapStoreManager;
@@ -108,6 +110,8 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
   // comma separated list of input segment numbers
   public static final String INPUT_SEGMENT_NUMBERS =
       "mapreduce.input.carboninputformat.segmentnumbers";
+  public static final String VALIDATE_INPUT_SEGMENT_IDs =
+            "mapreduce.input.carboninputformat.validsegments";
   // comma separated list of input files
   public static final String INPUT_FILES = "mapreduce.input.carboninputformat.files";
   public static final String ALTER_PARTITION_ID = "mapreduce.input.carboninputformat.partitionid";
@@ -253,6 +257,21 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
   }
 
   /**
+   * set list of segment to access
+   */
+  public static void setValidateSegmentsToAccess(Configuration configuration, Boolean validate) {
+    configuration.set(CarbonTableInputFormat.VALIDATE_INPUT_SEGMENT_IDs, validate.toString());
+  }
+
+  /**
+   * get list of segment to access
+   */
+  public static boolean getValidateSegmentsToAccess(Configuration configuration) {
+    return configuration.get(CarbonTableInputFormat.VALIDATE_INPUT_SEGMENT_IDs, "true")
+        .equalsIgnoreCase("true");
+  }
+
+  /**
    * Set list of files to access
    */
   public static void setFilesToAccess(Configuration configuration, List<String> validFiles) {
@@ -280,24 +299,51 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
    */
   @Override public List<InputSplit> getSplits(JobContext job) throws IOException {
     AbsoluteTableIdentifier identifier = getAbsoluteTableIdentifier(job.getConfiguration());
+    SegmentUpdateStatusManager updateStatusManager = new SegmentUpdateStatusManager(identifier);
     TableDataMap blockletMap =
         DataMapStoreManager.getInstance().getDataMap(identifier, BlockletDataMap.NAME,
             BlockletDataMapFactory.class.getName());
     List<String> invalidSegments = new ArrayList<>();
     List<UpdateVO> invalidTimestampsList = new ArrayList<>();
-    List<String> validSegments = Arrays.asList(getSegmentsToAccess(job));
     List<String> streamSegments = null;
-    // get all valid segments and set them into the configuration
-    SegmentUpdateStatusManager updateStatusManager = new SegmentUpdateStatusManager(identifier);
-    if (validSegments.size() == 0) {
+    List<String> filteredSegmentToAccess = new ArrayList<>();
+    if (getValidateSegmentsToAccess(job.getConfiguration())) {
+
+      String[] segmentsToAccess = getSegmentsToAccess(job);
+      Set<String> segmentToAccessSet = new HashSet<>();
+      for (String segmentToAccess : segmentsToAccess) {
+        segmentToAccessSet.add(segmentToAccess);
+      }
+      // get all valid segments and set them into the configuration
       SegmentStatusManager segmentStatusManager = new SegmentStatusManager(identifier);
       SegmentStatusManager.ValidAndInvalidSegmentsInfo segments =
           segmentStatusManager.getValidAndInvalidSegments();
-      validSegments = segments.getValidSegments();
+      List<String> validSegments = segments.getValidSegments();
       streamSegments = segments.getStreamSegments();
       if (validSegments.size() == 0) {
         return getSplitsOfStreaming(job, identifier, streamSegments);
       }
+      if (segmentsToAccess.length == 0 || segmentsToAccess[0].equalsIgnoreCase("*")) {
+        filteredSegmentToAccess.addAll(validSegments);
+      } else {
+        for (String validSegment : validSegments) {
+          if (segmentToAccessSet.contains(validSegment)) {
+            filteredSegmentToAccess.add(validSegment);
+          }
+        }
+        if (!filteredSegmentToAccess.containsAll(segmentToAccessSet)) {
+          List<String> filteredSegmentToAccessTemp = new ArrayList<>();
+          filteredSegmentToAccessTemp.addAll(filteredSegmentToAccess);
+          filteredSegmentToAccessTemp.removeAll(segmentToAccessSet);
+          LOG.info(
+              "Segments ignored are : " + Arrays.toString(filteredSegmentToAccessTemp.toArray()));
+        }
+      }
+      if (filteredSegmentToAccess.size() == 0) {
+        return new ArrayList<>(0);
+      } else {
+        setSegmentsToAccess(job.getConfiguration(), filteredSegmentToAccess);
+      }
       // remove entry in the segment index if there are invalid segments
       invalidSegments.addAll(segments.getInvalidSegments());
       for (String invalidSegmentId : invalidSegments) {
@@ -320,7 +366,7 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
       }
     }
     // Clean segments if refresh is needed
-    for (String segment : validSegments) {
+    for (String segment : filteredSegmentToAccess) {
       if (DataMapStoreManager.getInstance().getTableSegmentRefresher(identifier)
           .isRefreshNeeded(segment)) {
         toBeCleanedSegments.add(segment);
@@ -357,7 +403,8 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
 
     // do block filtering and get split
     List<InputSplit> splits =
-        getSplits(job, filterInterface, validSegments, matchedPartitions, partitionInfo, null);
+        getSplits(job, filterInterface, filteredSegmentToAccess, matchedPartitions, partitionInfo,
+            null);
     // pass the invalid segment to task side in order to remove index entry in task side
     if (invalidSegments.size() > 0) {
       for (InputSplit split : splits) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fd0bdf6f/integration/spark-common-test/src/test/resources/data1.csv
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/resources/data1.csv b/integration/spark-common-test/src/test/resources/data1.csv
new file mode 100644
index 0000000..9813184
--- /dev/null
+++ b/integration/spark-common-test/src/test/resources/data1.csv
@@ -0,0 +1,11 @@
+empno,empname,designation,doj,workgroupcategory,workgroupcategoryname,deptno,deptname,projectcode,projectjoindate,projectenddate,attendance,utilization,salary
+101,arvind,SE,17-01-2007,1,developer,10,network,928478,17-02-2007,29-11-2016,96,96,5040
+120,krithin,SSE,29-05-2008,1,developer,11,protocol,928378,29-06-2008,30-12-2016,85,95,7124
+103,madhan,TPL,07-07-2009,2,tester,10,network,928478,07-08-2009,30-12-2016,88,99,9054
+140,anandh,SA,29-12-2010,3,manager,11,protocol,928278,29-01-2011,29-06-2016,77,92,11248
+15,anu,SSA,09-11-2011,1,developer,12,security,928375,09-12-2011,29-05-2016,99,91,13245
+160,pramod,SE,14-10-2012,1,developer,13,configManagement,928478,14-11-2012,29-12-2016,86,93,5040
+107,gawrav,PL,22-09-2013,2,tester,12,security,928778,22-10-2013,15-11-2016,78,97,9574
+181,sibi,TL,15-08-2014,2,tester,14,Learning,928176,15-09-2014,29-05-2016,84,98,7245
+119,shivani,PL,12-05-2015,1,developer,10,network,928977,12-06-2015,12-11-2016,88,91,11254
+210,bill,PM,01-12-2015,3,manager,14,Learning,928479,01-01-2016,30-11-2016,75,94,13547

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fd0bdf6f/integration/spark-common/src/main/scala/org/apache/carbondata/api/CarbonStore.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/api/CarbonStore.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/api/CarbonStore.scala
index e77f5c3..a95bc01 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/api/CarbonStore.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/api/CarbonStore.scala
@@ -66,11 +66,17 @@ object CarbonStore {
       loadMetadataDetailsSortedArray
           .filter(_.getVisibility.equalsIgnoreCase("true"))
           .map { load =>
+            val mergedTo = if (load.getMergedLoadName != null) {
+         load.getMergedLoadName
+       } else {
+         ""
+       }
             Row(
               load.getLoadName,
               load.getLoadStatus,
               new java.sql.Timestamp(load.getLoadStartTime),
-              new java.sql.Timestamp(load.getLoadEndTime)
+              new java.sql.Timestamp(load.getLoadEndTime),
+              mergedTo
             )
           }.toSeq
     } else {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fd0bdf6f/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
index 7ec6b7b..5e46417 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
@@ -349,6 +349,14 @@ class CarbonScanRDD(
         CarbonCommonConstants.USE_DISTRIBUTED_DATAMAP_DEFAULT).toBoolean) {
       CarbonTableInputFormat.setDataMapJob(conf, new SparkDataMapJob)
     }
+    val dbName = identifier.getCarbonTableIdentifier.getDatabaseName.toLowerCase
+    val tbName = identifier.getCarbonTableIdentifier.getTableName.toLowerCase
+    val segmentNumbersFromProperty = CarbonProperties.getInstance()
+      .getProperty(CarbonCommonConstants.CARBON_INPUT_SEGMENTS + dbName + "." + tbName, "*")
+    if (!segmentNumbersFromProperty.trim.equals("*")) {
+      CarbonTableInputFormat
+        .setSegmentsToAccess(conf, segmentNumbersFromProperty.split(",").toList.asJava)
+    }
     format
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fd0bdf6f/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
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fd0bdf6f/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala
index bf86aca..4acb82c 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala
@@ -85,7 +85,8 @@ case class ShowLoadsCommand(
     Seq(AttributeReference("SegmentSequenceId", StringType, nullable = false)(),
       AttributeReference("Status", StringType, nullable = false)(),
       AttributeReference("Load Start Time", TimestampType, nullable = false)(),
-      AttributeReference("Load End Time", TimestampType, nullable = false)())
+      AttributeReference("Load End Time", TimestampType, nullable = false)(),
+      AttributeReference("Merged To", StringType, nullable = false)())
   }
 }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fd0bdf6f/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/IUDCommonUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/IUDCommonUtil.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/IUDCommonUtil.scala
new file mode 100644
index 0000000..b18ab78
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/IUDCommonUtil.scala
@@ -0,0 +1,78 @@
+/*
+ * 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.mutation
+
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan}
+import org.apache.spark.sql.execution.datasources.LogicalRelation
+import org.apache.spark.sql.hive.HiveSessionCatalog
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
+
+/**
+ * Util for IUD common function
+ */
+object IUDCommonUtil {
+
+  /**
+   * iterates the plan  and check whether CarbonCommonConstants.CARBON_INPUT_SEGMENTS set for any
+   * any table or not
+   * @param sparkSession
+   * @param logicalPlan
+   */
+  def checkIfSegmentListIsSet(sparkSession: SparkSession, logicalPlan: LogicalPlan): Unit = {
+    val carbonProperties = CarbonProperties.getInstance()
+    logicalPlan.foreach {
+      case unresolvedRelation: UnresolvedRelation =>
+        val dbAndTb =
+          sparkSession.sessionState.catalog.asInstanceOf[HiveSessionCatalog].getCurrentDatabase +
+          "." + unresolvedRelation.tableIdentifier.table
+        val segmentProperties = carbonProperties
+          .getProperty(CarbonCommonConstants.CARBON_INPUT_SEGMENTS + dbAndTb, "")
+        if (!(segmentProperties.equals("") || segmentProperties.trim.equals("*"))) {
+          throw new MalformedCarbonCommandException("carbon.input.segments." + dbAndTb +
+                                                    "should not be set for table used in DELETE " +
+                                                    "query. Please reset the property to carbon" +
+                                                    ".input.segments." +
+                                                    dbAndTb + "=*")
+        }
+      case logicalRelation: LogicalRelation if (logicalRelation.relation
+        .isInstanceOf[CarbonDatasourceHadoopRelation]) =>
+        val dbAndTb =
+          logicalRelation.relation.asInstanceOf[CarbonDatasourceHadoopRelation].carbonTable
+            .getDatabaseName + "." +
+          logicalRelation.relation.asInstanceOf[CarbonDatasourceHadoopRelation].carbonTable
+            .getFactTableName
+        val sementProperty = carbonProperties
+          .getProperty(CarbonCommonConstants.CARBON_INPUT_SEGMENTS + dbAndTb, "")
+        if (!(sementProperty.equals("") || sementProperty.trim.equals("*"))) {
+          throw new MalformedCarbonCommandException("carbon.input.segments." + dbAndTb +
+                                                    "should not be set for table used in UPDATE " +
+                                                    "query. Please reset the property to carbon" +
+                                                    ".input.segments." +
+                                                    dbAndTb + "=*")
+        }
+      case filter: Filter => filter.subqueries.toList
+        .foreach(subquery => checkIfSegmentListIsSet(sparkSession, subquery))
+      case _ =>
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fd0bdf6f/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForDeleteCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForDeleteCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForDeleteCommand.scala
index 10e6785..af971d0 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForDeleteCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForDeleteCommand.scala
@@ -42,6 +42,7 @@ private[sql] case class ProjectForDeleteCommand(
 
   override def processData(sparkSession: SparkSession): Seq[Row] = {
     val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
+      IUDCommonUtil.checkIfSegmentListIsSet(sparkSession, plan)
     val dataFrame = Dataset.ofRows(sparkSession, plan)
     //    dataFrame.show(truncate = false)
     //    dataFrame.collect().foreach(println)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fd0bdf6f/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForUpdateCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForUpdateCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForUpdateCommand.scala
index 2088396..faeb3af 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForUpdateCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForUpdateCommand.scala
@@ -43,6 +43,7 @@ private[sql] case class ProjectForUpdateCommand(
 
   override def processData(sparkSession: SparkSession): Seq[Row] = {
     val LOGGER = LogServiceFactory.getLogService(ProjectForUpdateCommand.getClass.getName)
+    IUDCommonUtil.checkIfSegmentListIsSet(sparkSession, plan)
     val res = plan find {
       case relation: LogicalRelation if relation.relation
         .isInstanceOf[CarbonDatasourceHadoopRelation] =>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fd0bdf6f/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 1d8bb8a..56560fd 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
@@ -22,7 +22,9 @@ import org.apache.spark.sql.catalyst.TableIdentifier
 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.spark.exception.MalformedCarbonCommandException
 
 case class CarbonDropDatabaseCommand(command: DropDatabaseCommand)
   extends RunnableCommand {
@@ -61,6 +63,16 @@ case class CarbonSetCommand(command: SetCommand)
         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.")
+          }
+        }
       case _ =>
 
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fd0bdf6f/integration/spark2/src/main/scala/org/apache/spark/sql/internal/CarbonSqlConf.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/internal/CarbonSqlConf.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/internal/CarbonSqlConf.scala
index 51b29a1..6c91e7e 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/internal/CarbonSqlConf.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/internal/CarbonSqlConf.scala
@@ -101,6 +101,11 @@ class CarbonSQLConf(sparkSession: SparkSession) {
         .doc("Property to configure data format for date type columns.")
         .stringConf
         .createWithDefault(CarbonLoadOptionConstants.CARBON_OPTIONS_DATEFORMAT_DEFAULT)
+    val CARBON_INPUT_SEGMENTS = SQLConfigBuilder(
+      "carbon.input.segments.<database_name>.<table_name>")
+      .doc("Property to configure the list of segments to query.").stringConf
+      .createWithDefault(carbonProperties
+        .getProperty("carbon.input.segments.<database_name>.<table_name>", "*"))
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fd0bdf6f/integration/spark2/src/main/scala/org/apache/spark/util/ShowSegments.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/util/ShowSegments.scala b/integration/spark2/src/main/scala/org/apache/spark/util/ShowSegments.scala
index 19d7dce..b98973e 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/util/ShowSegments.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/util/ShowSegments.scala
@@ -40,9 +40,9 @@ object ShowSegments {
   def showString(rows: Seq[Row]): String = {
     val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.s")
     val sb = new StringBuilder
-    sb.append("+------------+------------------+----------------------+----------------------+\n")
-      .append("|SegmentId   |Status            |Load Start Time       |Load End Time         |\n")
-      .append("+------------+------------------+----------------------+----------------------+\n")
+    sb.append("+------------+------------------+----------------------+----------------------+----------------------+\n")
+      .append("|SegmentId   |Status            |Load Start Time       |Load End Time         |Merged To             |\n")
+      .append("+------------+------------------+----------------------+----------------------+----------------------+\n")
       rows.foreach{row =>
         sb.append("|")
           .append(StringUtils.rightPad(row.getString(0), 12))
@@ -52,9 +52,11 @@ object ShowSegments {
           .append(sdf.format(row.getAs[java.sql.Timestamp](2)))
           .append("|")
           .append(sdf.format(row.getAs[java.sql.Timestamp](3)))
+          .append("|")
+          .append(StringUtils.rightPad(row.getString(4), 18))
           .append("|\n")
       }
-    sb.append("+------------+------------------+----------------------+----------------------+\n")
+    sb.append("+------------+------------------+----------------------+----------------------+----------------------+\n")
     sb.toString
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fd0bdf6f/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/segmentreading/TestSegmentReading.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/segmentreading/TestSegmentReading.scala b/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/segmentreading/TestSegmentReading.scala
new file mode 100644
index 0000000..8309b34
--- /dev/null
+++ b/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/segmentreading/TestSegmentReading.scala
@@ -0,0 +1,352 @@
+package org.apache.carbondata.spark.testsuite.segmentreading
+
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.test.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
+
+/**
+ * Created by rahul on 19/9/17.
+ */
+class TestSegmentReading extends QueryTest with BeforeAndAfterAll {
+
+  override def beforeAll(): Unit = {
+    sql("drop table if exists carbon_table")
+    sql(
+      "create table carbon_table(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 OPTIONS
+          |('DELIMITER'= ',', 'QUOTECHAR'= '\"')""".stripMargin)
+    sql(
+      s"""LOAD DATA local inpath '$resourcesPath/data1.csv' INTO TABLE carbon_table OPTIONS
+          |('DELIMITER'= ',', 'QUOTECHAR'= '\"')""".stripMargin)
+  }
+
+  test("test SET -V for segment reading property") {
+    try {
+      checkExistence(sql("SET -v"), true, "Property to configure the list of segments to query.")
+    }
+    finally {
+      sql("SET carbon.input.segments.default.carbon_table=*")
+    }
+  }
+  test("test count(*) for segment reading property") {
+    try {
+      sql("SET carbon.input.segments.default.carbon_table=1")
+      checkAnswer(sql("select count(*) from carbon_table"), Seq(Row(10)))
+    }
+    finally {
+      sql("SET carbon.input.segments.default.carbon_table=*")
+    }
+  }
+  test("test SET propertyname for segment reading property") {
+    try {
+      sql("SET carbon.input.segments.default.carbon_table=1")
+      checkAnswer(sql("SET carbon.input.segments.default.carbon_table"),
+        Seq(Row("carbon.input.segments.default.carbon_table", "1"))
+      )
+    }
+    finally {
+      sql("SET carbon.input.segments.default.carbon_table=*")
+    }
+  }
+  test("set valid segments and query from table") {
+    try {
+      checkAnswer(sql("select count(*) from carbon_table"), Seq(Row(20)))
+      sql("SET carbon.input.segments.default.carbon_table=1")
+      checkAnswer(sql("select count(*) from carbon_table"), Seq(Row(10)))
+      sql("SET carbon.input.segments.default.carbon_table=*")
+      checkAnswer(sql("select count(*) from carbon_table"), Seq(Row(20)))
+      sql("SET carbon.input.segments.default.carbon_table=0")
+      checkAnswer(sql("select count(*) from carbon_table"), Seq(Row(10)))
+    }
+    finally {
+      sql("SET carbon.input.segments.default.carbon_table=*")
+    }
+  }
+  test("test Multiple times set segment") {
+    try {
+      sql("SET carbon.input.segments.default.carbon_table=0")
+      checkAnswer(sql("select count(*) from carbon_table"), Seq(Row(10)))
+      sql("SET carbon.input.segments.default.carbon_table=1")
+      checkAnswer(sql("select count(*) from carbon_table"), Seq(Row(10)))
+      sql("SET carbon.input.segments.default.carbon_table=1,0,1")
+      checkAnswer(sql("select count(*) from carbon_table"), Seq(Row(20)))
+      sql("SET carbon.input.segments.default.carbon_table=2,0")
+      checkAnswer(sql("select count(*) from carbon_table"), Seq(Row(10)))
+      val trapped = intercept[Exception] {
+        sql("SET carbon.input.segments.default.carbon_table=2,a")
+      }
+      val trappedAgain = intercept[Exception] {
+        sql("SET carbon.input.segments.default.carbon_table=,")
+      }
+      assert(trapped.getMessage
+        .equalsIgnoreCase(
+          "carbon.input.segments.<database_name>.<table_name> value range is not valid"))
+      assert(trappedAgain.getMessage
+        .equalsIgnoreCase("carbon.input.segments.<database_name>.<table_name> value can't be empty."))
+    }
+    finally {
+      sql("SET carbon.input.segments.default.carbon_table=*")
+    }
+  }
+  test("test filter with segment reading"){
+    try {
+      sql("SET carbon.input.segments.default.carbon_table=*")
+      checkAnswer(sql("select count(empno) from carbon_table where empno = 15"),Seq(Row(2)))
+      sql("SET carbon.input.segments.default.carbon_table=1")
+      checkAnswer(sql("select count(empno) from carbon_table where empno = 15"),Seq(Row(1)))
+    }
+    finally {
+      sql("SET carbon.input.segments.default.carbon_table=*")
+    }
+  }
+  test("test group by with segment reading") {
+    try {
+      sql("SET carbon.input.segments.default.carbon_table=*")
+      checkAnswer(sql("select empno,count(empname) from carbon_table where empno = 15 group by empno"),Seq(Row(15,2)))
+      sql("SET carbon.input.segments.default.carbon_table=1")
+      checkAnswer(sql("select empno,count(empname) from carbon_table where empno = 15 group by empno"),Seq(Row(15,1)))
+    }
+    finally {
+      sql("SET carbon.input.segments.default.carbon_table=*")
+    }
+  }
+  test("test join with segment reading"){
+    try {
+      sql("SET carbon.input.segments.default.carbon_table=*")
+      sql("drop table if exists carbon_table_join")
+      sql(
+        "create table carbon_table_join(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("insert into carbon_table_join select * from carbon_table").show()
+      checkAnswer(sql("select count(a.empno) from carbon_table a inner join carbon_table_join b on a.empno = b.empno"),Seq(Row(22)))
+      sql("SET carbon.input.segments.default.carbon_table=1")
+      checkAnswer(sql("select count(a.empno) from carbon_table a inner join carbon_table_join b on a.empno = b.empno"),Seq(Row(11)))
+    }
+    finally {
+      sql("SET carbon.input.segments.default.carbon_table=*")
+    }
+  }
+  test("test aggregation with segment reading") {
+    try {
+      sql("SET carbon.input.segments.default.carbon_table=*")
+      checkAnswer(sql("select sum(empno) from carbon_table"), Seq(Row(1411)))
+      sql("SET carbon.input.segments.default.carbon_table=1")
+      checkAnswer(sql("select sum(empno) from carbon_table"), Seq(Row(1256)))
+    }
+    finally {
+      sql("SET carbon.input.segments.default.carbon_table=*")
+    }
+  }
+  test("test update query with segment reading"){
+    try {
+      sql("drop table if exists carbon_table_update")
+      sql(
+        "create table carbon_table_update(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_update OPTIONS
+            |('DELIMITER'= ',', 'QUOTECHAR'= '\"')""".stripMargin)
+      sql(
+        s"""LOAD DATA local inpath '$resourcesPath/data1.csv' INTO TABLE carbon_table_update OPTIONS
+            |('DELIMITER'= ',', 'QUOTECHAR'= '\"')""".stripMargin)
+      sql("SET carbon.input.segments.default.carbon_table=1")
+      intercept[MalformedCarbonCommandException]{
+        sql("update carbon_table_update a set(a.empname) = (select b.empname from carbon_table b where a.empno=b.empno)").show()
+      }
+      sql("SET carbon.input.segments.default.carbon_table=*")
+      sql("SET carbon.input.segments.default.carbon_table_update=1")
+      intercept[MalformedCarbonCommandException]{
+        sql("update carbon_table_update a set(a.empname) = (select b.empname from carbon_table b where a.empno=b.empno)").show()
+      }
+      sql("SET carbon.input.segments.default.carbon_table=*")
+      sql("SET carbon.input.segments.default.carbon_table_update=*")
+      checkAnswer(sql("select count(*) from carbon_table_update where empname='rahul'"), Seq(Row(0)))
+      sql("update carbon_table_update a set(a.empname) = ('rahul')").show()
+      checkAnswer(sql("select count(*) from carbon_table_update where empname='rahul'"), Seq(Row(20)))
+    }
+    finally {
+      sql("SET carbon.input.segments.default.carbon_table=*")
+    }
+  }
+  test("test delete query with segment reading"){
+    try {
+      sql("drop table if exists carbon_table_delete")
+      sql(
+        "create table carbon_table_delete(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_delete OPTIONS
+            |('DELIMITER'= ',', 'QUOTECHAR'= '\"')""".stripMargin)
+      sql(
+        s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE carbon_table_delete OPTIONS
+            |('DELIMITER'= ',', 'QUOTECHAR'= '\"')""".stripMargin)
+      sql("SET carbon.input.segments.default.carbon_table=*")
+      sql("SET carbon.input.segments.default.carbon_table=1")
+      intercept[MalformedCarbonCommandException]{
+        sql("delete from carbon_table_delete where empno IN (select empno from carbon_table where empname='ayushi')").show()
+      }
+      sql("SET carbon.input.segments.default.carbon_table_delete=1")
+      intercept[MalformedCarbonCommandException]{
+        sql("delete from carbon_table_delete where empno IN (select empno from carbon_table where empname='ayushi')").show()
+      }
+
+      sql("SET carbon.input.segments.default.carbon_table=*")
+      sql("SET carbon.input.segments.default.carbon_table_delete=*")
+      checkAnswer(sql("select count(*) from carbon_table_delete"), Seq(Row(20)))
+      sql("delete from carbon_table_delete where empno IN (select empno from carbon_table where empname='ayushi')").show()
+      checkAnswer(sql("select count(*) from carbon_table_delete"), Seq(Row(18)))
+    }
+    finally {
+      sql("SET carbon.input.segments.default.carbon_table=*")
+    }
+  }
+  test("test show segments"){
+    try {
+      sql("drop table if exists carbon_table_show_seg")
+      sql(
+        "create table carbon_table_show_seg(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_show_seg OPTIONS
+            |('DELIMITER'= ',', 'QUOTECHAR'= '\"')""".stripMargin)
+      sql(
+        s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE carbon_table_show_seg OPTIONS
+            |('DELIMITER'= ',', 'QUOTECHAR'= '\"')""".stripMargin)
+      sql("alter table carbon_table_show_seg compact 'major'")
+      sql(
+        s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE carbon_table_show_seg OPTIONS
+            |('DELIMITER'= ',', 'QUOTECHAR'= '\"')""".stripMargin)
+      val df = sql("SHOW SEGMENTS for table carbon_table_show_seg")
+      val col = df.collect().map{
+        row => Row(row.getString(0),row.getString(1),row.getString(4))
+      }.toSeq
+      assert(col.equals(Seq(Row("2","Success",""),
+        Row("1","Compacted","0.1"),
+        Row("0.1","Success",""),
+        Row("0","Compacted","0.1"))))
+    }
+    finally {
+      sql("SET carbon.input.segments.default.carbon_table=*")
+    }
+  }
+  test("test segment reading after compaction"){
+    sql("drop table if exists carbon_table_compact")
+    sql(
+      "create table carbon_table_compact(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_compact OPTIONS
+          |('DELIMITER'= ',', 'QUOTECHAR'= '\"')""".stripMargin)
+    sql(
+      s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE carbon_table_compact OPTIONS
+          |('DELIMITER'= ',', 'QUOTECHAR'= '\"')""".stripMargin)
+    sql("alter table carbon_table_compact compact 'major'")
+    sql(
+      s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE carbon_table_compact OPTIONS
+          |('DELIMITER'= ',', 'QUOTECHAR'= '\"')""".stripMargin)
+    checkAnswer(sql("select count(*) from carbon_table_compact"),Seq(Row(30)))
+    sql(" SET carbon.input.segments.default.carbon_table_compact=0.1")
+    checkAnswer(sql("select count(*) from carbon_table_compact"),Seq(Row(20)))
+  }
+  test("set segment id then alter table name and check select query") {
+    try {
+      sql("drop table if exists carbon_table_alter")
+      sql("drop table if exists carbon_table_alter_new")
+      sql(
+        "create table carbon_table_alter(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_alter OPTIONS
+            |('DELIMITER'= ',', 'QUOTECHAR'= '\"')""".
+          stripMargin)
+      sql(
+        s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE carbon_table_alter OPTIONS
+            |('DELIMITER'= ',', 'QUOTECHAR'= '\"')""".
+          stripMargin)
+      checkAnswer(sql("select count(*) from carbon_table_alter"),
+        Seq(Row(20)))
+      sql(
+        "SET carbon.input.segments.default.carbon_table_alter=1")
+      checkAnswer(sql(
+        "select count(*) from carbon_table_alter"), Seq(Row(10)))
+      sql(
+        "alter table carbon_table_alter rename to carbon_table_alter_new")
+      checkAnswer(sql(
+        "select count(*) from carbon_table_alter_new")
+        , Seq(Row(20)))
+    }
+    finally {
+      sql(
+        "SET carbon.input.segments.default.carbon_table=*")
+    }
+  }
+
+  test("drop and recreate table to check segment reading") {
+    try {
+      sql("drop table if exists carbon_table_recreate")
+      sql(
+        "create table carbon_table_recreate(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_recreate OPTIONS
+            |('DELIMITER'= ',', 'QUOTECHAR'= '\"')""".
+          stripMargin)
+      sql(
+        s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE carbon_table_recreate OPTIONS
+            |('DELIMITER'= ',', 'QUOTECHAR'= '\"')""".
+          stripMargin)
+      checkAnswer(sql("select count(*) from carbon_table_recreate"),
+        Seq(Row(20)))
+      sql(
+        "SET carbon.input.segments.default.carbon_table_recreate=1")
+      checkAnswer(sql(
+        "select count(*) from carbon_table_recreate"), Seq(Row(10)))
+      sql("drop table if exists carbon_table_recreate")
+      sql(
+        "create table carbon_table_recreate(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_recreate OPTIONS
+            |('DELIMITER'= ',', 'QUOTECHAR'= '\"')""".
+          stripMargin)
+      sql(
+        s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE carbon_table_recreate OPTIONS
+            |('DELIMITER'= ',', 'QUOTECHAR'= '\"')""".
+          stripMargin)
+      checkAnswer(sql(
+        "select count(*) from carbon_table_recreate"), Seq(Row(10)))
+    }
+    finally {
+      sql(
+        "SET carbon.input.segments.default.carbon_table=*")
+    }
+  }
+}


[16/49] carbondata git commit: [CARBONDATA-1618] Fix issue of not support table comment

Posted by ra...@apache.org.
[CARBONDATA-1618] Fix issue of not support table comment

Background: Current carbon do not support table comment when create table.
This PR will support table comment.

This closes #1437


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

Branch: refs/heads/fgdatamap
Commit: ae280e239aa76ff547ceedfdb8fb031ae5af078e
Parents: 0578ba0
Author: chenerlu <ch...@huawei.com>
Authored: Thu Oct 26 11:12:42 2017 +0800
Committer: ravipesala <ra...@gmail.com>
Committed: Tue Nov 7 19:10:24 2017 +0530

----------------------------------------------------------------------
 .../core/metadata/schema/table/TableInfo.java   | 11 +++
 .../TestCreateTableWithTableComment.scala       | 75 ++++++++++++++++++++
 .../spark/sql/catalyst/CarbonDDLSqlParser.scala |  6 +-
 .../command/carbonTableSchemaCommon.scala       |  6 +-
 .../CarbonDescribeFormattedCommand.scala        |  2 +
 .../spark/sql/parser/CarbonSparkSqlParser.scala |  5 +-
 6 files changed, 101 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/ae280e23/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 0c807f6..717eada 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
@@ -79,6 +79,9 @@ 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;
 
+  // table comment
+  private String tableComment;
+
   public TableInfo() {
   }
 
@@ -160,6 +163,14 @@ public class TableInfo implements Serializable, Writable {
     this.storePath = storePath;
   }
 
+  public String getTableComment() {
+    return tableComment;
+  }
+
+  public void setTableComment(String tableComment) {
+    this.tableComment = tableComment;
+  }
+
   /**
    * to generate the hash code
    */

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ae280e23/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableWithTableComment.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableWithTableComment.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableWithTableComment.scala
new file mode 100644
index 0000000..86b8327
--- /dev/null
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableWithTableComment.scala
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.spark.testsuite.createTable
+
+import org.apache.spark.sql.test.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+/**
+ * test functionality for create table with table comment
+ */
+class TestCreateTableWithTableComment extends QueryTest with BeforeAndAfterAll {
+
+  override def beforeAll {
+    sql("use default")
+    sql("drop table if exists withTableComment")
+    sql("drop table if exists withoutTableComment")
+  }
+
+  test("test create table with table comment") {
+    sql(
+      s"""
+         | create table withTableComment(
+         | id int,
+         | name string
+         | )
+         | comment "This table has table comment"
+         | STORED BY 'carbondata'
+       """.stripMargin
+    )
+
+    val result = sql("describe formatted withTableComment")
+
+    checkExistence(result, true, "Comment:")
+    checkExistence(result, true, "This table has table comment")
+  }
+
+  test("test create table without table comment") {
+    sql(
+      s"""
+         | create table withoutTableComment(
+         | id int,
+         | name string
+         | )
+         | STORED BY 'carbondata'
+       """.stripMargin
+    )
+
+    val result = sql("describe formatted withoutTableComment")
+
+    checkExistence(result, true, "Comment:")
+    checkExistence(result, false, "This table has table comment")
+  }
+
+  override def afterAll: Unit = {
+    sql("use default")
+    sql("drop table if exists withTableComment")
+    sql("drop table if exists withoutTableComment")
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ae280e23/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 edb471e..aae4f25 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
@@ -246,7 +246,8 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
       , tableName: String, fields: Seq[Field],
       partitionCols: Seq[PartitionerField],
       tableProperties: mutable.Map[String, String],
-      bucketFields: Option[BucketFields], isAlterFlow: Boolean = false): TableModel = {
+      bucketFields: Option[BucketFields], isAlterFlow: Boolean = false,
+      tableComment: Option[String] = None): TableModel = {
 
     fields.zipWithIndex.foreach { case (field, index) =>
       field.schemaOrdinal = index
@@ -286,7 +287,8 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
       groupCols,
       Some(colProps),
       bucketFields: Option[BucketFields],
-      partitionInfo)
+      partitionInfo,
+      tableComment)
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ae280e23/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 285abf4..fba3085 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
@@ -60,7 +60,8 @@ case class TableModel(
     colProps: Option[util.Map[String,
     util.List[ColumnProperty]]] = None,
     bucketFields: Option[BucketFields],
-    partitionInfo: Option[PartitionInfo])
+    partitionInfo: Option[PartitionInfo],
+    tableComment: Option[String] = None)
 
 case class Field(column: String, var dataType: Option[String], name: Option[String],
     children: Option[List[Field]], parent: String = null,
@@ -510,6 +511,8 @@ class TableNewProcessor(cm: TableModel) {
     cm.tableProperties.foreach {
       x => tablePropertiesMap.put(x._1, x._2)
     }
+    // Add table comment to table properties
+    tablePropertiesMap.put("comment", cm.tableComment.getOrElse(""))
     tableSchema.setTableProperties(tablePropertiesMap)
     if (cm.bucketFields.isDefined) {
       val bucketCols = cm.bucketFields.get.bucketColumns.map { b =>
@@ -548,6 +551,7 @@ class TableNewProcessor(cm: TableModel) {
     tableInfo.setTableUniqueName(cm.databaseName + "_" + cm.tableName)
     tableInfo.setLastUpdatedTime(System.currentTimeMillis())
     tableInfo.setFactTable(tableSchema)
+    tableInfo.setTableComment(cm.tableComment.getOrElse(""))
     tableInfo
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ae280e23/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonDescribeFormattedCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonDescribeFormattedCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonDescribeFormattedCommand.scala
index e5f6b75..e57f490 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonDescribeFormattedCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonDescribeFormattedCommand.scala
@@ -105,6 +105,8 @@ private[sql] case class CarbonDescribeFormattedCommand(
     results ++= Seq(("Table Name: ", relation.tableMeta.carbonTableIdentifier.getTableName, ""))
     results ++= Seq(("CARBON Store Path: ", relation.tableMeta.storePath, ""))
     val carbonTable = relation.tableMeta.carbonTable
+    // Carbon table support table comment
+    results ++= Seq(("Comment: ", carbonTable.getTableInfo.getTableComment, ""))
     results ++= Seq(("Table Block Size : ", carbonTable.getBlockSizeInMB + " MB", ""))
     results ++= Seq(("SORT_SCOPE", carbonTable.getTableInfo.getFactTable
       .getTableProperties.getOrDefault("sort_scope", CarbonCommonConstants

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ae280e23/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 52008f2..81ce73f 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
@@ -103,6 +103,7 @@ class CarbonSqlAstBuilder(conf: SQLConf) extends SparkSqlAstBuilder(conf) {
       val partitionerFields = partitionByStructFields.map { structField =>
         PartitionerField(structField.name, Some(structField.dataType.toString), null)
       }
+      val tableComment = Option(ctx.STRING()).map(string)
       val cols = Option(ctx.columns).toSeq.flatMap(visitColTypeList)
       val properties = Option(ctx.tablePropertyList).map(visitPropertyKeyValues)
         .getOrElse(Map.empty)
@@ -144,7 +145,9 @@ class CarbonSqlAstBuilder(conf: SQLConf) extends SparkSqlAstBuilder(conf) {
         fields,
         partitionerFields,
         tableProperties,
-        bucketFields)
+        bucketFields,
+        false,
+        tableComment)
 
       CarbonCreateTableCommand(tableModel)
     } else {


[26/49] carbondata git commit: [CARBONDATA-1660] Fixed bug related to filter of data based on decimal values in presto

Posted by ra...@apache.org.
[CARBONDATA-1660] Fixed bug related to filter of data based on decimal values in presto

1.Fixed bug related to filter of data based on decimal values in presto, 2.Added related test cases.

This closes #1463


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

Branch: refs/heads/fgdatamap
Commit: f3b507cb78f74e58299750ac9b1dd3c990050adb
Parents: d7393da
Author: Geetika Gupta <ge...@knoldus.in>
Authored: Fri Nov 3 13:08:50 2017 +0530
Committer: chenliang613 <ch...@huawei.com>
Committed: Thu Nov 9 19:35:53 2017 +0800

----------------------------------------------------------------------
 .../carbondata/presto/PrestoFilterUtil.java     | 13 +++---------
 .../presto/src/test/resources/alldatatype.csv   | 22 ++++++++++----------
 .../integrationtest/PrestoAllDataTypeTest.scala | 20 +++++++++++++++++-
 .../presto/util/CarbonDataStoreCreator.scala    | 15 ++++++++++++-
 4 files changed, 47 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/f3b507cb/integration/presto/src/main/java/org/apache/carbondata/presto/PrestoFilterUtil.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/PrestoFilterUtil.java b/integration/presto/src/main/java/org/apache/carbondata/presto/PrestoFilterUtil.java
index b8b2032..ded4889 100644
--- a/integration/presto/src/main/java/org/apache/carbondata/presto/PrestoFilterUtil.java
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/PrestoFilterUtil.java
@@ -27,6 +27,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.stream.Collectors;
 
+import com.facebook.presto.spi.type.*;
 import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.scan.expression.ColumnExpression;
@@ -46,16 +47,6 @@ import com.facebook.presto.spi.ColumnHandle;
 import com.facebook.presto.spi.predicate.Domain;
 import com.facebook.presto.spi.predicate.Range;
 import com.facebook.presto.spi.predicate.TupleDomain;
-import com.facebook.presto.spi.type.BigintType;
-import com.facebook.presto.spi.type.BooleanType;
-import com.facebook.presto.spi.type.DateType;
-import com.facebook.presto.spi.type.DecimalType;
-import com.facebook.presto.spi.type.DoubleType;
-import com.facebook.presto.spi.type.IntegerType;
-import com.facebook.presto.spi.type.SmallintType;
-import com.facebook.presto.spi.type.TimestampType;
-import com.facebook.presto.spi.type.Type;
-import com.facebook.presto.spi.type.VarcharType;
 import com.google.common.collect.ImmutableList;
 import io.airlift.slice.Slice;
 
@@ -260,6 +251,8 @@ public class PrestoFilterUtil {
       } else if (rawdata instanceof  Long) {
         return new BigDecimal(new BigInteger(String.valueOf(rawdata)),
             ((DecimalType) type).getScale());
+      } else if(rawdata instanceof Slice) {
+        return new BigDecimal(Decimals.decodeUnscaledValue((Slice) rawdata), ((DecimalType) type).getScale());
       }
     } else if (type.equals(TimestampType.TIMESTAMP)) {
       return (Long)rawdata * 1000;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f3b507cb/integration/presto/src/test/resources/alldatatype.csv
----------------------------------------------------------------------
diff --git a/integration/presto/src/test/resources/alldatatype.csv b/integration/presto/src/test/resources/alldatatype.csv
index 6b0259a..ba6156e 100644
--- a/integration/presto/src/test/resources/alldatatype.csv
+++ b/integration/presto/src/test/resources/alldatatype.csv
@@ -1,11 +1,11 @@
-ID,date,country,name,phonetype,serialname,salary,bonus,dob,shortfield
-1,2015-07-23,china,anubhav,phone197,ASD69643,5000000.00,1234.444,2016-04-14 15/00/09,10
-2,2015-07-24,china,jatin,phone756,ASD42892,150010.999,1234.5555,2016-04-14 15:00:09,10
-3,2015-07-25,china,liang,phone1904,ASD37014,15002.110,600.777,2016-01-14 15:07:09,8
-4,2015-07-26,china,prince,phone2435,ASD66902,15003.00,9999.999,1992-04-14 13:00:09,4
-5,2015-07-27,china,bhavya,phone2441,ASD90633,15004.00,5000.999,2010-06-19 14:10:06,11
-6,2015-07-28,china,akash,phone294,ASD59961,15005.00,500.59,2013-07-19 12:10:08,18
-7,2015-07-29,china,sahil,phone610,ASD14875,15006.00,500.99,,2007-04-19 11:10:06,17
-8,2015-07-30,china,geetika,phone1848,ASD57308,15007.500,500.88,2008-09-21 11:10:06,10
-9,2015-07-18,china,ravindra,phone706,ASD86717,15008.00,700.999,2009-06-19 15:10:06,1
-9,2015/07/18,china,jitesh,phone706,ASD86717,15008.00,500.414,2001-08-29 13:09:03,12
+ID,date,country,name,phonetype,serialname,salary,bonus,monthlyBonus,dob,shortfield
+1,2015-07-23,china,anubhav,phone197,ASD69643,5000000.00,1234.444,12.1234,2016-04-14 15/00/09,10
+2,2015-07-24,china,jatin,phone756,ASD42892,150010.999,1234.5555,15.13,2016-04-14 15:00:09,10
+3,2015-07-25,china,liang,phone1904,ASD37014,15002.110,600.777,16.181,2016-01-14 15:07:09,8
+4,2015-07-26,china,prince,phone2435,ASD66902,15003.00,9999.999,17.3654,1992-04-14 13:00:09,4
+5,2015-07-27,china,bhavya,phone2441,ASD90633,15004.00,5000.999,12.11,2010-06-19 14:10:06,11
+6,2015-07-28,china,akash,phone294,ASD59961,15005.00,500.59,18.65,2013-07-19 12:10:08,18
+7,2015-07-29,china,sahil,phone610,ASD14875,15006.00,500.99,,19.65,2007-04-19 11:10:06,17
+8,2015-07-30,china,geetika,phone1848,ASD57308,15007.500,500.88,200.97,2008-09-21 11:10:06,10
+9,2015-07-18,china,ravindra,phone706,ASD86717,15008.00,700.999,45.25,2009-06-19 15:10:06,1
+9,2015/07/18,china,jitesh,phone706,ASD86717,15008.00,500.414,11.655,2001-08-29 13:09:03,12

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f3b507cb/integration/presto/src/test/scala/org/apache/carbondata/presto/integrationtest/PrestoAllDataTypeTest.scala
----------------------------------------------------------------------
diff --git a/integration/presto/src/test/scala/org/apache/carbondata/presto/integrationtest/PrestoAllDataTypeTest.scala b/integration/presto/src/test/scala/org/apache/carbondata/presto/integrationtest/PrestoAllDataTypeTest.scala
index 5eb06e7..dc0472d 100644
--- a/integration/presto/src/test/scala/org/apache/carbondata/presto/integrationtest/PrestoAllDataTypeTest.scala
+++ b/integration/presto/src/test/scala/org/apache/carbondata/presto/integrationtest/PrestoAllDataTypeTest.scala
@@ -403,10 +403,28 @@ class PrestoAllDataTypeTest extends FunSuiteLike with BeforeAndAfterAll {
     assert(actualResult.equals(expectedResult))
   }
 
+  test("test longDecimal type of presto") {
+    val actualResult: List[Map[String, Any]] = PrestoServer
+      .executeQuery(
+        "SELECT ID from testdb.testtable WHERE bonus = DECIMAL '1234.5555'")
+    val expectedResult: List[Map[String, Any]] = List(Map("ID" -> 2))
+
+    assert(actualResult.equals(expectedResult))
+  }
+
+  test("test shortDecimal type of presto") {
+    val actualResult: List[Map[String, Any]] = PrestoServer
+      .executeQuery(
+        "SELECT ID from testdb.testtable WHERE monthlyBonus = 15.13")
+    val expectedResult: List[Map[String, Any]] = List(Map("ID" -> 2))
+
+    assert(actualResult.equals(expectedResult))
+  }
+
   test("test timestamp datatype using cast operator") {
     val actualResult: List[Map[String, Any]] = PrestoServer
       .executeQuery("SELECT NAME AS RESULT FROM TESTDB.TESTTABLE WHERE DOB = CAST('2016-04-14 15:00:09' AS TIMESTAMP)")
     val expectedResult: List[Map[String, Any]] = List(Map("RESULT" -> "jatin"))
     assert(actualResult.equals(expectedResult))
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f3b507cb/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala
----------------------------------------------------------------------
diff --git a/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala b/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala
index 09cddfe..e932213 100644
--- a/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala
+++ b/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala
@@ -124,7 +124,7 @@ object CarbonDataStoreCreator {
         "true")
       loadModel.setMaxColumns("15")
       loadModel.setCsvHeader(
-        "ID,date,country,name,phonetype,serialname,salary,bonus,dob,shortField")
+        "ID,date,country,name,phonetype,serialname,salary,bonus,monthlyBonus,dob,shortField")
       loadModel.setCsvHeaderColumns(loadModel.getCsvHeader.split(","))
       loadModel.setTaskNo("0")
       loadModel.setSegmentId("0")
@@ -245,6 +245,19 @@ object CarbonDataStoreCreator {
     bonus.setColumnReferenceId(bonus.getColumnUniqueId)
     columnSchemas.add(bonus)
 
+    val monthlyBonus: ColumnSchema = new ColumnSchema()
+    monthlyBonus.setColumnName("monthlyBonus")
+    monthlyBonus.setColumnar(true)
+    monthlyBonus.setDataType(DataTypes.createDecimalType(18, 4))
+    monthlyBonus.setPrecision(18)
+    monthlyBonus.setScale(4)
+    monthlyBonus.setEncodingList(encodings)
+    monthlyBonus.setColumnUniqueId(UUID.randomUUID().toString)
+    monthlyBonus.setDimensionColumn(false)
+    monthlyBonus.setColumnGroup(8)
+    monthlyBonus.setColumnReferenceId(monthlyBonus.getColumnUniqueId)
+    columnSchemas.add(monthlyBonus)
+
     val dob: ColumnSchema = new ColumnSchema()
     dob.setColumnName("dob")
     dob.setColumnar(true)


[17/49] carbondata git commit: [CARBONDATA-1611][Streaming] Reject Update and Delete operation for streaming table

Posted by ra...@apache.org.
[CARBONDATA-1611][Streaming] Reject Update and Delete operation for streaming table

This closes #1447


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

Branch: refs/heads/fgdatamap
Commit: 74bd52b66f7dae938c1d993e5dc3a7a225227866
Parents: ae280e2
Author: Jacky Li <ja...@qq.com>
Authored: Sun Oct 29 21:31:21 2017 +0530
Committer: QiangCai <qi...@qq.com>
Committed: Tue Nov 7 21:58:48 2017 +0800

----------------------------------------------------------------------
 .../core/metadata/schema/table/CarbonTable.java |  8 +++
 .../mutation/ProjectForUpdateCommand.scala      | 12 +---
 .../strategy/StreamingTableStrategy.scala       | 62 ++++++++++++++++++++
 .../spark/sql/hive/CarbonSessionState.scala     |  8 ++-
 .../TestStreamingTableOperation.scala           | 59 +++++++++++++++++++
 5 files changed, 136 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/74bd52b6/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 a6738a3..e1a7143 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
@@ -657,4 +657,12 @@ public class CarbonTable implements Serializable {
   public TableInfo getTableInfo() {
     return tableInfo;
   }
+
+  /**
+   * Return true if this is a streaming table (table with property "streaming"="true")
+   */
+  public boolean isStreamingTable() {
+    String streaming = getTableInfo().getFactTable().getTableProperties().get("streaming");
+    return streaming != null && streaming.equalsIgnoreCase("true");
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74bd52b6/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForUpdateCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForUpdateCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForUpdateCommand.scala
index 5e9d31f..2088396 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForUpdateCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForUpdateCommand.scala
@@ -43,13 +43,6 @@ private[sql] case class ProjectForUpdateCommand(
 
   override def processData(sparkSession: SparkSession): Seq[Row] = {
     val LOGGER = LogServiceFactory.getLogService(ProjectForUpdateCommand.getClass.getName)
-
-    //  sqlContext.sparkContext.setLocalProperty(org.apache.spark.sql.execution.SQLExecution
-    //  .EXECUTION_ID_KEY, null)
-    // DataFrame(sqlContext, plan).show(truncate = false)
-    // return Seq.empty
-
-
     val res = plan find {
       case relation: LogicalRelation if relation.relation
         .isInstanceOf[CarbonDatasourceHadoopRelation] =>
@@ -63,9 +56,6 @@ private[sql] case class ProjectForUpdateCommand(
     val relation = CarbonEnv.getInstance(sparkSession).carbonMetastore
       .lookupRelation(DeleteExecution.getTableIdentifier(tableIdentifier))(sparkSession).
       asInstanceOf[CarbonRelation]
-    //    val relation = CarbonEnv.get.carbonMetastore
-    //      .lookupRelation1(deleteExecution.getTableIdentifier(tableIdentifier))(sqlContext).
-    //      asInstanceOf[CarbonRelation]
     val carbonTable = relation.tableMeta.carbonTable
     val metadataLock = CarbonLockFactory
       .getCarbonLockObj(carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
@@ -75,7 +65,7 @@ private[sql] case class ProjectForUpdateCommand(
     val currentTime = CarbonUpdateUtil.readCurrentTime
     //    var dataFrame: DataFrame = null
     var dataSet: DataFrame = null
-    var isPersistEnabled = CarbonProperties.getInstance.isPersistUpdateDataset()
+    val isPersistEnabled = CarbonProperties.getInstance.isPersistUpdateDataset()
     try {
       lockStatus = metadataLock.lockWithRetries()
       if (lockStatus) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74bd52b6/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/StreamingTableStrategy.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/StreamingTableStrategy.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/StreamingTableStrategy.scala
new file mode 100644
index 0000000..0f0bc24
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/StreamingTableStrategy.scala
@@ -0,0 +1,62 @@
+/*
+ * 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.strategy
+
+import org.apache.spark.sql.{CarbonEnv, SparkSession}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.execution.{SparkPlan, SparkStrategy}
+import org.apache.spark.sql.execution.command.ExecutedCommandExec
+import org.apache.spark.sql.execution.command.mutation.{DeleteExecution, ProjectForDeleteCommand, ProjectForUpdateCommand}
+import org.apache.spark.sql.hive.CarbonRelation
+
+import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
+
+/**
+ * Strategy for streaming table, like blocking unsupported operation
+ */
+private[sql] class StreamingTableStrategy(sparkSession: SparkSession) extends SparkStrategy {
+
+  override def apply(plan: LogicalPlan): Seq[SparkPlan] = {
+    plan match {
+      case update@ProjectForUpdateCommand(_, tableIdentifier) =>
+        rejectIfStreamingTable(DeleteExecution.getTableIdentifier(tableIdentifier), "Data update")
+        ExecutedCommandExec(update) :: Nil
+      case delete@ProjectForDeleteCommand(_, tableIdentifier, _) =>
+        rejectIfStreamingTable(DeleteExecution.getTableIdentifier(tableIdentifier), "Date delete")
+        ExecutedCommandExec(delete) :: Nil
+      case _ => Nil
+    }
+  }
+
+  /**
+   * Validate whether Update operation is allowed for specified table in the command
+   */
+  private def rejectIfStreamingTable(tableIdentifier: TableIdentifier, operation: String): Unit = {
+    val streaming = CarbonEnv.getInstance(sparkSession).carbonMetastore
+      .lookupRelation(tableIdentifier)(sparkSession)
+      .asInstanceOf[CarbonRelation]
+      .tableMeta
+      .carbonTable
+      .isStreamingTable
+    if (streaming) {
+      throw new MalformedCarbonCommandException(
+        s"$operation is not allowed for streaming table")
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74bd52b6/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 6892dad..9cad7b0 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
@@ -27,7 +27,7 @@ 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.datasources._
-import org.apache.spark.sql.execution.strategy.{CarbonLateDecodeStrategy, DDLStrategy}
+import org.apache.spark.sql.execution.strategy.{CarbonLateDecodeStrategy, DDLStrategy, StreamingTableStrategy}
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.optimizer.CarbonLateDecodeRule
 import org.apache.spark.sql.parser.CarbonSparkSqlParser
@@ -133,7 +133,11 @@ class CarbonSessionState(sparkSession: SparkSession) extends HiveSessionState(sp
   override lazy val sqlParser: ParserInterface = new CarbonSparkSqlParser(conf, sparkSession)
 
   experimentalMethods.extraStrategies =
-    Seq(new CarbonLateDecodeStrategy, new DDLStrategy(sparkSession))
+    Seq(
+      new StreamingTableStrategy(sparkSession),
+      new CarbonLateDecodeStrategy,
+      new DDLStrategy(sparkSession)
+    )
   experimentalMethods.extraOptimizations = Seq(new CarbonLateDecodeRule)
 
   override lazy val optimizer: Optimizer = new CarbonOptimizer(catalog, conf, experimentalMethods)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/74bd52b6/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala
new file mode 100644
index 0000000..2c1c6b8
--- /dev/null
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala
@@ -0,0 +1,59 @@
+/*
+ * 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.carbondata
+
+import org.apache.spark.sql.test.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
+
+class TestStreamingTableOperation extends QueryTest with BeforeAndAfterAll {
+  override def beforeAll {
+    sql("DROP DATABASE IF EXISTS streaming CASCADE")
+    sql("CREATE DATABASE streaming")
+    sql("USE streaming")
+    sql(
+      """
+        | create table source(
+        |    c1 string,
+        |    c2 int,
+        |    c3 string,
+        |    c5 string
+        | ) STORED BY 'org.apache.carbondata.format'
+        | TBLPROPERTIES ('streaming' = 'true')
+      """.stripMargin)
+    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO TABLE source""")
+  }
+
+
+  test("test blocking update and delete operation on streaming table") {
+    intercept[MalformedCarbonCommandException] {
+      sql("""UPDATE source d SET (d.c2) = (d.c2 + 1) WHERE d.c1 = 'a'""").show()
+    }
+    intercept[MalformedCarbonCommandException] {
+      sql("""DELETE FROM source WHERE d.c1 = 'a'""").show()
+    }
+  }
+
+  override def afterAll {
+    sql("USE default")
+    sql("DROP DATABASE IF EXISTS streaming CASCADE")
+  }
+}


[48/49] carbondata git commit: [CARBONDATA-1523]Pre Aggregate table selection and Query Plan changes

Posted by ra...@apache.org.
[CARBONDATA-1523]Pre Aggregate table selection and Query Plan changes

This closes #1464


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

Branch: refs/heads/fgdatamap
Commit: e4f2843c6d9aa3dc8dfca52c6fcfa084b07492a5
Parents: 1c16afa
Author: kumarvishal <ku...@gmail.com>
Authored: Mon Oct 30 12:44:32 2017 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Tue Nov 14 00:48:16 2017 +0530

----------------------------------------------------------------------
 .../core/constants/CarbonCommonConstants.java   |   2 +
 .../ThriftWrapperSchemaConverterImpl.java       |   7 +-
 .../schema/table/AggregationDataMapSchema.java  | 212 +++++
 .../core/metadata/schema/table/CarbonTable.java |  16 +-
 .../metadata/schema/table/DataMapSchema.java    |  19 +-
 .../schema/table/DataMapSchemaFactory.java      |  38 +
 .../core/metadata/schema/table/TableInfo.java   |   7 +-
 .../core/metadata/schema/table/TableSchema.java |   5 +-
 .../core/preagg/AggregateTableSelector.java     | 135 +++
 .../carbondata/core/preagg/QueryColumn.java     |  70 ++
 .../carbondata/core/preagg/QueryPlan.java       |  59 ++
 .../TestPreAggregateTableSelection.scala        | 175 ++++
 .../spark/sql/catalyst/CarbonDDLSqlParser.scala |   7 +
 .../spark/rdd/CarbonDataRDDFactory.scala        |   2 +-
 .../scala/org/apache/spark/sql/CarbonEnv.scala  |   3 +
 .../CreatePreAggregateTableCommand.scala        |  12 +-
 .../preaaggregate/PreAggregateListeners.scala   |  24 +-
 .../preaaggregate/PreAggregateUtil.scala        | 184 ++--
 .../spark/sql/hive/CarbonAnalysisRules.scala    | 101 +--
 .../sql/hive/CarbonPreAggregateRules.scala      | 829 +++++++++++++++++++
 .../spark/sql/hive/CarbonSessionState.scala     |   6 +-
 .../sql/parser/CarbonSpark2SqlParser.scala      |   8 +
 .../spark/sql/parser/CarbonSparkSqlParser.scala |   3 +-
 23 files changed, 1709 insertions(+), 215 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/e4f2843c/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
index e27e5bd..8018c2b 100644
--- a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
+++ b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
@@ -1416,6 +1416,8 @@ public final class CarbonCommonConstants {
 
   public static final String CARBON_MERGE_INDEX_IN_SEGMENT_DEFAULT = "true";
 
+  public static final String AGGREGATIONDATAMAPSCHEMA = "AggregateDataMapHandler";
+
   private CarbonCommonConstants() {
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e4f2843c/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 b914e06..fef2e0f 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
@@ -31,6 +31,7 @@ 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.DataMapSchemaFactory;
 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;
@@ -628,10 +629,10 @@ public class ThriftWrapperSchemaConverterImpl implements SchemaConverter {
 
   @Override public DataMapSchema fromExternalToWrapperDataMapSchema(
       org.apache.carbondata.format.DataMapSchema thriftDataMapSchema) {
-    DataMapSchema childSchema =
-        new DataMapSchema(thriftDataMapSchema.getDataMapName(), thriftDataMapSchema.getClassName());
+    DataMapSchema childSchema = DataMapSchemaFactory.INSTANCE
+        .getDataMapSchema(thriftDataMapSchema.getDataMapName(), thriftDataMapSchema.getClassName());
     childSchema.setProperties(thriftDataMapSchema.getProperties());
-    if (thriftDataMapSchema.getRelationIdentifire() != null) {
+    if (null != thriftDataMapSchema.getRelationIdentifire()) {
       RelationIdentifier relationIdentifier =
           new RelationIdentifier(thriftDataMapSchema.getRelationIdentifire().getDatabaseName(),
               thriftDataMapSchema.getRelationIdentifire().getTableName(),

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e4f2843c/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/AggregationDataMapSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/AggregationDataMapSchema.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/AggregationDataMapSchema.java
new file mode 100644
index 0000000..87c07f4
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/AggregationDataMapSchema.java
@@ -0,0 +1,212 @@
+/*
+ * 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.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
+import org.apache.carbondata.core.metadata.schema.table.column.ParentColumnTableRelation;
+
+/**
+ * data map schema class for pre aggregation
+ */
+public class AggregationDataMapSchema extends DataMapSchema {
+
+  /**
+   * map of parent column name to set of child column column without
+   * aggregation function
+   */
+  private Map<String, Set<ColumnSchema>> parentToNonAggChildMapping;
+
+  /**
+   * map of parent column name to set of child columns column with
+   * aggregation function
+   */
+  private Map<String, Set<ColumnSchema>> parentToAggChildMapping;
+
+  /**
+   * map of parent column name to set of aggregation function applied in
+   * in parent column
+   */
+  private Map<String, Set<String>> parentColumnToAggregationsMapping;
+
+  public AggregationDataMapSchema(String dataMapName, String className) {
+    super(dataMapName, className);
+  }
+
+  public void setChildSchema(TableSchema childSchema) {
+    super.setChildSchema(childSchema);
+    List<ColumnSchema> listOfColumns = getChildSchema().getListOfColumns();
+    fillNonAggFunctionColumns(listOfColumns);
+    fillAggFunctionColumns(listOfColumns);
+    fillParentNameToAggregationMapping(listOfColumns);
+  }
+
+  /**
+   * Below method will be used to get the columns on which aggregate function is not applied
+   * @param columnName
+   *                parent column name
+   * @return child column schema
+   */
+  public ColumnSchema getNonAggChildColBasedByParent(String columnName) {
+    Set<ColumnSchema> columnSchemas = parentToNonAggChildMapping.get(columnName);
+    if (null != columnSchemas) {
+      Iterator<ColumnSchema> iterator = columnSchemas.iterator();
+      while (iterator.hasNext()) {
+        ColumnSchema next = iterator.next();
+        if (null == next.getAggFunction() || next.getAggFunction().isEmpty()) {
+          return next;
+        }
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Below method will be used to get the column schema based on parent column name
+   * @param columName
+   *                parent column name
+   * @return child column schema
+   */
+  public ColumnSchema getChildColByParentColName(String columName) {
+    List<ColumnSchema> listOfColumns = childSchema.getListOfColumns();
+    for (ColumnSchema columnSchema : listOfColumns) {
+      List<ParentColumnTableRelation> parentColumnTableRelations =
+          columnSchema.getParentColumnTableRelations();
+      if (parentColumnTableRelations.get(0).getColumnName().equals(columName)) {
+        return columnSchema;
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Below method will be used to get the child column schema based on parent name and aggregate
+   * function applied on column
+   * @param columnName
+   *                  parent column name
+   * @param aggFunction
+   *                  aggregate function applied
+   * @return child column schema
+   */
+  public ColumnSchema getAggChildColByParent(String columnName,
+      String aggFunction) {
+    Set<ColumnSchema> columnSchemas = parentToAggChildMapping.get(columnName);
+    if (null != columnSchemas) {
+      Iterator<ColumnSchema> iterator = columnSchemas.iterator();
+      while (iterator.hasNext()) {
+        ColumnSchema next = iterator.next();
+        if (null != next.getAggFunction() && next.getAggFunction().equalsIgnoreCase(aggFunction)) {
+          return next;
+        }
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Below method is to check if parent column with matching aggregate function
+   * @param parentColumnName
+   *                    parent column name
+   * @param aggFunction
+   *                    aggregate function
+   * @return is matching
+   */
+  public boolean isColumnWithAggFunctionExists(String parentColumnName, String aggFunction) {
+    Set<String> aggFunctions = parentColumnToAggregationsMapping.get(parentColumnName);
+    if (null != aggFunctions && aggFunctions.contains(aggFunction)) {
+      return true;
+    }
+    return false;
+  }
+
+
+  /**
+   * Method to prepare mapping of parent to list of aggregation function applied on that column
+   * @param listOfColumns
+   *        child column schema list
+   */
+  private void fillParentNameToAggregationMapping(List<ColumnSchema> listOfColumns) {
+    parentColumnToAggregationsMapping = new HashMap<>();
+    for (ColumnSchema column : listOfColumns) {
+      if (null != column.getAggFunction() && !column.getAggFunction().isEmpty()) {
+        List<ParentColumnTableRelation> parentColumnTableRelations =
+            column.getParentColumnTableRelations();
+        if (null != parentColumnTableRelations && parentColumnTableRelations.size() == 1) {
+          String columnName = column.getParentColumnTableRelations().get(0).getColumnName();
+          Set<String> aggFunctions = parentColumnToAggregationsMapping.get(columnName);
+          if (null == aggFunctions) {
+            aggFunctions = new HashSet<>();
+            parentColumnToAggregationsMapping.put(columnName, aggFunctions);
+          }
+          aggFunctions.add(column.getAggFunction());
+        }
+      }
+    }
+  }
+
+  /**
+   * Below method will be used prepare mapping between parent column to non aggregation function
+   * columns
+   * @param listOfColumns
+   *                    list of child columns
+   */
+  private void fillNonAggFunctionColumns(List<ColumnSchema> listOfColumns) {
+    parentToNonAggChildMapping = new HashMap<>();
+    for (ColumnSchema column : listOfColumns) {
+      if (null == column.getAggFunction() || column.getAggFunction().isEmpty()) {
+        fillMappingDetails(column, parentToNonAggChildMapping);
+      }
+    }
+  }
+
+  private void fillMappingDetails(ColumnSchema column,
+      Map<String, Set<ColumnSchema>> map) {
+    List<ParentColumnTableRelation> parentColumnTableRelations =
+        column.getParentColumnTableRelations();
+    if (null != parentColumnTableRelations && parentColumnTableRelations.size() == 1) {
+      String columnName = column.getParentColumnTableRelations().get(0).getColumnName();
+      Set<ColumnSchema> columnSchemas = map.get(columnName);
+      if (null == columnSchemas) {
+        columnSchemas = new HashSet<>();
+        map.put(columnName, columnSchemas);
+      }
+      columnSchemas.add(column);
+    }
+  }
+
+  /**
+   * Below method will be used to fill parent to list of aggregation column mapping
+   * @param listOfColumns
+   *        list of child columns
+   */
+  private void fillAggFunctionColumns(List<ColumnSchema> listOfColumns) {
+    parentToAggChildMapping = new HashMap<>();
+    for (ColumnSchema column : listOfColumns) {
+      if (null != column.getAggFunction() && !column.getAggFunction().isEmpty()) {
+        fillMappingDetails(column, parentToAggChildMapping);
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e4f2843c/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 ca0952d..0fd9fbf 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
@@ -126,6 +126,8 @@ public class CarbonTable implements Serializable {
 
   private int dimensionOrdinalMax;
 
+  private boolean hasDataMapSchema;
+
   private CarbonTable() {
     this.tableDimensionsMap = new HashMap<String, List<CarbonDimension>>();
     this.tableImplicitDimensionsMap = new HashMap<String, List<CarbonDimension>>();
@@ -158,6 +160,8 @@ public class CarbonTable implements Serializable {
       table.tablePartitionMap.put(tableInfo.getFactTable().getTableName(),
           tableInfo.getFactTable().getPartitionInfo());
     }
+    table.hasDataMapSchema =
+        null != tableInfo.getDataMapSchemaList() && tableInfo.getDataMapSchemaList().size() > 0;
     return table;
   }
 
@@ -702,13 +706,13 @@ public class CarbonTable implements Serializable {
     this.dimensionOrdinalMax = dimensionOrdinalMax;
   }
 
-  public boolean isPreAggregateTable() {
-    return tableInfo.getParentRelationIdentifiers() != null && !tableInfo
-        .getParentRelationIdentifiers().isEmpty();
+
+  public boolean hasDataMapSchema() {
+    return hasDataMapSchema;
   }
 
-  public boolean hasPreAggregateTables() {
-    return tableInfo.getDataMapSchemaList() != null && !tableInfo
-        .getDataMapSchemaList().isEmpty();
+  public boolean isChildDataMap() {
+    return null != tableInfo.getParentRelationIdentifiers()
+        && !tableInfo.getParentRelationIdentifiers().isEmpty();
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e4f2843c/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
index e0632d9..5a9017b 100644
--- 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
@@ -30,20 +30,20 @@ public class DataMapSchema implements Serializable, Writable {
 
   private static final long serialVersionUID = 6577149126264181553L;
 
-  private String dataMapName;
+  protected String dataMapName;
 
   private String className;
 
-  private RelationIdentifier relationIdentifier;
+  protected RelationIdentifier relationIdentifier;
   /**
    * child table schema
    */
-  private TableSchema childSchema;
+  protected TableSchema childSchema;
 
   /**
    * relation properties
    */
-  private Map<String, String> properties;
+  protected Map<String, String> properties;
 
   public DataMapSchema() {
   }
@@ -69,6 +69,10 @@ public class DataMapSchema implements Serializable, Writable {
     return properties;
   }
 
+  public String getDataMapName() {
+    return dataMapName;
+  }
+
   public void setRelationIdentifier(RelationIdentifier relationIdentifier) {
     this.relationIdentifier = relationIdentifier;
   }
@@ -81,10 +85,6 @@ public class DataMapSchema implements Serializable, Writable {
     this.properties = properties;
   }
 
-  public String getDataMapName() {
-    return dataMapName;
-  }
-
   @Override public void write(DataOutput out) throws IOException {
     out.writeUTF(dataMapName);
     out.writeUTF(className);
@@ -114,7 +114,7 @@ public class DataMapSchema implements Serializable, Writable {
     this.className = in.readUTF();
     boolean isRelationIdnentifierExists = in.readBoolean();
     if (isRelationIdnentifierExists) {
-      this.relationIdentifier = new RelationIdentifier();
+      this.relationIdentifier = new RelationIdentifier(null, null, null);
       this.relationIdentifier.readFields(in);
     }
     boolean isChildSchemaExists = in.readBoolean();
@@ -130,6 +130,5 @@ public class DataMapSchema implements Serializable, Writable {
       String value = in.readUTF();
       this.properties.put(key, value);
     }
-
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e4f2843c/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchemaFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchemaFactory.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchemaFactory.java
new file mode 100644
index 0000000..5729959
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchemaFactory.java
@@ -0,0 +1,38 @@
+/*
+ * 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 static org.apache.carbondata.core.constants.CarbonCommonConstants.AGGREGATIONDATAMAPSCHEMA;
+
+public class DataMapSchemaFactory {
+  public static final DataMapSchemaFactory INSTANCE = new DataMapSchemaFactory();
+
+  /**
+   * Below class will be used to get data map schema object
+   * based on class name
+   * @param className
+   * @return data map schema
+   */
+  public DataMapSchema getDataMapSchema(String dataMapName, String className) {
+    switch (className) {
+      case AGGREGATIONDATAMAPSCHEMA:
+        return new AggregationDataMapSchema(dataMapName, className);
+      default:
+        return new DataMapSchema(dataMapName, className);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e4f2843c/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 1d9e2ec..65878bc 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
@@ -295,7 +295,12 @@ public class TableInfo implements Serializable, Writable {
       for (int i = 0; i < numberOfChildTable; i++) {
         DataMapSchema childSchema = new DataMapSchema();
         childSchema.readFields(in);
-        dataMapSchemaList.add(childSchema);
+        DataMapSchema dataMapSchema = DataMapSchemaFactory.INSTANCE
+            .getDataMapSchema(childSchema.getDataMapName(), childSchema.getClassName());
+        dataMapSchema.setChildSchema(childSchema.getChildSchema());
+        dataMapSchema.setRelationIdentifier(childSchema.getRelationIdentifier());
+        dataMapSchema.setProperties(childSchema.getProperties());
+        dataMapSchemaList.add(dataMapSchema);
       }
     }
     boolean isParentTableRelationIndentifierExists = in.readBoolean();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e4f2843c/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 714e0d8..03848d9 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
@@ -263,9 +263,10 @@ public class TableSchema implements Serializable, Writable {
     Map<String, String> properties = new HashMap<>();
     properties.put("CHILD_SELECT QUERY", queryString);
     properties.put("QUERYTYPE", queryType);
-    DataMapSchema dataMapSchema = new DataMapSchema(dataMapName, className);
-    dataMapSchema.setChildSchema(this);
+    DataMapSchema dataMapSchema =
+        new DataMapSchema(dataMapName, className);
     dataMapSchema.setProperties(properties);
+    dataMapSchema.setChildSchema(this);
     dataMapSchema.setRelationIdentifier(relationIdentifier);
     return dataMapSchema;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e4f2843c/core/src/main/java/org/apache/carbondata/core/preagg/AggregateTableSelector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/preagg/AggregateTableSelector.java b/core/src/main/java/org/apache/carbondata/core/preagg/AggregateTableSelector.java
new file mode 100644
index 0000000..8b87a1a
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/preagg/AggregateTableSelector.java
@@ -0,0 +1,135 @@
+/*
+ * 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.preagg;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.carbondata.core.metadata.schema.table.AggregationDataMapSchema;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+import org.apache.carbondata.core.metadata.schema.table.DataMapSchema;
+import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
+
+/**
+ * Below class will be used to select the aggregate table based
+ * query plan. Rules for selecting the aggregate table is below:
+ * 1. Select all aggregate table based on projection
+ * 2. select aggregate table based on filter exp,
+ * 2. select if aggregate tables based on aggregate columns
+ */
+public class AggregateTableSelector {
+
+  /**
+   * current query plan
+   */
+  private QueryPlan queryPlan;
+
+  /**
+   * parent table
+   */
+  private CarbonTable parentTable;
+
+  public AggregateTableSelector(QueryPlan queryPlan, CarbonTable parentTable) {
+    this.queryPlan = queryPlan;
+    this.parentTable = parentTable;
+  }
+
+  /**
+   * Below method will be used to select pre aggregate tables based on query plan
+   * Rules for selecting the aggregate table is below:
+   * 1. Select all aggregate table based on projection
+   * 2. select aggregate table based on filter exp,
+   * 2. select if aggregate tables based on aggregate columns
+   *
+   * @return selected pre aggregate table schema
+   */
+  public List<DataMapSchema> selectPreAggDataMapSchema() {
+    List<QueryColumn> projectionColumn = queryPlan.getProjectionColumn();
+    List<QueryColumn> aggColumns = queryPlan.getAggregationColumns();
+    List<QueryColumn> filterColumns = queryPlan.getFilterColumns();
+    List<DataMapSchema> dataMapSchemaList = parentTable.getTableInfo().getDataMapSchemaList();
+    List<DataMapSchema> selectedDataMapSchema = new ArrayList<>();
+    boolean isMatch;
+    // match projection columns
+    if (null != projectionColumn && !projectionColumn.isEmpty()) {
+      for (DataMapSchema dmSchema : dataMapSchemaList) {
+        AggregationDataMapSchema aggregationDataMapSchema = (AggregationDataMapSchema) dmSchema;
+        isMatch = true;
+        for (QueryColumn queryColumn : projectionColumn) {
+          ColumnSchema columnSchemaByParentName = aggregationDataMapSchema
+              .getNonAggChildColBasedByParent(queryColumn.getColumnSchema().getColumnName());
+          if (null == columnSchemaByParentName) {
+            isMatch = false;
+          }
+        }
+        if (isMatch) {
+          selectedDataMapSchema.add(dmSchema);
+        }
+      }
+      // if projection column is present but selected table list size is zero then
+      if (selectedDataMapSchema.size() == 0) {
+        return selectedDataMapSchema;
+      }
+    }
+
+    // match filter columns
+    if (null != filterColumns && !filterColumns.isEmpty()) {
+      List<DataMapSchema> dmSchemaToIterate =
+          selectedDataMapSchema.isEmpty() ? dataMapSchemaList : selectedDataMapSchema;
+      selectedDataMapSchema = new ArrayList<>();
+      for (DataMapSchema dmSchema : dmSchemaToIterate) {
+        isMatch = true;
+        for (QueryColumn queryColumn : filterColumns) {
+          AggregationDataMapSchema aggregationDataMapSchema = (AggregationDataMapSchema) dmSchema;
+          ColumnSchema columnSchemaByParentName = aggregationDataMapSchema
+              .getNonAggChildColBasedByParent(queryColumn.getColumnSchema().getColumnName());
+          if (null == columnSchemaByParentName) {
+            isMatch = false;
+          }
+        }
+        if (isMatch) {
+          selectedDataMapSchema.add(dmSchema);
+        }
+      }
+      // if filter column is present and selection size is zero then return
+      if (selectedDataMapSchema.size() == 0) {
+        return selectedDataMapSchema;
+      }
+    }
+    // match aggregation columns
+    if (null != aggColumns && !aggColumns.isEmpty()) {
+      List<DataMapSchema> dmSchemaToIterate =
+          selectedDataMapSchema.isEmpty() ? dataMapSchemaList : selectedDataMapSchema;
+      selectedDataMapSchema = new ArrayList<>();
+      for (DataMapSchema dmSchema : dmSchemaToIterate) {
+        isMatch = true;
+        for (QueryColumn queryColumn : aggColumns) {
+          AggregationDataMapSchema aggregationDataMapSchema = (AggregationDataMapSchema) dmSchema;
+          if (!aggregationDataMapSchema
+              .isColumnWithAggFunctionExists(queryColumn.getColumnSchema().getColumnName(),
+                  queryColumn.getAggFunction())) {
+            isMatch = false;
+          }
+        }
+        if (isMatch) {
+          selectedDataMapSchema.add(dmSchema);
+        }
+      }
+    }
+    return selectedDataMapSchema;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e4f2843c/core/src/main/java/org/apache/carbondata/core/preagg/QueryColumn.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/preagg/QueryColumn.java b/core/src/main/java/org/apache/carbondata/core/preagg/QueryColumn.java
new file mode 100644
index 0000000..a62d556
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/preagg/QueryColumn.java
@@ -0,0 +1,70 @@
+/*
+ * 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.preagg;
+
+import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
+
+/**
+ * column present in query
+ */
+public class QueryColumn {
+
+  /**
+   * parent column schema
+   */
+  private ColumnSchema columnSchema;
+
+  /**
+   * to store the change data type in case of cast
+   */
+  private String changedDataType;
+
+  /**
+   * aggregation function applied
+   */
+  private String aggFunction;
+
+  /**
+   * is filter column
+   */
+  private boolean isFilterColumn;
+
+  public QueryColumn(ColumnSchema columnSchema, String changedDataType, String aggFunction,
+      boolean isFilterColumn) {
+    this.columnSchema = columnSchema;
+    this.changedDataType = changedDataType;
+    this.aggFunction = aggFunction;
+    this.isFilterColumn = isFilterColumn;
+  }
+
+  public ColumnSchema getColumnSchema() {
+    return columnSchema;
+  }
+
+  public String getChangedDataType() {
+    return changedDataType;
+  }
+
+  public String getAggFunction() {
+    return aggFunction;
+  }
+
+  public boolean isFilterColumn() {
+    return isFilterColumn;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e4f2843c/core/src/main/java/org/apache/carbondata/core/preagg/QueryPlan.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/preagg/QueryPlan.java b/core/src/main/java/org/apache/carbondata/core/preagg/QueryPlan.java
new file mode 100644
index 0000000..21a34fa
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/preagg/QueryPlan.java
@@ -0,0 +1,59 @@
+/*
+ * 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.preagg;
+
+import java.util.List;
+
+/**
+ * class to maintain the query plan to select the data map tables
+ */
+public class QueryPlan {
+
+  /**
+   * List of projection columns
+   */
+  private List<QueryColumn> projectionColumn;
+
+  /**
+   * list of aggregation columns
+   */
+  private List<QueryColumn> aggregationColumns;
+
+  /**
+   * list of filter columns
+   */
+  private List<QueryColumn> filterColumns;
+
+  public QueryPlan(List<QueryColumn> projectionColumn, List<QueryColumn> aggregationColumns,
+      List<QueryColumn> filterColumns) {
+    this.projectionColumn = projectionColumn;
+    this.aggregationColumns = aggregationColumns;
+    this.filterColumns = filterColumns;
+  }
+
+  public List<QueryColumn> getProjectionColumn() {
+    return projectionColumn;
+  }
+
+  public List<QueryColumn> getAggregationColumns() {
+    return aggregationColumns;
+  }
+
+  public List<QueryColumn> getFilterColumns() {
+    return filterColumns;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e4f2843c/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateTableSelection.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateTableSelection.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateTableSelection.scala
new file mode 100644
index 0000000..6b435c6
--- /dev/null
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateTableSelection.scala
@@ -0,0 +1,175 @@
+/*
+ * 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.preaTable1regate
+
+import org.apache.spark.sql.catalyst.analysis.UnresolvedAlias
+import org.apache.spark.sql.catalyst.expressions.Alias
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.execution.datasources.LogicalRelation
+import org.apache.spark.sql.{CarbonDatasourceHadoopRelation, DataFrame}
+import org.apache.spark.sql.test.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+class TestPreAggregateTableSelection extends QueryTest with BeforeAndAfterAll {
+
+  override def beforeAll: Unit = {
+    sql("drop table if exists mainTable")
+    sql("drop table if exists agg0")
+    sql("drop table if exists agg1")
+    sql("drop table if exists agg2")
+    sql("drop table if exists agg3")
+    sql("drop table if exists agg4")
+    sql("drop table if exists agg5")
+    sql("drop table if exists agg6")
+    sql("drop table if exists agg7")
+    sql("CREATE TABLE mainTable(id int, name string, city string, age string) STORED BY 'org.apache.carbondata.format'")
+    sql("create datamap agg0 on table mainTable using 'preaggregate' as select name from mainTable group by name")
+    sql("create datamap agg1 on table mainTable using 'preaggregate' as select name,sum(age) from mainTable group by name")
+    sql("create datamap agg2 on table mainTable using 'preaggregate' as select name,sum(id) from mainTable group by name")
+    sql("create datamap agg3 on table mainTable using 'preaggregate' as select name,count(id) from mainTable group by name")
+    sql("create datamap agg4 on table mainTable using 'preaggregate' as select name,sum(age),count(id) from mainTable group by name")
+    sql("create datamap agg5 on table mainTable using 'preaggregate' as select name,avg(age) from mainTable group by name")
+    sql("create datamap agg6 on table mainTable using 'preaggregate' as select name,min(age) from mainTable group by name")
+    sql("create datamap agg7 on table mainTable using 'preaggregate' as select name,max(age) from mainTable group by name")
+    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/measureinsertintotest.csv' into table mainTable")
+  }
+
+
+  test("test PreAggregate table selection 1") {
+    val df = sql("select name from mainTable group by name")
+    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg0")
+  }
+
+  test("test PreAggregate table selection 2") {
+    val df = sql("select name from mainTable where name in (select name from mainTable) group by name")
+    preAggTableValidator(df.queryExecution.analyzed, "mainTable")
+  }
+
+  test("test PreAggregate table selection 3") {
+    val df = sql("select name from mainTable where name in (select name from mainTable group by name) group by name")
+    preAggTableValidator(df.queryExecution.analyzed, "mainTable")
+  }
+
+  test("test PreAggregate table selection 4") {
+    val df = sql("select name from mainTable where name in('vishal') group by name")
+    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg0")
+  }
+
+  test("test PreAggregate table selection 5") {
+    val df = sql("select name, sum(age) from mainTable group by name")
+    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg1")
+  }
+
+  test("test PreAggregate table selection 6") {
+    val df = sql("select sum(age) from mainTable group by name")
+    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg1")
+  }
+
+  test("test PreAggregate table selection 7") {
+    val df = sql("select sum(id) from mainTable group by name")
+    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg2")
+  }
+
+  test("test PreAggregate table selection 8") {
+    val df = sql("select count(id) from mainTable group by name")
+    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg3")
+  }
+
+  test("test PreAggregate table selection 9") {
+    val df = sql("select sum(age), count(id) from mainTable group by name")
+    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg4")
+  }
+
+  test("test PreAggregate table selection 10") {
+    val df = sql("select avg(age) from mainTable group by name")
+    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg5")
+  }
+
+  test("test PreAggregate table selection 11") {
+    val df = sql("select max(age) from mainTable group by name")
+    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg7")
+  }
+
+  test("test PreAggregate table selection 12") {
+    val df = sql("select min(age) from mainTable group by name")
+    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg6")
+  }
+
+  test("test PreAggregate table selection 13") {
+    val df = sql("select name, sum(age) from mainTable where city = 'Bangalore' group by name")
+    preAggTableValidator(df.queryExecution.analyzed, "mainTable")
+  }
+
+  test("test PreAggregate table selection 14") {
+    val df = sql("select sum(age) from mainTable")
+    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg1")
+  }
+
+  test("test PreAggregate table selection 15") {
+    val df = sql("select avg(age) from mainTable")
+    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg5")
+  }
+
+  test("test PreAggregate table selection 16") {
+    val df = sql("select max(age) from mainTable")
+    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg7")
+  }
+
+  test("test PreAggregate table selection 17") {
+    val df = sql("select min(age) from mainTable")
+    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg6")
+  }
+
+  test("test PreAggregate table selection 18") {
+    val df = sql("select count(id) from mainTable")
+    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg3")
+  }
+
+  def preAggTableValidator(plan: LogicalPlan, actualTableName: String) : Unit ={
+    var isValidPlan = false
+    plan.transform {
+      // first check if any preaTable1 scala function is applied it is present is in plan
+      // then call is from create preaTable1regate table class so no need to transform the query plan
+      case logicalRelation:LogicalRelation =>
+        if(logicalRelation.relation.isInstanceOf[CarbonDatasourceHadoopRelation]) {
+          val relation = logicalRelation.relation.asInstanceOf[CarbonDatasourceHadoopRelation]
+          if(relation.carbonTable.getFactTableName.equalsIgnoreCase(actualTableName)) {
+            isValidPlan = true
+          }
+        }
+        logicalRelation
+    }
+    if(!isValidPlan) {
+      assert(false)
+    } else {
+      assert(true)
+    }
+  }
+
+  override def afterAll: Unit = {
+    sql("drop table if exists mainTable")
+    sql("drop table if exists agg0")
+    sql("drop table if exists agg1")
+    sql("drop table if exists agg2")
+    sql("drop table if exists agg3")
+    sql("drop table if exists agg4")
+    sql("drop table if exists agg5")
+    sql("drop table if exists agg6")
+    sql("drop table if exists agg7")
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e4f2843c/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 42447da..e83d96a 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
@@ -173,6 +173,7 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
   protected val DATAMAP = carbonKeyWord("DATAMAP")
   protected val ON = carbonKeyWord("ON")
   protected val DMPROPERTIES = carbonKeyWord("DMPROPERTIES")
+  protected val SELECT = carbonKeyWord("SELECT")
 
   protected val doubleQuotedString = "\"([^\"]+)\"".r
   protected val singleQuotedString = "'([^']+)'".r
@@ -989,6 +990,12 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
         Field(e1, e2.dataType, Some(e1), e2.children, null, e3)
     }
 
+  lazy val addPreAgg: Parser[String] =
+    SELECT ~> restInput <~ opt(";") ^^ {
+      case query =>
+        "select preAGG() as preAgg, " + query
+    }
+
   protected lazy val primitiveFieldType: Parser[Field] =
     primitiveTypes ^^ {
       case e1 =>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e4f2843c/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
index 9899be1..28dcbf2 100644
--- a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
@@ -463,7 +463,7 @@ object CarbonDataRDDFactory {
         throw new Exception(status(0)._2._2.errorMsg)
       }
       // if segment is empty then fail the data load
-      if (!carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable.isPreAggregateTable &&
+      if (!carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable.isChildDataMap &&
           !CarbonLoaderUtil.isValidSegment(carbonLoadModel, carbonLoadModel.getSegmentId.toInt)) {
         // update the load entry in table status file for changing the status to failure
         CommonUtil.updateTableStatusForFailure(carbonLoadModel)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e4f2843c/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 a37b55b..b69ef2f 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
@@ -52,6 +52,9 @@ class CarbonEnv {
 
   def init(sparkSession: SparkSession): Unit = {
     sparkSession.udf.register("getTupleId", () => "")
+    // added for handling preaggregate table creation. when user will fire create ddl for
+    // create table we are adding a udf so no need to apply PreAggregate rules.
+    sparkSession.udf.register("preAgg", () => "")
     if (!initialized) {
       // update carbon session parameters , preserve thread parameters
       val currentThreadSesssionInfo = ThreadLocalSessionInfo.getCarbonSessionInfo

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e4f2843c/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 ebf6273..3a78968 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
@@ -25,6 +25,8 @@ import org.apache.spark.sql.execution.command._
 import org.apache.spark.sql.hive.CarbonRelation
 import org.apache.spark.sql.parser.CarbonSpark2SqlParser
 
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+
 /**
  * Below command class will be used to create pre-aggregate table
  * and updating the parent table about the child table information
@@ -47,9 +49,10 @@ case class CreatePreAggregateTableCommand(
   }
 
   override def processSchema(sparkSession: SparkSession): Seq[Row] = {
-    val df = sparkSession.sql(queryString)
-    val fieldRelationMap = PreAggregateUtil
-      .validateActualSelectPlanAndGetAttrubites(df.logicalPlan, queryString)
+    val updatedQuery = new CarbonSpark2SqlParser().addPreAggFunction(queryString)
+    val df = sparkSession.sql(updatedQuery)
+    val fieldRelationMap = PreAggregateUtil.validateActualSelectPlanAndGetAttributes(
+      df.logicalPlan, queryString)
     val fields = fieldRelationMap.keySet.toSeq
     val tableProperties = mutable.Map[String, String]()
     dmproperties.foreach(t => tableProperties.put(t._1, t._2))
@@ -87,7 +90,8 @@ case class CreatePreAggregateTableCommand(
       val tableInfo = relation.tableMeta.carbonTable.getTableInfo
       // child schema object which will be updated on parent table about the
       val childSchema = tableInfo.getFactTable.buildChildSchema(
-        dataMapName, "", tableInfo.getDatabaseName, queryString, "AGGREGATION")
+        dataMapName, CarbonCommonConstants.AGGREGATIONDATAMAPSCHEMA,
+        tableInfo.getDatabaseName, queryString, "AGGREGATION")
       dmproperties.foreach(f => childSchema.getProperties.put(f._1, f._2))
       // updating the parent table about child table
       PreAggregateUtil.updateMainTable(parentDbName, parentTableName, childSchema, sparkSession)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e4f2843c/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateListeners.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateListeners.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateListeners.scala
index 8271e57..7a66e88 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateListeners.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateListeners.scala
@@ -105,7 +105,7 @@ object PreAggregateDataTypeChangePreListener extends OperationEventListener {
           }
       }
 
-      if (carbonTable.isPreAggregateTable) {
+      if (carbonTable.isChildDataMap) {
         throw new UnsupportedOperationException(s"Cannot change data type for columns in " +
                                                 s"pre-aggreagate table ${
                                                   carbonTable.getDatabaseName
@@ -126,12 +126,12 @@ object PreAggregateDeleteSegmentByDatePreListener extends OperationEventListener
     val deleteSegmentByDatePreEvent = event.asInstanceOf[DeleteSegmentByDatePreEvent]
     val carbonTable = deleteSegmentByDatePreEvent.carbonTable
     if (carbonTable != null) {
-      if (carbonTable.hasPreAggregateTables) {
+      if (carbonTable.hasDataMapSchema) {
         throw new UnsupportedOperationException(
           "Delete segment operation is not supported on tables which have a pre-aggregate table. " +
           "Drop pre-aggregation table to continue")
       }
-      if (carbonTable.isPreAggregateTable) {
+      if (carbonTable.isChildDataMap) {
         throw new UnsupportedOperationException(
           "Delete segment operation is not supported on pre-aggregate table")
       }
@@ -150,11 +150,11 @@ object PreAggregateDeleteSegmentByIdPreListener extends OperationEventListener {
     val tableEvent = event.asInstanceOf[DeleteSegmentByIdPreEvent]
     val carbonTable = tableEvent.carbonTable
     if (carbonTable != null) {
-      if (carbonTable.hasPreAggregateTables) {
+      if (carbonTable.hasDataMapSchema) {
         throw new UnsupportedOperationException(
           "Delete segment operation is not supported on tables which have a pre-aggregate table")
       }
-      if (carbonTable.isPreAggregateTable) {
+      if (carbonTable.isChildDataMap) {
         throw new UnsupportedOperationException(
           "Delete segment operation is not supported on pre-aggregate table")
       }
@@ -190,7 +190,7 @@ object PreAggregateDropColumnPreListener extends OperationEventListener {
               s"pre-aggregate table ${ dataMapSchema.getRelationIdentifier.toString}")
           }
       }
-      if (carbonTable.isPreAggregateTable) {
+      if (carbonTable.isChildDataMap) {
         throw new UnsupportedOperationException(s"Cannot drop columns in pre-aggreagate table ${
           carbonTable.getDatabaseName}.${ carbonTable.getFactTableName }")
       }
@@ -209,11 +209,11 @@ object PreAggregateRenameTablePreListener extends OperationEventListener {
       operationContext: OperationContext): Unit = {
     val renameTablePostListener = event.asInstanceOf[AlterTableRenamePreEvent]
     val carbonTable = renameTablePostListener.carbonTable
-    if (carbonTable.isPreAggregateTable) {
+    if (carbonTable.isChildDataMap) {
       throw new UnsupportedOperationException(
         "Rename operation for pre-aggregate table is not supported.")
     }
-    if (carbonTable.hasPreAggregateTables) {
+    if (carbonTable.hasDataMapSchema) {
       throw new UnsupportedOperationException(
         "Rename operation is not supported for table with pre-aggregate tables")
     }
@@ -231,12 +231,12 @@ object UpdatePreAggregatePreListener extends OperationEventListener {
     val tableEvent = event.asInstanceOf[UpdateTablePreEvent]
     val carbonTable = tableEvent.carbonTable
     if (carbonTable != null) {
-      if (carbonTable.hasPreAggregateTables) {
+      if (carbonTable.hasDataMapSchema) {
         throw new UnsupportedOperationException(
           "Update operation is not supported for tables which have a pre-aggregate table. Drop " +
           "pre-aggregate tables to continue.")
       }
-      if (carbonTable.isPreAggregateTable) {
+      if (carbonTable.isChildDataMap) {
         throw new UnsupportedOperationException(
           "Update operation is not supported for pre-aggregate table")
       }
@@ -255,12 +255,12 @@ object DeletePreAggregatePreListener extends OperationEventListener {
     val tableEvent = event.asInstanceOf[DeleteFromTablePreEvent]
     val carbonTable = tableEvent.carbonTable
     if (carbonTable != null) {
-      if (carbonTable.hasPreAggregateTables) {
+      if (carbonTable.hasDataMapSchema) {
         throw new UnsupportedOperationException(
           "Delete operation is not supported for tables which have a pre-aggregate table. Drop " +
           "pre-aggregate tables to continue.")
       }
-      if (carbonTable.isPreAggregateTable) {
+      if (carbonTable.isChildDataMap) {
         throw new UnsupportedOperationException(
           "Delete operation is not supported for pre-aggregate table")
       }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e4f2843c/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 b926705..62e7623 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
@@ -16,13 +16,14 @@
  */
 package org.apache.spark.sql.execution.command.preaaggregate
 
-import scala.collection.mutable.ListBuffer
+import scala.collection.mutable.{ArrayBuffer, ListBuffer}
 import scala.collection.JavaConverters._
 
 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.analysis.{UnresolvedAlias, UnresolvedFunction, UnresolvedRelation}
+import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, Cast, Expression, NamedExpression, ScalaUDF}
 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}
@@ -51,9 +52,14 @@ object PreAggregateUtil {
 
   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 Aggregate(_, _, SubqueryAlias(_, logicalRelation: LogicalRelation, _))
+        if logicalRelation.relation.isInstanceOf[CarbonDatasourceHadoopRelation] =>
+        logicalRelation.relation.asInstanceOf[CarbonDatasourceHadoopRelation].
+          carbonRelation.metaData.carbonTable
+      case Aggregate(_, _, logicalRelation: LogicalRelation)
+        if logicalRelation.relation.isInstanceOf[CarbonDatasourceHadoopRelation] =>
+        logicalRelation.relation.asInstanceOf[CarbonDatasourceHadoopRelation].
+          carbonRelation.metaData.carbonTable
       case _ => throw new MalformedCarbonCommandException("table does not exist")
     }
   }
@@ -67,54 +73,86 @@ object PreAggregateUtil {
    * @param selectStmt
    * @return list of fields
    */
-  def validateActualSelectPlanAndGetAttrubites(plan: LogicalPlan,
+  def validateActualSelectPlanAndGetAttributes(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) {
+      case Aggregate(groupByExp, aggExp, SubqueryAlias(_, logicalRelation: LogicalRelation, _)) =>
+        getFieldsFromPlan(groupByExp, aggExp, logicalRelation, selectStmt)
+      case Aggregate(groupByExp, aggExp, logicalRelation: LogicalRelation) =>
+        getFieldsFromPlan(groupByExp, aggExp, logicalRelation, selectStmt)
+    }
+  }
+
+  /**
+   * Below method will be used to get the fields from expressions
+   * @param groupByExp
+   *                  grouping expression
+   * @param aggExp
+   *               aggregate expression
+   * @param logicalRelation
+   *                        logical relation
+   * @param selectStmt
+   *                   select statement
+   * @return fields from expressions
+   */
+  def getFieldsFromPlan(groupByExp: Seq[Expression],
+      aggExp: Seq[NamedExpression], logicalRelation: LogicalRelation, selectStmt: String):
+  scala.collection.mutable.LinkedHashMap[Field, DataMapField] = {
+    val fieldToDataMapFieldMap = scala.collection.mutable.LinkedHashMap.empty[Field, DataMapField]
+    if (!logicalRelation.relation.isInstanceOf[CarbonDatasourceHadoopRelation]) {
+      throw new MalformedCarbonCommandException("Un-supported table")
+    }
+    val carbonTable = logicalRelation.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")
+    }
+    groupByExp.map {
+      case 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 Function in select Statement:${
+          selectStmt } ")
+    }
+    aggExp.map {
+      case Alias(attr: AggregateExpression, _) =>
+        if (attr.isDistinct) {
           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 } ")
+            "Distinct is not supported On Pre Aggregation")
         }
-        Some(carbonTable)
+        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 _@Alias(s: ScalaUDF, name) if name.equals("preAgg") =>
       case _ =>
-        throw new MalformedCarbonCommandException(s"Unsupported Select Statement:${ selectStmt } ")
+        throw new MalformedCarbonCommandException(s"Unsupported Select Statement:${
+          selectStmt } ")
     }
     fieldToDataMapFieldMap
   }
@@ -347,30 +385,6 @@ object PreAggregateUtil {
   }
 
   /**
-   * 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
@@ -453,4 +467,30 @@ object PreAggregateUtil {
   def checkMainTableLoad(carbonTable: CarbonTable): Boolean = {
     SegmentStatusManager.readLoadMetadata(carbonTable.getMetaDataFilepath).nonEmpty
   }
+
+  /**
+   * Below method will be used to update logical plan
+   * this is required for creating pre aggregate tables,
+   * so @CarbonPreAggregateRules will not be applied during creation
+   * @param logicalPlan
+   *                    actual logical plan
+   * @return updated plan
+   */
+  def updatePreAggQueyPlan(logicalPlan: LogicalPlan): LogicalPlan = {
+    val updatedPlan = logicalPlan.transform {
+      case _@Project(projectList, child) =>
+        val buffer = new ArrayBuffer[NamedExpression]()
+        buffer ++= projectList
+        buffer += UnresolvedAlias(Alias(UnresolvedFunction("preAgg",
+          Seq.empty, isDistinct = false), "preAgg")())
+        Project(buffer, child)
+      case Aggregate(groupByExp, aggExp, l: UnresolvedRelation) =>
+        val buffer = new ArrayBuffer[NamedExpression]()
+        buffer ++= aggExp
+        buffer += UnresolvedAlias(Alias(UnresolvedFunction("preAgg",
+          Seq.empty, isDistinct = false), "preAgg")())
+        Aggregate(groupByExp, buffer, l)
+    }
+    updatedPlan
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e4f2843c/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
index ba7e1eb..7bd0fad 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
@@ -20,111 +20,12 @@ package org.apache.spark.sql.hive
 import org.apache.spark.sql._
 import org.apache.spark.sql.catalyst.CarbonTableIdentifierImplicit
 import org.apache.spark.sql.catalyst.analysis.{UnresolvedAlias, UnresolvedAttribute, UnresolvedFunction, UnresolvedRelation, UnresolvedStar}
-import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, Cast, ExprId, NamedExpression}
-import org.apache.spark.sql.catalyst.expressions.aggregate.{DeclarativeAggregate, _}
+import org.apache.spark.sql.catalyst.expressions.Alias
 import org.apache.spark.sql.catalyst.plans.Inner
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.rules._
 import org.apache.spark.sql.execution.SparkSqlParser
 import org.apache.spark.sql.execution.command.mutation.ProjectForDeleteCommand
-import org.apache.spark.sql.execution.datasources.LogicalRelation
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-
-
-/**
- * Insert into carbon table from other source
- */
-object CarbonPreInsertionCasts extends Rule[LogicalPlan] {
-  def apply(plan: LogicalPlan): LogicalPlan = {
-    plan.transform {
-      // Wait until children are resolved.
-      case p: LogicalPlan if !p.childrenResolved => p
-
-      case p@InsertIntoTable(relation: LogicalRelation, _, child, _, _)
-        if relation.relation.isInstanceOf[CarbonDatasourceHadoopRelation] =>
-        castChildOutput(p, relation.relation.asInstanceOf[CarbonDatasourceHadoopRelation], child)
-    }
-  }
-
-  def castChildOutput(p: InsertIntoTable,
-      relation: CarbonDatasourceHadoopRelation,
-      child: LogicalPlan)
-  : LogicalPlan = {
-    if (relation.carbonRelation.output.size > CarbonCommonConstants
-      .DEFAULT_MAX_NUMBER_OF_COLUMNS) {
-      sys
-        .error("Maximum supported column by carbon is:" + CarbonCommonConstants
-          .DEFAULT_MAX_NUMBER_OF_COLUMNS
-        )
-    }
-    val isAggregateTable = !relation.carbonRelation.tableMeta.carbonTable.getTableInfo
-      .getParentRelationIdentifiers.isEmpty
-    // transform logical plan if the load is for aggregate table.
-    val childPlan = if (isAggregateTable) {
-      transformAggregatePlan(child)
-    } else {
-      child
-    }
-    if (childPlan.output.size >= relation.carbonRelation.output.size) {
-      val newChildOutput = childPlan.output.zipWithIndex.map { columnWithIndex =>
-        columnWithIndex._1 match {
-          case attr: Alias =>
-            Alias(attr.child, s"col${ columnWithIndex._2 }")(attr.exprId)
-          case attr: Attribute =>
-            Alias(attr, s"col${ columnWithIndex._2 }")(NamedExpression.newExprId)
-          case attr => attr
-        }
-      }
-      val newChild: LogicalPlan = if (newChildOutput == childPlan.output) {
-        p.child
-      } else {
-        Project(newChildOutput, childPlan)
-      }
-      InsertIntoCarbonTable(relation, p.partition, newChild, p.overwrite, p.ifNotExists)
-    } else {
-      sys.error("Cannot insert into target table because column number are different")
-    }
-  }
-
-  /**
-   * Transform the logical plan with average(col1) aggregation type to sum(col1) and count(col1).
-   *
-   * @param logicalPlan
-   * @return
-   */
-  private def transformAggregatePlan(logicalPlan: LogicalPlan): LogicalPlan = {
-    logicalPlan transform {
-      case aggregate@Aggregate(_, aExp, _) =>
-        val newExpressions = aExp flatMap {
-          case alias@Alias(attrExpression: AggregateExpression, _) =>
-            attrExpression.aggregateFunction flatMap {
-              case Average(attr: AttributeReference) =>
-                Seq(Alias(attrExpression
-                  .copy(aggregateFunction = Sum(attr.withName(attr.name)),
-                    resultId = NamedExpression.newExprId),
-                  attr.name)(),
-                  Alias(attrExpression
-                    .copy(aggregateFunction = Count(attr.withName(attr.name)),
-                      resultId = NamedExpression.newExprId), attr.name)())
-              case Average(Cast(attr: AttributeReference, _)) =>
-                Seq(Alias(attrExpression
-                  .copy(aggregateFunction = Sum(attr.withName(attr.name)),
-                    resultId = NamedExpression.newExprId),
-                  attr.name)(),
-                  Alias(attrExpression
-                    .copy(aggregateFunction = Count(attr.withName(attr.name)),
-                      resultId = NamedExpression.newExprId), attr.name)())
-              case _: DeclarativeAggregate => Seq(alias)
-              case _ => Nil
-            }
-          case namedExpr: NamedExpression => Seq(namedExpr)
-        }
-        aggregate.copy(aggregateExpressions = newExpressions)
-      case plan: LogicalPlan => plan
-    }
-  }
-}
 
 case class CarbonIUDAnalysisRule(sparkSession: SparkSession) extends Rule[LogicalPlan] {
 


[11/49] carbondata git commit: [CARBONDATA-1668] Remove isTableSplitPartition in data loading

Posted by ra...@apache.org.
[CARBONDATA-1668] Remove isTableSplitPartition in data loading

This closes #1466


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

Branch: refs/heads/fgdatamap
Commit: d6967bffcd5bec622eff22154edec3edf7a64dec
Parents: 6f68971
Author: Jacky Li <ja...@qq.com>
Authored: Sun Nov 5 20:35:09 2017 +0800
Committer: QiangCai <qi...@qq.com>
Committed: Tue Nov 7 11:11:29 2017 +0800

----------------------------------------------------------------------
 .../spark/rdd/NewCarbonDataLoadRDD.scala        | 122 ++++---------
 .../spark/rdd/CarbonDataRDDFactory.scala        | 172 ++++++++-----------
 2 files changed, 105 insertions(+), 189 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/d6967bff/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
index 1d6ad70..74f7528 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
@@ -177,8 +177,7 @@ class NewCarbonDataLoadRDD[K, V](
     sc: SparkContext,
     result: DataLoadResult[K, V],
     carbonLoadModel: CarbonLoadModel,
-    blocksGroupBy: Array[(String, Array[BlockDetails])],
-    isTableSplitPartition: Boolean)
+    blocksGroupBy: Array[(String, Array[BlockDetails])])
   extends CarbonRDD[(K, V)](sc, Nil) {
 
   sc.setLocalProperty("spark.scheduler.pool", "DDL")
@@ -193,22 +192,8 @@ class NewCarbonDataLoadRDD[K, V](
     sc.broadcast(new SerializableConfiguration(sc.hadoopConfiguration))
 
   override def getPartitions: Array[Partition] = {
-    if (isTableSplitPartition) {
-      // for table split partition
-      var splits: Array[TableSplit] = null
-      splits = CarbonQueryUtil.getTableSplitsForDirectLoad(carbonLoadModel.getFactFilePath)
-
-      splits.zipWithIndex.map { s =>
-        // filter the same partition unique id, because only one will match, so get 0 element
-        val blocksDetails: Array[BlockDetails] = blocksGroupBy.filter(p =>
-          p._1 == s._1.getPartition.getUniqueID)(0)._2
-        new CarbonTableSplitPartition(id, s._2, s._1, blocksDetails)
-      }
-    } else {
-      // for node partition
-      blocksGroupBy.zipWithIndex.map { b =>
-        new CarbonNodePartition(id, b._2, b._1._1, b._1._2)
-      }
+    blocksGroupBy.zipWithIndex.map { b =>
+      new CarbonNodePartition(id, b._2, b._1._1, b._1._2)
     }
   }
 
@@ -278,50 +263,25 @@ class NewCarbonDataLoadRDD[K, V](
         CommonUtil.configureCSVInputFormat(configuration, carbonLoadModel)
         val hadoopAttemptContext = new TaskAttemptContextImpl(configuration, attemptId)
         val format = new CSVInputFormat
-        if (isTableSplitPartition) {
-          // for table split partition
-          val split = theSplit.asInstanceOf[CarbonTableSplitPartition]
-          logInfo("Input split: " + split.serializableHadoopSplit.value)
-          carbonLoadModel.setTaskNo(String.valueOf(theSplit.index))
-          model = carbonLoadModel.getCopyWithPartition(
-              split.serializableHadoopSplit.value.getPartition.getUniqueID,
-              split.serializableHadoopSplit.value.getPartition.getFilesPath,
-              carbonLoadModel.getCsvHeader, carbonLoadModel.getCsvDelimiter)
-          partitionID = split.serializableHadoopSplit.value.getPartition.getUniqueID
-          StandardLogService.setThreadName(StandardLogService
-            .getPartitionID(model.getCarbonDataLoadSchema.getCarbonTable.getTableUniqueName)
-            , ThreadLocalTaskInfo.getCarbonTaskInfo.getTaskId + "")
-          CarbonTimeStatisticsFactory.getLoadStatisticsInstance.recordPartitionBlockMap(
-              partitionID, split.partitionBlocksDetail.length)
-          val readers =
-          split.partitionBlocksDetail.map(format.createRecordReader(_, hadoopAttemptContext))
-          readers.zipWithIndex.map { case (reader, index) =>
-            new CSVRecordReaderIterator(reader,
-              split.partitionBlocksDetail(index),
-              hadoopAttemptContext)
-          }
-        } else {
-          // for node partition
-          val split = theSplit.asInstanceOf[CarbonNodePartition]
-          logInfo("Input split: " + split.serializableHadoopSplit)
-          logInfo("The Block Count in this node :" + split.nodeBlocksDetail.length)
-          CarbonTimeStatisticsFactory.getLoadStatisticsInstance.recordHostBlockMap(
-              split.serializableHadoopSplit, split.nodeBlocksDetail.length)
-          val blocksID = gernerateBlocksID
-          carbonLoadModel.setTaskNo(String.valueOf(theSplit.index))
-          val filelist: java.util.List[String] = new java.util.ArrayList[String](
-              CarbonCommonConstants.CONSTANT_SIZE_TEN)
-          CarbonQueryUtil.splitFilePath(carbonLoadModel.getFactFilePath, filelist, ",")
-          model = carbonLoadModel.getCopyWithPartition(partitionID, filelist,
-              carbonLoadModel.getCsvHeader, carbonLoadModel.getCsvDelimiter)
-          StandardLogService.setThreadName(StandardLogService
-            .getPartitionID(model.getCarbonDataLoadSchema.getCarbonTable.getTableUniqueName)
-            , ThreadLocalTaskInfo.getCarbonTaskInfo.getTaskId + "")
-          val readers =
-            split.nodeBlocksDetail.map(format.createRecordReader(_, hadoopAttemptContext))
-          readers.zipWithIndex.map { case (reader, index) =>
-            new CSVRecordReaderIterator(reader, split.nodeBlocksDetail(index), hadoopAttemptContext)
-          }
+
+        val split = theSplit.asInstanceOf[CarbonNodePartition]
+        logInfo("Input split: " + split.serializableHadoopSplit)
+        logInfo("The Block Count in this node :" + split.nodeBlocksDetail.length)
+        CarbonTimeStatisticsFactory.getLoadStatisticsInstance.recordHostBlockMap(
+            split.serializableHadoopSplit, split.nodeBlocksDetail.length)
+        carbonLoadModel.setTaskNo(String.valueOf(theSplit.index))
+        val fileList: java.util.List[String] = new java.util.ArrayList[String](
+            CarbonCommonConstants.CONSTANT_SIZE_TEN)
+        CarbonQueryUtil.splitFilePath(carbonLoadModel.getFactFilePath, fileList, ",")
+        model = carbonLoadModel.getCopyWithPartition(partitionID, fileList,
+            carbonLoadModel.getCsvHeader, carbonLoadModel.getCsvDelimiter)
+        StandardLogService.setThreadName(StandardLogService
+          .getPartitionID(model.getCarbonDataLoadSchema.getCarbonTable.getTableUniqueName)
+          , ThreadLocalTaskInfo.getCarbonTaskInfo.getTaskId + "")
+        val readers =
+          split.nodeBlocksDetail.map(format.createRecordReader(_, hadoopAttemptContext))
+        readers.zipWithIndex.map { case (reader, index) =>
+          new CSVRecordReaderIterator(reader, split.nodeBlocksDetail(index), hadoopAttemptContext)
         }
       }
       /**
@@ -330,14 +290,8 @@ class NewCarbonDataLoadRDD[K, V](
        * @return
        */
       def gernerateBlocksID: String = {
-        if (isTableSplitPartition) {
-          carbonLoadModel.getDatabaseName + "_" + carbonLoadModel.getTableName + "_" +
-          theSplit.asInstanceOf[CarbonTableSplitPartition].serializableHadoopSplit.value
-            .getPartition.getUniqueID + "_" + UUID.randomUUID()
-        } else {
-          carbonLoadModel.getDatabaseName + "_" + carbonLoadModel.getTableName + "_" +
-          UUID.randomUUID()
-        }
+        carbonLoadModel.getDatabaseName + "_" + carbonLoadModel.getTableName + "_" +
+        UUID.randomUUID()
       }
 
       var finished = false
@@ -355,23 +309,17 @@ class NewCarbonDataLoadRDD[K, V](
   }
 
   override def getPreferredLocations(split: Partition): Seq[String] = {
-    if (isTableSplitPartition) {
-      val theSplit = split.asInstanceOf[CarbonTableSplitPartition]
-      val location = theSplit.serializableHadoopSplit.value.getLocations.asScala
-      location
-    } else {
-      val theSplit = split.asInstanceOf[CarbonNodePartition]
-      val firstOptionLocation: Seq[String] = List(theSplit.serializableHadoopSplit)
-      logInfo("Preferred Location for split : " + firstOptionLocation.mkString(","))
-      /**
-       * At original logic, we were adding the next preferred location so that in case of the
-       * failure the Spark should know where to schedule the failed task.
-       * Remove the next preferred location is because some time Spark will pick the same node
-       * for 2 tasks, so one node is getting over loaded with the task and one have no task to
-       * do. And impacting the performance despite of any failure.
-       */
-      firstOptionLocation
-    }
+    val theSplit = split.asInstanceOf[CarbonNodePartition]
+    val firstOptionLocation: Seq[String] = List(theSplit.serializableHadoopSplit)
+    logInfo("Preferred Location for split : " + firstOptionLocation.mkString(","))
+    /**
+     * At original logic, we were adding the next preferred location so that in case of the
+     * failure the Spark should know where to schedule the failed task.
+     * Remove the next preferred location is because some time Spark will pick the same node
+     * for 2 tasks, so one node is getting over loaded with the task and one have no task to
+     * do. And impacting the performance despite of any failure.
+     */
+    firstOptionLocation
   }
 }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d6967bff/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
index cfd8cff..1ad25c3 100644
--- a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
@@ -64,8 +64,7 @@ import org.apache.carbondata.processing.loading.exception.NoRetryException
 import org.apache.carbondata.processing.loading.model.CarbonLoadModel
 import org.apache.carbondata.processing.loading.sort.SortScopeOptions
 import org.apache.carbondata.processing.merger.{CarbonCompactionUtil, CarbonDataMergerUtil, CompactionType}
-import org.apache.carbondata.processing.splits.TableSplit
-import org.apache.carbondata.processing.util.{CarbonDataProcessorUtil, CarbonLoaderUtil, CarbonQueryUtil}
+import org.apache.carbondata.processing.util.{CarbonDataProcessorUtil, CarbonLoaderUtil}
 import org.apache.carbondata.spark.{DataLoadResultImpl, PartitionFactory, _}
 import org.apache.carbondata.spark.load._
 import org.apache.carbondata.spark.util.{CarbonScalaUtil, CommonUtil, Util}
@@ -635,117 +634,86 @@ object CarbonDataRDDFactory {
       // Check if any load need to be deleted before loading new data
       DataManagementFunc.deleteLoadsAndUpdateMetadata(carbonLoadModel.getDatabaseName,
         carbonLoadModel.getTableName, storePath, false, carbonTable)
-      // get partition way from configuration
-      // val isTableSplitPartition = CarbonProperties.getInstance().getProperty(
-      // CarbonCommonConstants.TABLE_SPLIT_PARTITION,
-      // CarbonCommonConstants.TABLE_SPLIT_PARTITION_DEFAULT_VALUE).toBoolean
-      val isTableSplitPartition = false
       var blocksGroupBy: Array[(String, Array[BlockDetails])] = null
       var status: Array[(String, (LoadMetadataDetails, ExecutionErrors))] = null
       var res: Array[List[(String, (LoadMetadataDetails, ExecutionErrors))]] = null
 
       def loadDataFile(): Unit = {
-        if (isTableSplitPartition) {
-          /*
-         * when data handle by table split partition
-         * 1) get partition files, direct load or not will get the different files path
-         * 2) get files blocks by using SplitUtils
-         * 3) output Array[(partitionID,Array[BlockDetails])] to blocksGroupBy
+        /*
+         * when data load handle by node partition
+         * 1)clone the hadoop configuration,and set the file path to the configuration
+         * 2)use org.apache.hadoop.mapreduce.lib.input.TextInputFormat to get splits,size info
+         * 3)use CarbonLoaderUtil.nodeBlockMapping to get mapping info of node and block,
+         *   for locally writing carbondata files(one file one block) in nodes
+         * 4)use NewCarbonDataLoadRDD to load data and write to carbondata files
          */
-          var splits = Array[TableSplit]()
-          // get all table Splits, this part means files were divide to different partitions
-          splits = CarbonQueryUtil.getTableSplitsForDirectLoad(carbonLoadModel.getFactFilePath)
-          // get all partition blocks from file list
-          blocksGroupBy = splits.map {
-            split =>
-              val pathBuilder = new StringBuilder()
-              for (path <- split.getPartition.getFilesPath.asScala) {
-                pathBuilder.append(path).append(",")
-              }
-              if (pathBuilder.nonEmpty) {
-                pathBuilder.substring(0, pathBuilder.size - 1)
-              }
-              (split.getPartition.getUniqueID, SparkUtil.getSplits(pathBuilder.toString(),
-                sqlContext.sparkContext
-              ))
-          }
-        } else {
-          /*
-           * when data load handle by node partition
-           * 1)clone the hadoop configuration,and set the file path to the configuration
-           * 2)use org.apache.hadoop.mapreduce.lib.input.TextInputFormat to get splits,size info
-           * 3)use CarbonLoaderUtil.nodeBlockMapping to get mapping info of node and block,
-           *   for locally writing carbondata files(one file one block) in nodes
-           * 4)use NewCarbonDataLoadRDD to load data and write to carbondata files
-           */
-          val hadoopConfiguration = new Configuration(sqlContext.sparkContext.hadoopConfiguration)
-          // FileUtils will skip file which is no csv, and return all file path which split by ','
-          val filePaths = carbonLoadModel.getFactFilePath
-          hadoopConfiguration.set(FileInputFormat.INPUT_DIR, filePaths)
-          hadoopConfiguration.set(FileInputFormat.INPUT_DIR_RECURSIVE, "true")
-          hadoopConfiguration.set("io.compression.codecs",
-            """org.apache.hadoop.io.compress.GzipCodec,
-               org.apache.hadoop.io.compress.DefaultCodec,
-               org.apache.hadoop.io.compress.BZip2Codec""".stripMargin)
-
-          CommonUtil.configSplitMaxSize(sqlContext.sparkContext, filePaths, hadoopConfiguration)
-
-          val inputFormat = new org.apache.hadoop.mapreduce.lib.input.TextInputFormat
-          val jobContext = new Job(hadoopConfiguration)
-          val rawSplits = inputFormat.getSplits(jobContext).toArray
-          val blockList = rawSplits.map { inputSplit =>
-            val fileSplit = inputSplit.asInstanceOf[FileSplit]
-            new TableBlockInfo(fileSplit.getPath.toString,
-              fileSplit.getStart, "1",
-              fileSplit.getLocations, fileSplit.getLength, ColumnarFormatVersion.V1, null
-            ).asInstanceOf[Distributable]
-          }
-          // group blocks to nodes, tasks
-          val startTime = System.currentTimeMillis
-          val activeNodes = DistributionUtil
-              .ensureExecutorsAndGetNodeList(blockList, sqlContext.sparkContext)
-          val nodeBlockMapping =
-            CarbonLoaderUtil
-                .nodeBlockMapping(blockList.toSeq.asJava, -1, activeNodes.toList.asJava).asScala
-                .toSeq
-          val timeElapsed: Long = System.currentTimeMillis - startTime
-          LOGGER.info("Total Time taken in block allocation: " + timeElapsed)
-          LOGGER.info(s"Total no of blocks: ${ blockList.length }, " +
-              s"No.of Nodes: ${nodeBlockMapping.size}")
-          var str = ""
-          nodeBlockMapping.foreach(entry => {
-            val tableBlock = entry._2
-            str = str + "#Node: " + entry._1 + " no.of.blocks: " + tableBlock.size()
-            tableBlock.asScala.foreach(tableBlockInfo =>
-              if (!tableBlockInfo.getLocations.exists(hostentry =>
-                hostentry.equalsIgnoreCase(entry._1)
-              )) {
-                str = str + " , mismatch locations: " + tableBlockInfo.getLocations
-                    .foldLeft("")((a, b) => a + "," + b)
-              }
-            )
-            str = str + "\n"
-          }
+        val hadoopConfiguration = new Configuration(sqlContext.sparkContext.hadoopConfiguration)
+        // FileUtils will skip file which is no csv, and return all file path which split by ','
+        val filePaths = carbonLoadModel.getFactFilePath
+        hadoopConfiguration.set(FileInputFormat.INPUT_DIR, filePaths)
+        hadoopConfiguration.set(FileInputFormat.INPUT_DIR_RECURSIVE, "true")
+        hadoopConfiguration.set("io.compression.codecs",
+          """org.apache.hadoop.io.compress.GzipCodec,
+             org.apache.hadoop.io.compress.DefaultCodec,
+             org.apache.hadoop.io.compress.BZip2Codec""".stripMargin)
+
+        CommonUtil.configSplitMaxSize(sqlContext.sparkContext, filePaths, hadoopConfiguration)
+
+        val inputFormat = new org.apache.hadoop.mapreduce.lib.input.TextInputFormat
+        val jobContext = new Job(hadoopConfiguration)
+        val rawSplits = inputFormat.getSplits(jobContext).toArray
+        val blockList = rawSplits.map { inputSplit =>
+          val fileSplit = inputSplit.asInstanceOf[FileSplit]
+          new TableBlockInfo(fileSplit.getPath.toString,
+            fileSplit.getStart, "1",
+            fileSplit.getLocations, fileSplit.getLength, ColumnarFormatVersion.V1, null
+          ).asInstanceOf[Distributable]
+        }
+        // group blocks to nodes, tasks
+        val startTime = System.currentTimeMillis
+        val activeNodes = DistributionUtil
+            .ensureExecutorsAndGetNodeList(blockList, sqlContext.sparkContext)
+        val nodeBlockMapping =
+          CarbonLoaderUtil
+              .nodeBlockMapping(blockList.toSeq.asJava, -1, activeNodes.toList.asJava).asScala
+              .toSeq
+        val timeElapsed: Long = System.currentTimeMillis - startTime
+        LOGGER.info("Total Time taken in block allocation: " + timeElapsed)
+        LOGGER.info(s"Total no of blocks: ${ blockList.length }, " +
+            s"No.of Nodes: ${nodeBlockMapping.size}")
+        var str = ""
+        nodeBlockMapping.foreach(entry => {
+          val tableBlock = entry._2
+          str = str + "#Node: " + entry._1 + " no.of.blocks: " + tableBlock.size()
+          tableBlock.asScala.foreach(tableBlockInfo =>
+            if (!tableBlockInfo.getLocations.exists(hostentry =>
+              hostentry.equalsIgnoreCase(entry._1)
+            )) {
+              str = str + " , mismatch locations: " + tableBlockInfo.getLocations
+                  .foldLeft("")((a, b) => a + "," + b)
+            }
           )
-          LOGGER.info(str)
-          blocksGroupBy = nodeBlockMapping.map(entry => {
-            val blockDetailsList =
-              entry._2.asScala.map(distributable => {
-                val tableBlock = distributable.asInstanceOf[TableBlockInfo]
-                new BlockDetails(new Path(tableBlock.getFilePath),
-                  tableBlock.getBlockOffset, tableBlock.getBlockLength, tableBlock.getLocations
-                )
-              }).toArray
-            (entry._1, blockDetailsList)
-          }
-          ).toArray
+          str = str + "\n"
         }
+        )
+        LOGGER.info(str)
+        blocksGroupBy = nodeBlockMapping.map { entry =>
+          val blockDetailsList =
+            entry._2.asScala.map { distributable =>
+              val tableBlock = distributable.asInstanceOf[TableBlockInfo]
+              new BlockDetails(new Path(tableBlock.getFilePath),
+                tableBlock.getBlockOffset, tableBlock.getBlockLength, tableBlock.getLocations
+              )
+            }.toArray
+          (entry._1, blockDetailsList)
+        }.toArray
 
-        status = new NewCarbonDataLoadRDD(sqlContext.sparkContext,
+        status = new NewCarbonDataLoadRDD(
+          sqlContext.sparkContext,
           new DataLoadResultImpl(),
           carbonLoadModel,
-          blocksGroupBy,
-          isTableSplitPartition).collect()
+          blocksGroupBy
+        ).collect()
       }
 
       def loadDataFrame(): Unit = {


[20/49] carbondata git commit: [CARBONDATA-1596] Fixed IntermediateFileMerger for decimal types

Posted by ra...@apache.org.
[CARBONDATA-1596] Fixed IntermediateFileMerger for decimal types

Analysis: casting bigdecimal to byte[] was throwing ClassCastException in IntermediateFileMerger.

Solution: Use DataType#bigDecimalToByte to convert bigdecimal to byte[].

This closes #1420


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

Branch: refs/heads/fgdatamap
Commit: fd28b15610c1e048f4d07ec0af1905ef2bcf5e8f
Parents: 8789252
Author: kunal642 <ku...@gmail.com>
Authored: Thu Oct 19 11:56:37 2017 +0530
Committer: Jacky Li <ja...@qq.com>
Committed: Wed Nov 8 20:14:30 2017 +0800

----------------------------------------------------------------------
 .../dataload/TestLoadDataGeneral.scala          | 27 ++++++++++++++++++++
 .../sort/sortdata/IntermediateFileMerger.java   |  5 +++-
 2 files changed, 31 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/fd28b156/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
index a749f12..b90a5ea 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
@@ -23,11 +23,15 @@ import scala.collection.mutable.ArrayBuffer
 
 import org.apache.spark.sql.Row
 import org.scalatest.BeforeAndAfterAll
+
 import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
 import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.metadata.CarbonMetadata
 import org.apache.spark.sql.test.util.QueryTest
 
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.CarbonProperties
+
 class TestLoadDataGeneral extends QueryTest with BeforeAndAfterAll {
 
   override def beforeAll {
@@ -147,8 +151,31 @@ class TestLoadDataGeneral extends QueryTest with BeforeAndAfterAll {
     sql("DROP TABLE load_test_singlepass")
   }
 
+  test("test load data with decimal type and sort intermediate files as 1") {
+    sql("drop table if exists carbon_table")
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.SORT_INTERMEDIATE_FILES_LIMIT, "1")
+      .addProperty(CarbonCommonConstants.SORT_SIZE, "1")
+      .addProperty(CarbonCommonConstants.DATA_LOAD_BATCH_SIZE, "1")
+    sql("create table if not exists carbonBigDecimal (ID Int, date Timestamp, country String, name String, phonetype String, serialname String, salary decimal(27, 10)) STORED BY 'org.apache.carbondata.format'")
+    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/decimalBoundaryDataCarbon.csv' into table carbonBigDecimal")
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.SORT_INTERMEDIATE_FILES_LIMIT,
+        CarbonCommonConstants.SORT_INTERMEDIATE_FILES_LIMIT_DEFAULT_VALUE)
+      .addProperty(CarbonCommonConstants.SORT_SIZE, CarbonCommonConstants.SORT_SIZE_DEFAULT_VAL)
+      .addProperty(CarbonCommonConstants.DATA_LOAD_BATCH_SIZE,
+        CarbonCommonConstants.DATA_LOAD_BATCH_SIZE_DEFAULT)
+    sql("drop table if exists carbon_table")
+  }
+
   override def afterAll {
     sql("DROP TABLE if exists loadtest")
     sql("drop table if exists invalidMeasures")
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.SORT_INTERMEDIATE_FILES_LIMIT,
+        CarbonCommonConstants.SORT_INTERMEDIATE_FILES_LIMIT_DEFAULT_VALUE)
+      .addProperty(CarbonCommonConstants.SORT_SIZE, CarbonCommonConstants.SORT_SIZE_DEFAULT_VAL)
+      .addProperty(CarbonCommonConstants.DATA_LOAD_BATCH_SIZE,
+        CarbonCommonConstants.DATA_LOAD_BATCH_SIZE_DEFAULT)
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fd28b156/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/IntermediateFileMerger.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/IntermediateFileMerger.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/IntermediateFileMerger.java
index 266e69a..bc65026 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/IntermediateFileMerger.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/IntermediateFileMerger.java
@@ -23,6 +23,7 @@ import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.IOException;
+import java.math.BigDecimal;
 import java.util.AbstractQueue;
 import java.util.PriorityQueue;
 import java.util.concurrent.Callable;
@@ -32,6 +33,7 @@ import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.core.util.DataTypeUtil;
 import org.apache.carbondata.core.util.NonDictionaryUtil;
 import org.apache.carbondata.processing.sort.exception.CarbonSortKeyAndGroupByException;
 
@@ -358,7 +360,8 @@ public class IntermediateFileMerger implements Callable<Void> {
           } else if (dataType == DataTypes.DOUBLE) {
             stream.writeDouble((Double) NonDictionaryUtil.getMeasure(fieldIndex, row));
           } else if (DataTypes.isDecimal(dataType)) {
-            byte[] bigDecimalInBytes = (byte[]) NonDictionaryUtil.getMeasure(fieldIndex, row);
+            byte[] bigDecimalInBytes = DataTypeUtil
+                .bigDecimalToByte((BigDecimal) NonDictionaryUtil.getMeasure(fieldIndex, row));
             stream.writeInt(bigDecimalInBytes.length);
             stream.write(bigDecimalInBytes);
           } else {


[47/49] carbondata git commit: [CARBONDATA-1523]Pre Aggregate table selection and Query Plan changes

Posted by ra...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/e4f2843c/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonPreAggregateRules.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonPreAggregateRules.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonPreAggregateRules.scala
new file mode 100644
index 0000000..3fb0db0
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonPreAggregateRules.scala
@@ -0,0 +1,829 @@
+/*
+ * 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.hive
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.{AnalysisException, CarbonDatasourceHadoopRelation, InsertIntoCarbonTable, SparkSession}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.analysis.{UnresolvedAlias, UnresolvedAttribute}
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, Cast, Divide, Expression, NamedExpression, PredicateSubquery, ScalaUDF}
+import org.apache.spark.sql.catalyst.expressions.aggregate._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.execution.datasources.LogicalRelation
+import org.apache.spark.sql.types._
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.metadata.schema.table.{AggregationDataMapSchema, CarbonTable, DataMapSchema}
+import org.apache.carbondata.core.preagg.{AggregateTableSelector, QueryColumn, QueryPlan}
+import org.apache.carbondata.spark.util.CarbonScalaUtil
+
+/**
+ * Class for applying Pre Aggregate rules
+ * Responsibility.
+ * 1. Check plan is valid plan for updating the parent table plan with child table
+ * 2. Updated the plan based on child schema
+ *
+ * Rules for Upadating the plan
+ * 1. Grouping expression rules
+ *    1.1 Change the parent attribute reference for of group expression
+ * to child attribute reference
+ *
+ * 2. Aggregate expression rules
+ *    2.1 Change the parent attribute reference for of group expression to
+ * child attribute reference
+ *    2.2 Change the count AggregateExpression to Sum as count
+ * is already calculated so in case of aggregate table
+ * we need to apply sum to get the count
+ *    2.2 In case of average aggregate function select 2 columns from aggregate table with
+ * aggregation
+ * sum and count. Then add divide(sum(column with sum), sum(column with count)).
+ * Note: During aggregate table creation for average table will be created with two columns
+ * one for sum(column) and count(column) to support rollup
+ *
+ * 3. Filter Expression rules.
+ *    3.1 Updated filter expression attributes with child table attributes
+ * 4. Update the Parent Logical relation with child Logical relation
+ *
+ * @param sparkSession
+ * spark session
+ */
+case class CarbonPreAggregateQueryRules(sparkSession: SparkSession) extends Rule[LogicalPlan] {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+    var needAnalysis = true
+    plan.transformExpressions {
+      // first check if any preAgg scala function is applied it is present is in plan
+      // then call is from create preaggregate table class so no need to transform the query plan
+      case al@Alias(udf: ScalaUDF, name) if name.equalsIgnoreCase("preAgg") =>
+        needAnalysis = false
+        al
+      // in case of query if any unresolve alias is present then wait for plan to be resolved
+      // return the same plan as we can tranform the plan only when everything is resolved
+      case unresolveAlias@UnresolvedAlias(_, _) =>
+        needAnalysis = false
+        unresolveAlias
+      case attr@UnresolvedAttribute(_) =>
+        needAnalysis = false
+        attr
+    }
+    // if plan is not valid for transformation then return same plan
+    if (!needAnalysis) {
+      plan
+    } else {
+      // create buffer to collect all the column and its metadata information
+      val list = scala.collection.mutable.ListBuffer.empty[QueryColumn]
+      var isValidPlan = true
+      val carbonTable = plan match {
+        // matching the plan based on supported plan
+        // if plan is matches with any case it will validate and get all
+        // information required for transforming the plan
+
+        // When plan has grouping expression, aggregate expression
+        // subquery
+        case Aggregate(groupingExp,
+        aggregateExp,
+        SubqueryAlias(_, logicalRelation: LogicalRelation, _))
+          // only carbon query plan is supported checking whether logical relation is
+          // is for carbon
+          if logicalRelation.relation.isInstanceOf[CarbonDatasourceHadoopRelation]   &&
+             logicalRelation.relation.asInstanceOf[CarbonDatasourceHadoopRelation].carbonTable
+               .hasDataMapSchema =>
+          val (carbonTable, tableName) = getCarbonTableAndTableName(logicalRelation)
+          // if it is valid plan then extract the query columns
+          isValidPlan = extractQueryColumnsFromAggExpression(groupingExp,
+            aggregateExp,
+            carbonTable,
+            tableName,
+            list)
+          carbonTable
+
+        // below case for handling filter query
+        // When plan has grouping expression, aggregate expression
+        // filter expression
+        case Aggregate(groupingExp, aggregateExp,
+        Filter(filterExp,
+        SubqueryAlias(_, logicalRelation: LogicalRelation, _)))
+          // only carbon query plan is supported checking whether logical relation is
+          // is for carbon
+          if logicalRelation.relation.isInstanceOf[CarbonDatasourceHadoopRelation]   &&
+             logicalRelation.relation.asInstanceOf[CarbonDatasourceHadoopRelation].carbonTable
+               .hasDataMapSchema =>
+          val (carbonTable, tableName) = getCarbonTableAndTableName(logicalRelation)
+          // if it is valid plan then extract the query columns
+          isValidPlan = extractQueryColumnsFromAggExpression(groupingExp,
+            aggregateExp,
+            carbonTable,
+            tableName,
+            list)
+          // TODO need to handle filter predicate subquery scenario
+          isValidPlan = !PredicateSubquery.hasPredicateSubquery(filterExp)
+          // getting the columns from filter expression
+          if(isValidPlan) {
+            filterExp.transform {
+              case attr: AttributeReference =>
+                list += getQueryColumn(attr.name, carbonTable, tableName, isFilterColumn = true)
+                attr
+            }
+          }
+          carbonTable
+
+        // When plan has grouping expression, aggregate expression
+        // logical relation
+        case Aggregate(groupingExp, aggregateExp, logicalRelation: LogicalRelation)
+          // only carbon query plan is supported checking whether logical relation is
+          // is for carbon
+          if logicalRelation.relation.isInstanceOf[CarbonDatasourceHadoopRelation] &&
+             logicalRelation.relation.asInstanceOf[CarbonDatasourceHadoopRelation].carbonTable
+               .hasDataMapSchema =>
+          val (carbonTable, tableName) = getCarbonTableAndTableName(logicalRelation)
+          // if it is valid plan then extract the query columns
+          isValidPlan = extractQueryColumnsFromAggExpression(groupingExp,
+            aggregateExp,
+            carbonTable,
+            tableName,
+            list)
+          carbonTable
+        case _ =>
+          isValidPlan = false
+          null
+      }
+      // if plan is valid then update the plan with child attributes
+      if (isValidPlan) {
+        // getting all the projection columns
+        val listProjectionColumn = list
+          .filter(queryColumn => queryColumn.getAggFunction.isEmpty && !queryColumn.isFilterColumn)
+        // getting all the filter columns
+        val listFilterColumn = list
+          .filter(queryColumn => queryColumn.getAggFunction.isEmpty && queryColumn.isFilterColumn)
+        // getting all the aggregation columns
+        val listAggregationColumn = list.filter(queryColumn => !queryColumn.getAggFunction.isEmpty)
+        // create a query plan object which will be used to select the list of pre aggregate tables
+        // matches with this plan
+        val queryPlan = new QueryPlan(listProjectionColumn.asJava,
+          listAggregationColumn.asJava,
+          listFilterColumn.asJava)
+        // create aggregate table selector object
+        val aggregateTableSelector = new AggregateTableSelector(queryPlan, carbonTable)
+        // select the list of valid child tables
+        val selectedDataMapSchemas = aggregateTableSelector.selectPreAggDataMapSchema()
+        // if it doesnot match with any pre aggregate table return the same plan
+        if (!selectedDataMapSchemas.isEmpty) {
+          // sort the selected child schema based on size to select smallest pre aggregate table
+          val (aggDataMapSchema, carbonRelation) =
+            selectedDataMapSchemas.asScala.map { selectedDataMapSchema =>
+              val catalog = sparkSession.sessionState.catalog
+              val carbonRelation = catalog
+                .lookupRelation(TableIdentifier(selectedDataMapSchema.getRelationIdentifier
+                  .getTableName,
+                  Some(selectedDataMapSchema.getRelationIdentifier
+                    .getDatabaseName))).asInstanceOf[SubqueryAlias].child
+                .asInstanceOf[LogicalRelation]
+              (selectedDataMapSchema, carbonRelation)
+            }.minBy(f => f._2.relation.asInstanceOf[CarbonDatasourceHadoopRelation].sizeInBytes)
+          // transform the query plan based on selected child schema
+          transformPreAggQueryPlan(plan, aggDataMapSchema, carbonRelation)
+        } else {
+          plan
+        }
+      } else {
+        plan
+      }
+    }
+  }
+
+  /**
+   * Below method will be used to get the child attribute reference
+   * based on parent name
+   *
+   * @param dataMapSchema
+   * child schema
+   * @param attributeReference
+   * parent attribute reference
+   * @param childCarbonRelation
+   * child logical relation
+   * @param aggFunction
+   * aggregation function applied on child
+   * @return child attribute reference
+   */
+  def getChildAttributeReference(dataMapSchema: DataMapSchema,
+      attributeReference: AttributeReference,
+      childCarbonRelation: LogicalRelation,
+      aggFunction: String = ""): AttributeReference = {
+    val aggregationDataMapSchema = dataMapSchema.asInstanceOf[AggregationDataMapSchema];
+    val columnSchema = if (aggFunction.isEmpty) {
+      aggregationDataMapSchema.getChildColByParentColName(attributeReference.name)
+    } else {
+      aggregationDataMapSchema.getAggChildColByParent(attributeReference.name, aggFunction)
+    }
+    // here column schema cannot be null, if it is null then aggregate table selection
+    // logic has some problem
+    if (null == columnSchema) {
+      throw new AnalysisException("Column doesnot exists in Pre Aggregate table")
+    }
+    // finding the child attribute from child logical relation
+    childCarbonRelation.attributeMap.find(p => p._2.name.equals(columnSchema.getColumnName)).get._2
+  }
+
+  /**
+   * Below method will be used to transform the main table plan to child table plan
+   * rules for transformming is as below.
+   * 1. Grouping expression rules
+   *    1.1 Change the parent attribute reference for of group expression
+   * to child attribute reference
+   *
+   * 2. Aggregate expression rules
+   *    2.1 Change the parent attribute reference for of group expression to
+   * child attribute reference
+   *    2.2 Change the count AggregateExpression to Sum as count
+   * is already calculated so in case of aggregate table
+   * we need to apply sum to get the count
+   *    2.2 In case of average aggregate function select 2 columns from aggregate table with
+   * aggregation sum and count. Then add divide(sum(column with sum), sum(column with count)).
+   * Note: During aggregate table creation for average table will be created with two columns
+   * one for sum(column) and count(column) to support rollup
+   * 3. Filter Expression rules.
+   *    3.1 Updated filter expression attributes with child table attributes
+   * 4. Update the Parent Logical relation with child Logical relation
+   *
+   * @param logicalPlan
+   * parent logical plan
+   * @param aggDataMapSchema
+   * select data map schema
+   * @param childCarbonRelation
+   * child carbon table relation
+   * @return transformed plan
+   */
+  def transformPreAggQueryPlan(logicalPlan: LogicalPlan,
+      aggDataMapSchema: DataMapSchema, childCarbonRelation: LogicalRelation): LogicalPlan = {
+    logicalPlan.transform {
+      case Aggregate(grExp, aggExp, child@SubqueryAlias(_, l: LogicalRelation, _))
+        if l.relation.isInstanceOf[CarbonDatasourceHadoopRelation] &&
+           l.relation.asInstanceOf[CarbonDatasourceHadoopRelation].carbonTable.hasDataMapSchema =>
+        val (updatedGroupExp, updatedAggExp, newChild, None) =
+          getUpdatedExpressions(grExp,
+            aggExp,
+            child,
+            None,
+            aggDataMapSchema,
+            childCarbonRelation)
+        Aggregate(updatedGroupExp,
+          updatedAggExp,
+          newChild)
+      case Aggregate(grExp,
+      aggExp,
+      Filter(expression, child@SubqueryAlias(_, l: LogicalRelation, _)))
+        if l.relation.isInstanceOf[CarbonDatasourceHadoopRelation] &&
+           l.relation.asInstanceOf[CarbonDatasourceHadoopRelation].carbonTable.hasDataMapSchema =>
+        val (updatedGroupExp, updatedAggExp, newChild, updatedFilterExpression) =
+          getUpdatedExpressions(grExp,
+            aggExp,
+            child,
+            Some(expression),
+            aggDataMapSchema,
+            childCarbonRelation)
+        Aggregate(updatedGroupExp,
+          updatedAggExp,
+          Filter(updatedFilterExpression.get,
+            newChild))
+      case Aggregate(grExp, aggExp, l: LogicalRelation)
+        if l.relation.isInstanceOf[CarbonDatasourceHadoopRelation] &&
+           l.relation.asInstanceOf[CarbonDatasourceHadoopRelation].carbonTable.hasDataMapSchema =>
+        val (updatedGroupExp, updatedAggExp, newChild, None) =
+          getUpdatedExpressions(grExp,
+            aggExp,
+            l,
+            None,
+            aggDataMapSchema,
+            childCarbonRelation)
+        Aggregate(updatedGroupExp,
+          updatedAggExp,
+          newChild)
+    }
+  }
+
+  /**
+   * Below method will be used to get the updated expression for pre aggregated table.
+   * It will replace the attribute of actual plan with child table attributes.
+   * Updation will be done for below expression.
+   * 1. Grouping expression
+   * 2. aggregate expression
+   * 3. child logical plan
+   * 4. filter expression if present
+   *
+   * @param groupingExpressions
+   * actual plan grouping expression
+   * @param aggregateExpressions
+   * actual plan aggregate expression
+   * @param child
+   * child logical plan
+   * @param filterExpression
+   * filter expression
+   * @param aggDataMapSchema
+   * pre aggregate table schema
+   * @param childCarbonRelation
+   * pre aggregate table logical relation
+   * @return tuple of(updated grouping expression,
+   *         updated aggregate expression,
+   *         updated child logical plan,
+   *         updated filter expression if present in actual plan)
+   */
+  def getUpdatedExpressions(groupingExpressions: Seq[Expression],
+      aggregateExpressions: Seq[NamedExpression],
+      child: LogicalPlan, filterExpression: Option[Expression] = None,
+      aggDataMapSchema: DataMapSchema,
+      childCarbonRelation: LogicalRelation): (Seq[Expression], Seq[NamedExpression], LogicalPlan,
+    Option[Expression]) = {
+    // transforming the group by expression attributes with child attributes
+    val updatedGroupExp = groupingExpressions.map { exp =>
+      exp.transform {
+        case attr: AttributeReference =>
+          getChildAttributeReference(aggDataMapSchema, attr, childCarbonRelation)
+      }
+    }
+    // below code is for updating the aggregate expression.
+    // Note: In case of aggregate expression updation we need to return alias as
+    //       while showing the final result we need to show based on actual query
+    //       for example: If query is "select name from table group by name"
+    //       if we only update the attributes it will show child table column name in final output
+    //       so for handling this if attributes does not have alias we need to return alias of
+    // parent
+    //       table column name
+    // Rules for updating aggregate expression.
+    // 1. If it matches with attribute reference return alias of child attribute reference
+    // 2. If it matches with alias return same alias with child attribute reference
+    // 3. If it matches with alias of any supported aggregate function return aggregate function
+    // with child attribute reference. Please check class level documentation how when aggregate
+    // function will be updated
+
+    val updatedAggExp = aggregateExpressions.map {
+      // case for attribute reference
+      case attr: AttributeReference =>
+        val childAttributeReference = getChildAttributeReference(aggDataMapSchema,
+          attr,
+          childCarbonRelation)
+        // returning the alias to show proper column name in output
+        Alias(childAttributeReference,
+          attr.name)(NamedExpression.newExprId,
+          childAttributeReference.qualifier).asInstanceOf[NamedExpression]
+      // case for alias
+      case Alias(attr: AttributeReference, name) =>
+        val childAttributeReference = getChildAttributeReference(aggDataMapSchema,
+          attr,
+          childCarbonRelation)
+        // returning alias with child attribute reference
+        Alias(childAttributeReference,
+          name)(NamedExpression.newExprId,
+          childAttributeReference.qualifier).asInstanceOf[NamedExpression]
+      // for aggregate function case
+      case alias@Alias(attr: AggregateExpression, name) =>
+        // get the updated aggregate aggregate function
+        val aggExp = getUpdatedAggregateExpressionForChild(attr,
+          aggDataMapSchema,
+          childCarbonRelation)
+        // returning alias with child attribute reference
+        Alias(aggExp,
+          name)(NamedExpression.newExprId,
+          alias.qualifier).asInstanceOf[NamedExpression]
+    }
+    // transformaing the logical relation
+    val newChild = child.transform {
+      case _: LogicalRelation =>
+        childCarbonRelation
+      case _: SubqueryAlias =>
+        childCarbonRelation
+    }
+    // updating the filter expression if present
+    val updatedFilterExpression = if (filterExpression.isDefined) {
+      val filterExp = filterExpression.get
+      Some(filterExp.transform {
+        case attr: AttributeReference =>
+          getChildAttributeReference(aggDataMapSchema, attr, childCarbonRelation)
+      })
+    } else {
+      None
+    }
+    (updatedGroupExp, updatedAggExp, newChild, updatedFilterExpression)
+  }
+
+  /**
+   * Below method will be used to get the aggregate expression based on match
+   * Aggregate expression updation rules
+   * 1 Change the count AggregateExpression to Sum as count
+   * is already calculated so in case of aggregate table
+   * we need to apply sum to get the count
+   * 2 In case of average aggregate function select 2 columns from aggregate table
+   * with aggregation sum and count.
+   * Then add divide(sum(column with sum), sum(column with count)).
+   * Note: During aggregate table creation for average aggregation function
+   * table will be created with two columns one for sum(column) and count(column)
+   * to support rollup
+   *
+   * @param aggExp
+   * aggregate expression
+   * @param dataMapSchema
+   * child data map schema
+   * @param childCarbonRelation
+   * child logical relation
+   * @return updated expression
+   */
+  def getUpdatedAggregateExpressionForChild(aggExp: AggregateExpression,
+      dataMapSchema: DataMapSchema,
+      childCarbonRelation: LogicalRelation):
+  Expression = {
+    aggExp.aggregateFunction match {
+      // Change the count AggregateExpression to Sum as count
+      // is already calculated so in case of aggregate table
+      // we need to apply sum to get the count
+      case count@Count(Seq(attr: AttributeReference)) =>
+        AggregateExpression(Sum(Cast(getChildAttributeReference(dataMapSchema,
+          attr,
+          childCarbonRelation,
+          count.prettyName),
+          LongType)),
+          aggExp.mode,
+          isDistinct = false)
+      case sum@Sum(attr: AttributeReference) =>
+        AggregateExpression(Sum(getChildAttributeReference(dataMapSchema,
+          attr,
+          childCarbonRelation,
+          sum.prettyName)),
+          aggExp.mode,
+          isDistinct = false)
+      case max@Max(attr: AttributeReference) =>
+        AggregateExpression(Max(getChildAttributeReference(dataMapSchema,
+          attr,
+          childCarbonRelation,
+          max.prettyName)),
+          aggExp.mode,
+          isDistinct = false)
+      case min@Min(attr: AttributeReference) =>
+        AggregateExpression(Min(getChildAttributeReference(dataMapSchema,
+          attr,
+          childCarbonRelation,
+          min.prettyName)),
+          aggExp.mode,
+          isDistinct = false)
+      case sum@Sum(Cast(attr: AttributeReference, changeDataType: DataType)) =>
+        AggregateExpression(Sum(Cast(getChildAttributeReference(dataMapSchema,
+          attr,
+          childCarbonRelation,
+          sum.prettyName),
+          changeDataType)),
+          aggExp.mode,
+          isDistinct = false)
+      case min@Min(Cast(attr: AttributeReference, changeDataType: DataType)) =>
+        AggregateExpression(Min(Cast(getChildAttributeReference(dataMapSchema,
+          attr,
+          childCarbonRelation,
+          min.prettyName),
+          changeDataType)),
+          aggExp.mode,
+          isDistinct = false)
+      case max@Max(Cast(attr: AttributeReference, changeDataType: DataType)) =>
+        AggregateExpression(Max(Cast(getChildAttributeReference(dataMapSchema,
+          attr,
+          childCarbonRelation,
+          max.prettyName),
+          changeDataType)),
+          aggExp.mode,
+          isDistinct = false)
+
+      // In case of average aggregate function select 2 columns from aggregate table
+      // with aggregation sum and count.
+      // Then add divide(sum(column with sum), sum(column with count)).
+      case Average(attr: AttributeReference) =>
+        Divide(AggregateExpression(Sum(getChildAttributeReference(dataMapSchema,
+          attr,
+          childCarbonRelation,
+          "sum")),
+          aggExp.mode,
+          isDistinct = false),
+          AggregateExpression(Sum(Cast(getChildAttributeReference(dataMapSchema,
+            attr,
+            childCarbonRelation,
+            "count"),
+            LongType)),
+            aggExp.mode,
+            isDistinct = false))
+      // In case of average aggregate function select 2 columns from aggregate table
+      // with aggregation sum and count.
+      // Then add divide(sum(column with sum), sum(column with count)).
+      case Average(Cast(attr: AttributeReference, changeDataType: DataType)) =>
+        Divide(AggregateExpression(Sum(Cast(getChildAttributeReference(dataMapSchema,
+          attr,
+          childCarbonRelation,
+          "sum"),
+          changeDataType)),
+          aggExp.mode,
+          isDistinct = false),
+          AggregateExpression(Sum(Cast(getChildAttributeReference(dataMapSchema,
+            attr,
+            childCarbonRelation,
+            "count"),
+            LongType)),
+            aggExp.mode,
+            isDistinct = false))
+    }
+  }
+
+  /**
+   * Method to get the carbon table and table name
+   *
+   * @param parentLogicalRelation
+   * parent table relation
+   * @return tuple of carbon table and table name
+   */
+  def getCarbonTableAndTableName(parentLogicalRelation: LogicalRelation): (CarbonTable, String) = {
+    val carbonTable = parentLogicalRelation.relation.asInstanceOf[CarbonDatasourceHadoopRelation]
+      .carbonRelation
+      .metaData.carbonTable
+    val tableName = carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier
+      .getTableName
+    (carbonTable, tableName)
+  }
+
+  /**
+   * Below method will be used to get the query columns from plan
+   *
+   * @param groupByExpression
+   * group by expression
+   * @param aggregateExpressions
+   * aggregate expression
+   * @param carbonTable
+   * parent carbon table
+   * @param tableName
+   * parent table name
+   * @param list
+   * list of attributes
+   * @return plan is valid
+   */
+  def extractQueryColumnsFromAggExpression(groupByExpression: Seq[Expression],
+      aggregateExpressions: Seq[NamedExpression],
+      carbonTable: CarbonTable, tableName: String,
+      list: scala.collection.mutable.ListBuffer[QueryColumn]): Boolean = {
+    aggregateExpressions.map {
+      case attr: AttributeReference =>
+        list += getQueryColumn(attr.name,
+          carbonTable,
+          tableName);
+      case Alias(attr: AttributeReference, _) =>
+        list += getQueryColumn(attr.name,
+          carbonTable,
+          tableName);
+      case Alias(attr: AggregateExpression, _) =>
+        if (attr.isDistinct) {
+          return false
+        }
+        val queryColumn = validateAggregateFunctionAndGetFields(carbonTable,
+          attr.aggregateFunction,
+          tableName)
+        if (queryColumn.nonEmpty) {
+          list ++= queryColumn
+        } else {
+          return false
+        }
+    }
+    true
+  }
+
+  /**
+   * Below method will be used to validate aggregate function and get the attribute information
+   * which is applied on select query.
+   * Currently sum, max, min, count, avg is supported
+   * in case of any other aggregate function it will return empty sequence
+   * In case of avg it will return two fields one for count
+   * and other of sum of that column to support rollup
+   *
+   * @param carbonTable
+   * parent table
+   * @param aggFunctions
+   * aggregation function
+   * @param tableName
+   * parent table name
+   * @return list of fields
+   */
+  def validateAggregateFunctionAndGetFields(carbonTable: CarbonTable,
+      aggFunctions: AggregateFunction,
+      tableName: String
+  ): Seq[QueryColumn] = {
+    val changedDataType = true
+    aggFunctions match {
+      case sum@Sum(attr: AttributeReference) =>
+        Seq(getQueryColumn(attr.name,
+          carbonTable,
+          tableName,
+          sum.prettyName))
+      case sum@Sum(Cast(attr: AttributeReference, changeDataType: DataType)) =>
+        Seq(getQueryColumn(attr.name,
+          carbonTable,
+          tableName,
+          sum.prettyName,
+          changeDataType.typeName,
+          changedDataType))
+      case count@Count(Seq(attr: AttributeReference)) =>
+        Seq(getQueryColumn(attr.name,
+          carbonTable,
+          tableName,
+          count.prettyName))
+      case min@Min(attr: AttributeReference) =>
+        Seq(getQueryColumn(attr.name,
+          carbonTable,
+          tableName,
+          min.prettyName))
+      case min@Min(Cast(attr: AttributeReference, changeDataType: DataType)) =>
+        Seq(getQueryColumn(attr.name,
+          carbonTable,
+          tableName,
+          min.prettyName,
+          changeDataType.typeName,
+          changedDataType))
+      case max@Max(attr: AttributeReference) =>
+        Seq(getQueryColumn(attr.name,
+          carbonTable,
+          tableName,
+          max.prettyName))
+      case max@Max(Cast(attr: AttributeReference, changeDataType: DataType)) =>
+        Seq(getQueryColumn(attr.name,
+          carbonTable,
+          tableName,
+          max.prettyName,
+          changeDataType.typeName,
+          changedDataType))
+      // in case of average need to return two columns
+      // sum and count of the column to added during table creation to support rollup
+      case Average(attr: AttributeReference) =>
+        Seq(getQueryColumn(attr.name,
+          carbonTable,
+          tableName,
+          "sum"
+        ), getQueryColumn(attr.name,
+          carbonTable,
+          tableName,
+          "count"
+        ))
+      // in case of average need to return two columns
+      // sum and count of the column to added during table creation to support rollup
+      case Average(Cast(attr: AttributeReference, changeDataType: DataType)) =>
+        Seq(getQueryColumn(attr.name,
+          carbonTable,
+          tableName,
+          "sum",
+          changeDataType.typeName,
+          changedDataType), getQueryColumn(attr.name,
+          carbonTable,
+          tableName,
+          "count",
+          changeDataType.typeName,
+          changedDataType))
+      case _ =>
+        Seq.empty
+    }
+  }
+
+  /**
+   * Below method will be used to get the query column object which
+   * will have details of the column and its property
+   *
+   * @param columnName
+   * parent column name
+   * @param carbonTable
+   * parent carbon table
+   * @param tableName
+   * parent table name
+   * @param aggFunction
+   * aggregate function applied
+   * @param dataType
+   * data type of the column
+   * @param isChangedDataType
+   * is cast is applied on column
+   * @param isFilterColumn
+   * is filter is applied on column
+   * @return query column
+   */
+  def getQueryColumn(columnName: String,
+      carbonTable: CarbonTable,
+      tableName: String,
+      aggFunction: String = "",
+      dataType: String = "",
+      isChangedDataType: Boolean = false,
+      isFilterColumn: Boolean = false): QueryColumn = {
+    val columnSchema = carbonTable.getColumnByName(tableName, columnName).getColumnSchema
+    if (isChangedDataType) {
+      new QueryColumn(columnSchema, columnSchema.getDataType.getName, aggFunction, isFilterColumn)
+    } else {
+      new QueryColumn(columnSchema,
+        CarbonScalaUtil.convertSparkToCarbonSchemaDataType(dataType),
+        aggFunction, isFilterColumn)
+    }
+  }
+}
+
+/**
+ * Insert into carbon table from other source
+ */
+object CarbonPreInsertionCasts extends Rule[LogicalPlan] {
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    plan.transform {
+      // Wait until children are resolved.
+      case p: LogicalPlan if !p.childrenResolved => p
+
+      case p@InsertIntoTable(relation: LogicalRelation, _, child, _, _)
+        if relation.relation.isInstanceOf[CarbonDatasourceHadoopRelation] =>
+        castChildOutput(p, relation.relation.asInstanceOf[CarbonDatasourceHadoopRelation], child)
+    }
+  }
+
+  def castChildOutput(p: InsertIntoTable,
+      relation: CarbonDatasourceHadoopRelation,
+      child: LogicalPlan)
+  : LogicalPlan = {
+    if (relation.carbonRelation.output.size > CarbonCommonConstants
+      .DEFAULT_MAX_NUMBER_OF_COLUMNS) {
+      sys
+        .error("Maximum supported column by carbon is:" + CarbonCommonConstants
+          .DEFAULT_MAX_NUMBER_OF_COLUMNS
+        )
+    }
+    val isAggregateTable = !relation.carbonRelation.tableMeta.carbonTable.getTableInfo
+      .getParentRelationIdentifiers.isEmpty
+    // transform logical plan if the load is for aggregate table.
+    val childPlan = if (isAggregateTable) {
+      transformAggregatePlan(child)
+    } else {
+      child
+    }
+    if (childPlan.output.size >= relation.carbonRelation.output.size) {
+      val newChildOutput = childPlan.output.zipWithIndex.map { columnWithIndex =>
+        columnWithIndex._1 match {
+          case attr: Alias =>
+            Alias(attr.child, s"col${ columnWithIndex._2 }")(attr.exprId)
+          case attr: Attribute =>
+            Alias(attr, s"col${ columnWithIndex._2 }")(NamedExpression.newExprId)
+          case attr => attr
+        }
+      }
+      val newChild: LogicalPlan = if (newChildOutput == childPlan.output) {
+        p.child
+      } else {
+        Project(newChildOutput, childPlan)
+      }
+      InsertIntoCarbonTable(relation, p.partition, newChild, p.overwrite, p.ifNotExists)
+    } else {
+      sys.error("Cannot insert into target table because column number are different")
+    }
+  }
+
+  /**
+   * Transform the logical plan with average(col1) aggregation type to sum(col1) and count(col1).
+   *
+   * @param logicalPlan
+   * @return
+   */
+  private def transformAggregatePlan(logicalPlan: LogicalPlan): LogicalPlan = {
+    logicalPlan transform {
+      case aggregate@Aggregate(_, aExp, _) =>
+        val newExpressions = aExp.flatMap {
+          case alias@Alias(attrExpression: AggregateExpression, _) =>
+            attrExpression.aggregateFunction match {
+              case Average(attr: AttributeReference) =>
+                Seq(Alias(attrExpression
+                  .copy(aggregateFunction = Sum(attr),
+                    resultId = NamedExpression.newExprId), attr.name + "_sum")(),
+                  Alias(attrExpression
+                    .copy(aggregateFunction = Count(attr),
+                      resultId = NamedExpression.newExprId), attr.name + "_count")())
+              case Average(cast@Cast(attr: AttributeReference, _)) =>
+                Seq(Alias(attrExpression
+                  .copy(aggregateFunction = Sum(cast),
+                    resultId = NamedExpression.newExprId),
+                  attr.name + "_sum")(),
+                  Alias(attrExpression
+                    .copy(aggregateFunction = Count(cast),
+                      resultId = NamedExpression.newExprId), attr.name + "_count")())
+              case _ => Seq(alias)
+            }
+          case namedExpr: NamedExpression => Seq(namedExpr)
+        }
+        aggregate.copy(aggregateExpressions = newExpressions.asInstanceOf[Seq[NamedExpression]])
+      case plan: LogicalPlan => plan
+    }
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e4f2843c/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 f698dd4..205b716 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
@@ -169,15 +169,15 @@ class CarbonSessionState(sparkSession: SparkSession) extends HiveSessionState(sp
         catalog.ParquetConversions ::
         catalog.OrcConversions ::
         CarbonPreInsertionCasts ::
+        CarbonPreAggregateQueryRules(sparkSession) ::
         CarbonIUDAnalysisRule(sparkSession) ::
         AnalyzeCreateTable(sparkSession) ::
         PreprocessTableInsertion(conf) ::
         DataSourceAnalysis(conf) ::
         (if (conf.runSQLonFile) {
           new ResolveDataSource(sparkSession) :: Nil
-        } else {
-          Nil
-        })
+        } else {  Nil }
+           )
 
       override val extendedCheckRules = Seq(
         PreWriteCheck(conf, catalog))

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e4f2843c/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 46a2515..3bed9d1 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
@@ -476,6 +476,14 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
     }
   }
 
+  def addPreAggFunction(sql: String): String = {
+    addPreAgg(new lexical.Scanner(sql.toLowerCase)) match {
+      case Success(query, _) => query
+      case failureOrError => throw new MalformedCarbonCommandException(
+        s"Unsupported query")
+    }
+  }
+
   def getBucketFields(
       properties: mutable.Map[String, String],
       fields: Seq[Field],

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e4f2843c/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 5c51156..401b149 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.{CarbonSession, DataFrame, Dataset, SparkSession}
+import org.apache.spark.sql.{CarbonEnv, 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._
@@ -74,6 +74,7 @@ class CarbonSqlAstBuilder(conf: SQLConf) extends SparkSqlAstBuilder(conf) {
   val parser = new CarbonSpark2SqlParser
 
   override def visitCreateTable(ctx: CreateTableContext): LogicalPlan = {
+    Option(ctx.query()).map(plan)
     val fileStorage = Option(ctx.createFileFormat) match {
       case Some(value) =>
         if (value.children.get(1).getText.equalsIgnoreCase("by")) {


[05/49] carbondata git commit: [CARBONDATA-1593] Add partition to table cause NoSuchTableException

Posted by ra...@apache.org.
[CARBONDATA-1593] Add partition to table cause NoSuchTableException

AlterTableSplitCarbonPartition's processSchema method doesn't provide db info to sparkSession.catalog.refreshTable, this will cause NoSuchTableException when we add partitions to carbondata table.

This closes #1452


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

Branch: refs/heads/fgdatamap
Commit: e9454499dcb89ed69b1e18f79b3003ea8e5d8d25
Parents: 0586146
Author: wyp <wy...@163.com>
Authored: Mon Oct 30 12:49:53 2017 +0800
Committer: lionelcao <wh...@gmail.com>
Committed: Thu Nov 2 10:10:45 2017 +0800

----------------------------------------------------------------------
 .../AlterTableSplitCarbonPartitionCommand.scala |  3 +-
 .../partition/TestAlterPartitionTable.scala     | 57 ++++++++++++++++++++
 2 files changed, 59 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/e9454499/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/AlterTableSplitCarbonPartitionCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/AlterTableSplitCarbonPartitionCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/AlterTableSplitCarbonPartitionCommand.scala
index e16dfc9..12bf31e 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/AlterTableSplitCarbonPartitionCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/AlterTableSplitCarbonPartitionCommand.scala
@@ -23,6 +23,7 @@ import java.util
 import scala.collection.JavaConverters._
 
 import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
+import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.execution.command.{AlterTableSplitPartitionModel, DataProcessCommand, RunnableCommand, SchemaProcessCommand}
 import org.apache.spark.sql.hive.{CarbonMetaStore, CarbonRelation}
 import org.apache.spark.util.{AlterTableUtil, PartitionUtils}
@@ -109,7 +110,7 @@ case class AlterTableSplitCarbonPartitionCommand(
     CarbonUtil.writeThriftTableToSchemaFile(schemaFilePath, thriftTable)
     // update the schema modified time
     carbonMetaStore.updateAndTouchSchemasUpdatedTime(storePath)
-    sparkSession.catalog.refreshTable(tableName)
+    sparkSession.sessionState.catalog.refreshTable(TableIdentifier(tableName, Option(dbName)))
     Seq.empty
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e9454499/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestAlterPartitionTable.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestAlterPartitionTable.scala b/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestAlterPartitionTable.scala
index 5e81044..04de9a3 100644
--- a/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestAlterPartitionTable.scala
+++ b/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestAlterPartitionTable.scala
@@ -779,6 +779,63 @@ class TestAlterPartitionTable extends QueryTest with BeforeAndAfterAll {
          .contains("Data in range info must be the same type with the partition field's type"))
   }
 
+  test("Add partition to table in or not in default database") {
+    sql("DROP TABLE IF EXISTS carbon_table_default_db")
+    sql(
+      """
+        | CREATE TABLE carbon_table_default_db(id INT, name STRING) PARTITIONED BY (dt STRING)
+        | STORED BY 'carbondata' TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='2015,2016')
+      """.stripMargin)
+    sql("ALTER TABLE carbon_table_default_db ADD PARTITION ('2017')")
+
+    val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default_carbon_table_default_db")
+    val partitionInfo = carbonTable.getPartitionInfo(carbonTable.getFactTableName)
+    val partitionIds = partitionInfo.getPartitionIds
+    val range_info = partitionInfo.getRangeInfo
+    assert(partitionIds == List(0, 1, 2, 3).map(Integer.valueOf(_)).asJava)
+    assert(partitionInfo.getMaxPartitionId == 3)
+    assert(partitionInfo.getNumPartitions == 4)
+    assert(range_info.get(0) == "2015")
+    assert(range_info.get(1) == "2016")
+    assert(range_info.get(2) == "2017")
+
+    sql("CREATE DATABASE IF NOT EXISTS carbondb")
+    sql("DROP TABLE IF EXISTS carbondb.carbontable")
+    sql(
+      """
+        | CREATE TABLE carbondb.carbontable(id INT, name STRING) PARTITIONED BY (dt STRING)
+        | STORED BY 'carbondata' TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='2015,2016')
+      """.stripMargin)
+    sql("ALTER TABLE carbondb.carbontable ADD PARTITION ('2017')")
+
+    val carbonTable1 = CarbonMetadata.getInstance().getCarbonTable("carbondb_carbontable")
+    val partitionInfo1 = carbonTable1.getPartitionInfo(carbonTable1.getFactTableName)
+    val partitionIds1 = partitionInfo1.getPartitionIds
+    val range_info1 = partitionInfo1.getRangeInfo
+    assert(partitionIds1 == List(0, 1, 2, 3).map(Integer.valueOf(_)).asJava)
+    assert(partitionInfo1.getMaxPartitionId == 3)
+    assert(partitionInfo1.getNumPartitions == 4)
+    assert(range_info1.get(0) == "2015")
+    assert(range_info1.get(1) == "2016")
+    assert(range_info1.get(2) == "2017")
+  }
+
+  test("test exception when alter partition's table doesn't exist in a perticular database") {
+    val exception_test_add_partition: Exception = intercept[Exception] {
+      sql("CREATE DATABASE IF NOT EXISTS carbondb")
+      sql("USE default")
+      sql(
+        """
+          | CREATE TABLE carbon_table_in_default_db(id INT, name STRING)
+          | PARTITIONED BY (dt STRING) STORED BY 'carbondata'
+          | TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='2015,2016')
+        """.stripMargin)
+      sql("ALTER TABLE carbondb.carbon_table_in_default_db ADD PARTITION ('2017')")
+    }
+    assert(exception_test_add_partition.getMessage
+      .contains("Table or view 'carbon_table_in_default_db' not found in database 'carbondb'"))
+  }
+
   def validateDataFiles(tableUniqueName: String, segmentId: String, partitions: Seq[Int]): Unit = {
     val carbonTable = CarbonMetadata.getInstance().getCarbonTable(tableUniqueName)
     val dataFiles = getDataFiles(carbonTable, segmentId)


[22/49] carbondata git commit: [CARBONDATA-1661] Fixed bug related to display of timestamp values in presto

Posted by ra...@apache.org.
[CARBONDATA-1661] Fixed bug related to display of timestamp values in presto

This closes #1459


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

Branch: refs/heads/fgdatamap
Commit: fa193318278ad7a85b887856c6126e9fdebfe41d
Parents: 4673113
Author: Geetika Gupta <ge...@knoldus.in>
Authored: Wed Nov 1 16:42:20 2017 +0530
Committer: chenliang613 <ch...@huawei.com>
Committed: Wed Nov 8 23:22:32 2017 +0800

----------------------------------------------------------------------
 .../java/org/apache/carbondata/presto/PrestoFilterUtil.java | 2 ++
 .../presto/integrationtest/PrestoAllDataTypeTest.scala      | 9 ++++++++-
 2 files changed, 10 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/fa193318/integration/presto/src/main/java/org/apache/carbondata/presto/PrestoFilterUtil.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/PrestoFilterUtil.java b/integration/presto/src/main/java/org/apache/carbondata/presto/PrestoFilterUtil.java
index 0982254..b8b2032 100644
--- a/integration/presto/src/main/java/org/apache/carbondata/presto/PrestoFilterUtil.java
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/PrestoFilterUtil.java
@@ -261,6 +261,8 @@ public class PrestoFilterUtil {
         return new BigDecimal(new BigInteger(String.valueOf(rawdata)),
             ((DecimalType) type).getScale());
       }
+    } else if (type.equals(TimestampType.TIMESTAMP)) {
+      return (Long)rawdata * 1000;
     }
 
     return rawdata;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fa193318/integration/presto/src/test/scala/org/apache/carbondata/presto/integrationtest/PrestoAllDataTypeTest.scala
----------------------------------------------------------------------
diff --git a/integration/presto/src/test/scala/org/apache/carbondata/presto/integrationtest/PrestoAllDataTypeTest.scala b/integration/presto/src/test/scala/org/apache/carbondata/presto/integrationtest/PrestoAllDataTypeTest.scala
index 433ddd4..5eb06e7 100644
--- a/integration/presto/src/test/scala/org/apache/carbondata/presto/integrationtest/PrestoAllDataTypeTest.scala
+++ b/integration/presto/src/test/scala/org/apache/carbondata/presto/integrationtest/PrestoAllDataTypeTest.scala
@@ -402,4 +402,11 @@ class PrestoAllDataTypeTest extends FunSuiteLike with BeforeAndAfterAll {
 
     assert(actualResult.equals(expectedResult))
   }
-}
\ No newline at end of file
+
+  test("test timestamp datatype using cast operator") {
+    val actualResult: List[Map[String, Any]] = PrestoServer
+      .executeQuery("SELECT NAME AS RESULT FROM TESTDB.TESTTABLE WHERE DOB = CAST('2016-04-14 15:00:09' AS TIMESTAMP)")
+    val expectedResult: List[Map[String, Any]] = List(Map("RESULT" -> "jatin"))
+    assert(actualResult.equals(expectedResult))
+  }
+}


[15/49] carbondata git commit: [CARBONDATA-1669] Clean up code in CarbonDataRDDFactory

Posted by ra...@apache.org.
[CARBONDATA-1669] Clean up code in CarbonDataRDDFactory

This closes #1467


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

Branch: refs/heads/fgdatamap
Commit: 0578ba0f2f9931a89f9759ea4be97975957280ae
Parents: 11661eb
Author: Jacky Li <ja...@qq.com>
Authored: Tue Nov 7 11:41:30 2017 +0800
Committer: QiangCai <qi...@qq.com>
Committed: Tue Nov 7 19:47:17 2017 +0800

----------------------------------------------------------------------
 .../carbondata/core/locks/CarbonLockUtil.java   |   19 +-
 .../org/apache/carbondata/api/CarbonStore.scala |   39 +-
 .../spark/rdd/DataManagementFunc.scala          |  231 +--
 .../carbondata/spark/rdd/UpdateDataLoad.scala   |    3 +-
 .../spark/rdd/CarbonDataRDDFactory.scala        | 1444 ++++++++----------
 .../spark/sql/CarbonCatalystOperators.scala     |    1 -
 .../AlterTableCompactionCommand.scala           |  124 +-
 .../AlterTableDropCarbonPartitionCommand.scala  |  133 +-
 .../AlterTableSplitCarbonPartitionCommand.scala |  156 +-
 .../restructure/AlterTableRevertTestCase.scala  |    2 +-
 10 files changed, 1044 insertions(+), 1108 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/0578ba0f/core/src/main/java/org/apache/carbondata/core/locks/CarbonLockUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/locks/CarbonLockUtil.java b/core/src/main/java/org/apache/carbondata/core/locks/CarbonLockUtil.java
index 60a7564..c02a168 100644
--- a/core/src/main/java/org/apache/carbondata/core/locks/CarbonLockUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/locks/CarbonLockUtil.java
@@ -65,20 +65,27 @@ public class CarbonLockUtil {
   /**
    * Given a lock type this method will return a new lock object if not acquired by any other
    * operation
-   *
-   * @param identifier
-   * @param lockType
-   * @return
    */
-  public static ICarbonLock getLockObject(CarbonTableIdentifier identifier, String lockType) {
+  public static ICarbonLock getLockObject(CarbonTableIdentifier identifier, String lockType,
+      String errorMsg) {
     ICarbonLock carbonLock = CarbonLockFactory.getCarbonLockObj(identifier, lockType);
     LOGGER.info("Trying to acquire lock: " + carbonLock);
     if (carbonLock.lockWithRetries()) {
       LOGGER.info("Successfully acquired the lock " + carbonLock);
     } else {
-      throw new RuntimeException("Table is locked for updation. Please try after some time");
+      LOGGER.error(errorMsg);
+      throw new RuntimeException(errorMsg);
     }
     return carbonLock;
   }
 
+  /**
+   * Get and lock with default error message
+   */
+  public static ICarbonLock getLockObject(CarbonTableIdentifier identifier, String lockType) {
+    return getLockObject(identifier,
+        lockType,
+        "Acquire table lock failed after retry, please try after some time");
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/0578ba0f/integration/spark-common/src/main/scala/org/apache/carbondata/api/CarbonStore.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/api/CarbonStore.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/api/CarbonStore.scala
index 4a66d0f..e77f5c3 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/api/CarbonStore.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/api/CarbonStore.scala
@@ -28,8 +28,11 @@ import org.apache.spark.sql.types.TimestampType
 
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonMetadata}
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.locks.{CarbonLockFactory, CarbonLockUtil, ICarbonLock, LockUsage}
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier}
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+import org.apache.carbondata.core.mutate.CarbonUpdateUtil
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager
 import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 import org.apache.carbondata.spark.rdd.DataManagementFunc
@@ -79,16 +82,34 @@ object CarbonStore {
       dbName: String,
       tableName: String,
       storePath: String,
-      carbonTable: CarbonTable, forceTableClean: Boolean): Unit = {
+      carbonTable: CarbonTable,
+      forceTableClean: Boolean): Unit = {
     LOGGER.audit(s"The clean files request has been received for $dbName.$tableName")
+    var carbonCleanFilesLock: ICarbonLock = null
+    val identifier = new CarbonTableIdentifier(dbName, tableName, "")
     try {
-      DataManagementFunc.cleanFiles(dbName, tableName, storePath, carbonTable, forceTableClean)
-      LOGGER.audit(s"Clean files operation is success for $dbName.$tableName.")
-    } catch {
-      case ex: Exception =>
-        sys.error(ex.getMessage)
+      val errorMsg = "Clean files request is failed for " +
+                     s"$dbName.$tableName" +
+                     ". Not able to acquire the clean files lock due to another clean files " +
+                     "operation is running in the background."
+      carbonCleanFilesLock =
+        CarbonLockUtil.getLockObject(identifier, LockUsage.CLEAN_FILES_LOCK, errorMsg)
+      if (forceTableClean) {
+        val absIdent = AbsoluteTableIdentifier.from(storePath, dbName, tableName)
+        FileFactory.deleteAllCarbonFilesOfDir(
+          FileFactory.getCarbonFile(absIdent.getTablePath,
+            FileFactory.getFileType(absIdent.getTablePath)))
+      } else {
+        DataManagementFunc.deleteLoadsAndUpdateMetadata(dbName, tableName, storePath,
+          isForceDeletion = true, carbonTable)
+        CarbonUpdateUtil.cleanUpDeltaFiles(carbonTable, true)
+      }
+    } finally {
+      if (carbonCleanFilesLock != null) {
+        CarbonLockUtil.fileUnlock(carbonCleanFilesLock, LockUsage.CLEAN_FILES_LOCK)
+      }
     }
-    Seq.empty
+    LOGGER.audit(s"Clean files operation is success for $dbName.$tableName.")
   }
 
   // validates load ids
@@ -163,7 +184,7 @@ object CarbonStore {
     val validAndInvalidSegments: SegmentStatusManager.ValidAndInvalidSegmentsInfo = new
         SegmentStatusManager(
           identifier).getValidAndInvalidSegments
-    return validAndInvalidSegments.getValidSegments.contains(segmentId)
+    validAndInvalidSegments.getValidSegments.contains(segmentId)
   }
 
   private def validateTimeFormat(timestamp: String): Long = {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/0578ba0f/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/DataManagementFunc.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/DataManagementFunc.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/DataManagementFunc.scala
index c2029e5..cbdb336 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/DataManagementFunc.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/DataManagementFunc.scala
@@ -21,25 +21,20 @@ import java.util
 import java.util.concurrent._
 
 import scala.collection.JavaConverters._
-import scala.collection.mutable.ListBuffer
 
 import org.apache.spark.sql.SQLContext
-import org.apache.spark.sql.execution.command.{CompactionCallableModel, CompactionModel, DropPartitionCallableModel, SplitPartitionCallableModel}
+import org.apache.spark.sql.execution.command.{CompactionCallableModel, CompactionModel}
 
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.locks.{CarbonLockFactory, CarbonLockUtil, LockUsage}
-import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier}
+import org.apache.carbondata.core.metadata.CarbonTableIdentifier
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
-import org.apache.carbondata.core.mutate.CarbonUpdateUtil
 import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
-import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, CarbonLoadModel}
+import org.apache.carbondata.processing.loading.model.CarbonLoadModel
 import org.apache.carbondata.processing.merger.{CarbonDataMergerUtil, CompactionType}
 import org.apache.carbondata.processing.util.{CarbonLoaderUtil, DeleteLoadFolders, LoadMetadataUtil}
-import org.apache.carbondata.spark._
 import org.apache.carbondata.spark.compaction.CompactionCallable
-import org.apache.carbondata.spark.partition.{DropPartitionCallable, SplitPartitionCallable}
 import org.apache.carbondata.spark.util.CommonUtil
 
 /**
@@ -49,105 +44,6 @@ object DataManagementFunc {
 
   private val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
 
-  def deleteLoadByDate(
-      sqlContext: SQLContext,
-      schema: CarbonDataLoadSchema,
-      databaseName: String,
-      tableName: String,
-      storePath: String,
-      dateField: String,
-      dateFieldActualName: String,
-      dateValue: String) {
-
-    val sc = sqlContext
-    // Delete the records based on data
-    val table = schema.getCarbonTable
-    val loadMetadataDetailsArray =
-      SegmentStatusManager.readLoadMetadata(table.getMetaDataFilepath).toList
-    val resultMap = new CarbonDeleteLoadByDateRDD(
-      sc.sparkContext,
-      new DeletedLoadResultImpl(),
-      databaseName,
-      table.getDatabaseName,
-      dateField,
-      dateFieldActualName,
-      dateValue,
-      table.getFactTableName,
-      tableName,
-      storePath,
-      loadMetadataDetailsArray).collect.groupBy(_._1)
-
-    var updatedLoadMetadataDetailsList = new ListBuffer[LoadMetadataDetails]()
-    if (resultMap.nonEmpty) {
-      if (resultMap.size == 1) {
-        if (resultMap.contains("")) {
-          LOGGER.error("Delete by Date request is failed")
-          sys.error("Delete by Date request is failed, potential causes " +
-              "Empty store or Invalid column type, For more details please refer logs.")
-        }
-      }
-      val updatedloadMetadataDetails = loadMetadataDetailsArray.map { elem => {
-        var statusList = resultMap.get(elem.getLoadName)
-        // check for the merged load folder.
-        if (statusList.isEmpty && null != elem.getMergedLoadName) {
-          statusList = resultMap.get(elem.getMergedLoadName)
-        }
-
-        if (statusList.isDefined) {
-          elem.setModificationOrdeletionTimesStamp(elem.getTimeStamp(CarbonLoaderUtil
-            .readCurrentTime()))
-          // if atleast on CarbonCommonConstants.MARKED_FOR_UPDATE status exist,
-          // use MARKED_FOR_UPDATE
-          if (statusList.get
-              .forall(status => status._2 == CarbonCommonConstants.MARKED_FOR_DELETE)) {
-            elem.setLoadStatus(CarbonCommonConstants.MARKED_FOR_DELETE)
-          } else {
-            elem.setLoadStatus(CarbonCommonConstants.MARKED_FOR_UPDATE)
-            updatedLoadMetadataDetailsList += elem
-          }
-          elem
-        } else {
-          elem
-        }
-      }
-
-      }
-
-      // Save the load metadata
-      val carbonLock = CarbonLockFactory
-          .getCarbonLockObj(table.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
-            LockUsage.METADATA_LOCK
-          )
-      try {
-        if (carbonLock.lockWithRetries()) {
-          LOGGER.info("Successfully got the table metadata file lock")
-          if (updatedLoadMetadataDetailsList.nonEmpty) {
-            // TODO: Load Aggregate tables after retention.
-          }
-
-          // write
-          CarbonLoaderUtil.writeLoadMetadata(
-            storePath,
-            databaseName,
-            table.getDatabaseName,
-            updatedloadMetadataDetails.asJava
-          )
-        }
-      } finally {
-        if (carbonLock.unlock()) {
-          LOGGER.info("unlock the table metadata file successfully")
-        } else {
-          LOGGER.error("Unable to unlock the metadata lock")
-        }
-      }
-    } else {
-      LOGGER.error("Delete by Date request is failed")
-      LOGGER.audit(s"The delete load by date is failed for $databaseName.$tableName")
-      sys.error("Delete by Date request is failed, potential causes " +
-          "Empty store or Invalid column type, For more details please refer logs.")
-    }
-  }
-
   def executeCompaction(carbonLoadModel: CarbonLoadModel,
       compactionModel: CompactionModel,
       executor: ExecutorService,
@@ -226,8 +122,6 @@ object DataManagementFunc {
 
   /**
    * This will submit the loads to be merged into the executor.
-   *
-   * @param futureList
    */
   private def scanSegmentsAndSubmitJob(futureList: util.List[Future[Void]],
       loadsToMerge: util.List[LoadMetadataDetails],
@@ -235,14 +129,14 @@ object DataManagementFunc {
       sqlContext: SQLContext,
       compactionModel: CompactionModel,
       carbonLoadModel: CarbonLoadModel,
-      storeLocation: String): Unit = {
-
-    loadsToMerge.asScala.foreach(seg => {
+      storeLocation: String
+  ): Unit = {
+    loadsToMerge.asScala.foreach { seg =>
       LOGGER.info("loads identified for merge is " + seg.getLoadName)
     }
-    )
 
-    val compactionCallableModel = CompactionCallableModel(carbonLoadModel,
+    val compactionCallableModel = CompactionCallableModel(
+      carbonLoadModel,
       storeLocation,
       compactionModel.carbonTable,
       loadsToMerge,
@@ -254,80 +148,6 @@ object DataManagementFunc {
     futureList.add(future)
   }
 
-  def executePartitionSplit( sqlContext: SQLContext,
-      carbonLoadModel: CarbonLoadModel,
-      executor: ExecutorService,
-      segment: String,
-      partitionId: String,
-      oldPartitionIdList: List[Int]): Unit = {
-    val futureList: util.List[Future[Void]] = new util.ArrayList[Future[Void]](
-      CarbonCommonConstants.DEFAULT_COLLECTION_SIZE
-    )
-    scanSegmentsForSplitPartition(futureList, executor, segment, partitionId,
-      sqlContext, carbonLoadModel, oldPartitionIdList)
-    try {
-        futureList.asScala.foreach(future => {
-          future.get
-        }
-      )
-    } catch {
-      case e: Exception =>
-        LOGGER.error(e, s"Exception in partition split thread ${ e.getMessage }")
-        throw e
-    }
-  }
-
-  private def scanSegmentsForSplitPartition(futureList: util.List[Future[Void]],
-      executor: ExecutorService,
-      segmentId: String,
-      partitionId: String,
-      sqlContext: SQLContext,
-      carbonLoadModel: CarbonLoadModel,
-      oldPartitionIdList: List[Int]): Unit = {
-
-    val splitModel = SplitPartitionCallableModel(carbonLoadModel,
-      segmentId,
-      partitionId,
-      oldPartitionIdList,
-      sqlContext)
-
-    val future: Future[Void] = executor.submit(new SplitPartitionCallable(splitModel))
-    futureList.add(future)
-  }
-
-  def executeDroppingPartition(sqlContext: SQLContext,
-      carbonLoadModel: CarbonLoadModel,
-      executor: ExecutorService,
-      segmentId: String,
-      partitionId: String,
-      dropWithData: Boolean,
-      oldPartitionIds: List[Int]): Unit = {
-    val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
-    val model = new DropPartitionCallableModel(carbonLoadModel,
-      segmentId, partitionId, oldPartitionIds, dropWithData, carbonTable, sqlContext)
-    val future: Future[Void] = executor.submit(new DropPartitionCallable(model))
-    try {
-        future.get
-    } catch {
-      case e: Exception =>
-        LOGGER.error(e, s"Exception in partition drop thread ${ e.getMessage }")
-        throw e
-    }
-  }
-
-  def prepareCarbonLoadModel(table: CarbonTable, newCarbonLoadModel: CarbonLoadModel): Unit = {
-    newCarbonLoadModel.setTableName(table.getFactTableName)
-    val dataLoadSchema = new CarbonDataLoadSchema(table)
-    // Need to fill dimension relation
-    newCarbonLoadModel.setCarbonDataLoadSchema(dataLoadSchema)
-    newCarbonLoadModel.setTableName(table.getCarbonTableIdentifier.getTableName)
-    newCarbonLoadModel.setDatabaseName(table.getCarbonTableIdentifier.getDatabaseName)
-    newCarbonLoadModel.setStorePath(table.getStorePath)
-    CommonUtil.readLoadMetadataDetails(newCarbonLoadModel)
-    val loadStartTime = CarbonUpdateUtil.readCurrentTime();
-    newCarbonLoadModel.setFactTimeStamp(loadStartTime)
-  }
-
   def deletePartialLoadsInCompaction(carbonLoadModel: CarbonLoadModel): Unit = {
     // Deleting the any partially loaded data if present.
     // in some case the segment folder which is present in store will not have entry in
@@ -397,39 +217,4 @@ object DataManagementFunc {
     }
   }
 
-  def cleanFiles(
-      dbName: String,
-      tableName: String,
-      storePath: String,
-      carbonTable: CarbonTable,
-      forceTableClean: Boolean): Unit = {
-    val identifier = new CarbonTableIdentifier(dbName, tableName, "")
-    val carbonCleanFilesLock =
-      CarbonLockFactory.getCarbonLockObj(identifier, LockUsage.CLEAN_FILES_LOCK)
-    try {
-      if (carbonCleanFilesLock.lockWithRetries()) {
-        LOGGER.info("Clean files lock has been successfully acquired.")
-        if (forceTableClean) {
-          val absIdent = AbsoluteTableIdentifier.from(storePath, dbName, tableName)
-          FileFactory.deleteAllCarbonFilesOfDir(
-            FileFactory.getCarbonFile(absIdent.getTablePath,
-            FileFactory.getFileType(absIdent.getTablePath)))
-        } else {
-          deleteLoadsAndUpdateMetadata(dbName, tableName, storePath,
-            isForceDeletion = true, carbonTable)
-          CarbonUpdateUtil.cleanUpDeltaFiles(carbonTable, true)
-        }
-      } else {
-        val errorMsg = "Clean files request is failed for " +
-            s"$dbName.$tableName" +
-            ". Not able to acquire the clean files lock due to another clean files " +
-            "operation is running in the background."
-        LOGGER.audit(errorMsg)
-        LOGGER.error(errorMsg)
-        throw new Exception(errorMsg + " Please try after some time.")
-      }
-    } finally {
-      CarbonLockUtil.fileUnlock(carbonCleanFilesLock, LockUsage.CLEAN_FILES_LOCK)
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/0578ba0f/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/UpdateDataLoad.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/UpdateDataLoad.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/UpdateDataLoad.scala
index 4d782c9..eb07240 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/UpdateDataLoad.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/UpdateDataLoad.scala
@@ -35,7 +35,8 @@ import org.apache.carbondata.processing.util.CarbonLoaderUtil
  */
 object UpdateDataLoad {
 
-  def DataLoadForUpdate(segId: String,
+  def DataLoadForUpdate(
+      segId: String,
       index: Int,
       iter: Iterator[Row],
       carbonLoadModel: CarbonLoadModel,


[40/49] 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/3d1d1ce8
Tree: http://git-wip-us.apache.org/repos/asf/carbondata/tree/3d1d1ce8
Diff: http://git-wip-us.apache.org/repos/asf/carbondata/diff/3d1d1ce8

Branch: refs/heads/fgdatamap
Commit: 3d1d1ce85a5905f0877ce3e367f255fd71e18ad2
Parents: 3169918
Author: kumarvishal <ku...@gmail.com>
Authored: Sun Oct 15 18:05:55 2017 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Tue Nov 14 00:48:16 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/3d1d1ce8/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/3d1d1ce8/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/3d1d1ce8/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/3d1d1ce8/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/3d1d1ce8/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/3d1d1ce8/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/3d1d1ce8/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/3d1d1ce8/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/3d1d1ce8/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/3d1d1ce8/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/3d1d1ce8/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/3d1d1ce8/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/3d1d1ce8/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/3d1d1ce8/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/3d1d1ce8/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/3d1d1ce8/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)
     }
   }


[08/49] carbondata git commit: [CARBONDATA-1594] Add precision and scale to DecimalType

Posted by ra...@apache.org.
[CARBONDATA-1594] Add precision and scale to DecimalType

Refactor on DecimalType to include precision and scale parameter.
Precision and scale parameter is required for Decimal data type. In earlier code, they are stored in following classes:

ColumnSpec
ColumnPageEncoderMeta
PrimitivePageStatsCollector
ColumnSchema
Since now we have changed DataType from enum to class, precision and scale should be stored in DecimalType object only. The PR does this change.

No new test case is added in this PR since no functionality change.

This closes #1417


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

Branch: refs/heads/fgdatamap
Commit: f209e8ee315a272f1f60a7a037d6c15fc08b6add
Parents: e945449
Author: Jacky Li <ja...@qq.com>
Authored: Wed Nov 1 00:18:47 2017 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Nov 2 14:20:30 2017 +0530

----------------------------------------------------------------------
 .../cache/dictionary/ColumnDictionaryInfo.java  |   2 +-
 .../carbondata/core/datastore/TableSpec.java    |  75 ++++---
 .../core/datastore/block/SegmentProperties.java |   2 +-
 .../core/datastore/page/ColumnPage.java         |  23 +--
 .../core/datastore/page/LazyColumnPage.java     |   2 +-
 .../datastore/page/UnsafeDecimalColumnPage.java |  12 +-
 .../page/UnsafeFixLengthColumnPage.java         |   2 +-
 .../datastore/page/VarLengthColumnPageBase.java |  15 +-
 .../page/encoding/ColumnPageEncoder.java        |   4 +-
 .../page/encoding/ColumnPageEncoderMeta.java    |  46 +++--
 .../page/encoding/DefaultEncodingFactory.java   |   2 +-
 .../page/encoding/EncodingFactory.java          |   6 +-
 .../adaptive/AdaptiveDeltaIntegralCodec.java    |   4 +-
 .../adaptive/AdaptiveIntegralCodec.java         |   2 +-
 .../encoding/compress/DirectCompressCodec.java  |   2 +-
 .../page/statistics/KeyPageStatsCollector.java  |   7 -
 .../page/statistics/LVStringStatsCollector.java |   7 -
 .../statistics/PrimitivePageStatsCollector.java |  44 ++--
 .../page/statistics/SimpleStatsResult.java      |   3 -
 .../blockletindex/BlockletDataMap.java          |   4 +-
 .../ThriftWrapperSchemaConverterImpl.java       |  32 ++-
 .../core/metadata/datatype/DataType.java        |   5 +-
 .../core/metadata/datatype/DataTypes.java       |  23 ++-
 .../datatype/DecimalConverterFactory.java       |  39 +---
 .../core/metadata/datatype/DecimalType.java     |  28 ++-
 .../schema/table/column/ColumnSchema.java       |  41 ++--
 .../impl/AbstractScannedResultCollector.java    |   4 +-
 .../RestructureBasedVectorResultCollector.java  |   2 +-
 .../scan/executor/util/RestructureUtil.java     |   4 +-
 .../core/scan/expression/ExpressionResult.java  |   6 +-
 .../conditional/EqualToExpression.java          |   2 +-
 .../GreaterThanEqualToExpression.java           |   2 +-
 .../conditional/GreaterThanExpression.java      |   2 +-
 .../expression/conditional/InExpression.java    |   2 +-
 .../conditional/LessThanEqualToExpression.java  |   2 +-
 .../conditional/LessThanExpression.java         |   2 +-
 .../conditional/NotEqualsExpression.java        |   2 +-
 .../expression/conditional/NotInExpression.java |   2 +-
 .../carbondata/core/scan/filter/FilterUtil.java |   4 +-
 .../executer/ExcludeFilterExecuterImpl.java     |  22 +-
 .../executer/IncludeFilterExecuterImpl.java     |   4 +-
 .../executer/RowLevelFilterExecuterImpl.java    |   6 +-
 .../core/scan/partition/PartitionUtil.java      |   4 +-
 .../vector/MeasureDataVectorProcessor.java      |   2 +-
 .../util/AbstractDataFileFooterConverter.java   |  12 +-
 .../core/util/CarbonMetadataUtil.java           |   2 +-
 .../apache/carbondata/core/util/CarbonUtil.java |  29 ++-
 .../carbondata/core/util/DataTypeUtil.java      |  98 +--------
 .../core/util/comparator/Comparator.java        |   4 +-
 .../sortindex/CarbonDictionarySortModel.java    |   2 +-
 .../dictionary/ColumnDictionaryInfoTest.java    |   3 +-
 .../datastore/page/encoding/RLECodecSuite.java  |   6 +-
 .../ThriftWrapperSchemaConverterImplTest.java   |  20 +-
 .../scan/executor/util/RestructureUtilTest.java |   5 +
 .../scan/expression/ExpressionResultTest.java   |  13 +-
 .../conditional/EqualToExpressionUnitTest.java  |   3 +-
 .../GreaterThanEqualToExpressionUnitTest.java   |   5 +-
 .../GreaterThanExpressionUnitTest.java          |   5 +-
 .../conditional/InExpressionUnitTest.java       |   5 +-
 .../LessThanEqualToExpressionUnitTest.java      |   5 +-
 .../conditional/LessThanExpressionUnitTest.java |   5 +-
 .../NotEqualsExpressionUnitTest.java            |   5 +-
 .../conditional/NotInExpressionUnitTest.java    |   5 +-
 .../core/scan/filter/FilterUtilTest.java        |   3 +-
 .../core/util/CarbonMetadataUtilTest.java       |   2 +-
 .../carbondata/core/util/CarbonUtilTest.java    |   3 +-
 .../carbondata/core/util/DataTypeUtilTest.java  |  23 +--
 .../CarbonDictionarySortModelTest.java          |  33 +--
 .../hive/CarbonDictionaryDecodeReadSupport.java |   2 +-
 .../carbondata/presto/CarbonTypeUtil.java       |   2 +-
 .../presto/CarbonVectorizedRecordReader.java    |   2 +-
 .../presto/CarbondataColumnHandle.java          |   6 +-
 .../carbondata/presto/CarbondataMetadata.java   |   2 +-
 .../carbondata/presto/PrestoFilterUtil.java     |   5 +-
 .../presto/util/CarbonDataStoreCreator.scala    |   4 +-
 .../carbondata/spark/util/CarbonScalaUtil.scala |  43 ++--
 .../spark/util/DataTypeConverterUtil.scala      |  37 ++--
 .../spark/sql/catalyst/CarbonDDLSqlParser.scala |   2 +-
 .../command/carbonTableSchemaCommon.scala       | 201 +++++++++----------
 .../VectorizedCarbonRecordReader.java           |   2 +-
 .../spark/sql/CarbonDataFrameWriter.scala       |   3 +-
 .../spark/sql/CarbonDictionaryDecoder.scala     | 145 ++++++-------
 .../apache/spark/sql/hive/CarbonRelation.scala  |   7 +-
 .../partition/impl/HashPartitionerImpl.java     |   2 +-
 .../sort/unsafe/UnsafeCarbonRowPage.java        |   6 +-
 .../holder/UnsafeSortTempFileChunkHolder.java   |   2 +-
 .../merger/UnsafeIntermediateFileMerger.java    |   2 +-
 .../merger/CompactionResultSortProcessor.java   |   2 +-
 .../sort/sortdata/IntermediateFileMerger.java   |   2 +-
 .../processing/sort/sortdata/SortDataRows.java  |   2 +-
 .../sort/sortdata/SortTempFileChunkHolder.java  |   2 +-
 .../store/CarbonFactDataHandlerColumnar.java    |   2 +-
 .../carbondata/processing/store/TablePage.java  |   7 +-
 93 files changed, 623 insertions(+), 671 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ColumnDictionaryInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ColumnDictionaryInfo.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ColumnDictionaryInfo.java
index 223812e..db49fa1 100644
--- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ColumnDictionaryInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ColumnDictionaryInfo.java
@@ -291,7 +291,7 @@ public class ColumnDictionaryInfo extends AbstractColumnDictionaryInfo {
         dateToStr = parser.parse(memberVal);
         dictionaryDate = parser.parse(dictionaryVal);
         return dictionaryDate.compareTo(dateToStr);
-      } else if (dataType == DataTypes.DECIMAL) {
+      } else if (DataTypes.isDecimal(dataType)) {
         java.math.BigDecimal javaDecValForDictVal = new java.math.BigDecimal(dictionaryVal);
         java.math.BigDecimal javaDecValForMemberVal = new java.math.BigDecimal(memberVal);
         return javaDecValForDictVal.compareTo(javaDecValForMemberVal);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/main/java/org/apache/carbondata/core/datastore/TableSpec.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/TableSpec.java b/core/src/main/java/org/apache/carbondata/core/datastore/TableSpec.java
index 9f29e27..eb36c8d 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/TableSpec.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/TableSpec.java
@@ -24,6 +24,7 @@ import java.util.List;
 
 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.schema.table.Writable;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
@@ -84,8 +85,7 @@ public class TableSpec {
   private void addMeasures(List<CarbonMeasure> measures) {
     for (int i = 0; i < measures.size(); i++) {
       CarbonMeasure measure = measures.get(i);
-      measureSpec[i] = new MeasureSpec(measure.getColName(), measure.getDataType(), measure
-          .getScale(), measure.getPrecision());
+      measureSpec[i] = new MeasureSpec(measure.getColName(), measure.getDataType());
     }
   }
 
@@ -122,27 +122,29 @@ public class TableSpec {
     // dimension type of this dimension
     private ColumnType columnType;
 
-    // scale and precision is for decimal column only
-    // TODO: make DataType a class instead of enum
-    private int scale;
-    private int precision;
-
     public ColumnSpec() {
     }
 
-    public ColumnSpec(String fieldName, DataType schemaDataType, ColumnType columnType) {
-      // for backward compatibility as the precision and scale is not stored, the values should be
-      // initialized with -1 for both precision and scale
-      this(fieldName, schemaDataType, columnType, -1, -1);
-    }
-
-    public ColumnSpec(String fieldName, DataType schemaDataType, ColumnType columnType,
-        int scale, int precision) {
+    private ColumnSpec(String fieldName, DataType schemaDataType, ColumnType columnType) {
       this.fieldName = fieldName;
       this.schemaDataType = schemaDataType;
       this.columnType = columnType;
-      this.scale = scale;
-      this.precision = precision;
+    }
+
+    public static ColumnSpec newInstance(String fieldName, DataType schemaDataType,
+        ColumnType columnType) {
+      return new ColumnSpec(fieldName, schemaDataType, columnType);
+    }
+
+    public static ColumnSpec newInstanceLegacy(String fieldName, DataType schemaDataType,
+        ColumnType columnType) {
+      // for backward compatibility as the precision and scale is not stored, the values should be
+      // initialized with -1 for both precision and scale
+      if (schemaDataType instanceof DecimalType) {
+        ((DecimalType) schemaDataType).setPrecision(-1);
+        ((DecimalType) schemaDataType).setScale(-1);
+      }
+      return new ColumnSpec(fieldName, schemaDataType, columnType);
     }
 
     public DataType getSchemaDataType() {
@@ -158,11 +160,21 @@ public class TableSpec {
     }
 
     public int getScale() {
-      return scale;
+      if (DataTypes.isDecimal(schemaDataType)) {
+        return ((DecimalType) schemaDataType).getScale();
+      } else if (schemaDataType == DataTypes.BYTE_ARRAY) {
+        return -1;
+      }
+      throw new UnsupportedOperationException();
     }
 
     public int getPrecision() {
-      return precision;
+      if (DataTypes.isDecimal(schemaDataType)) {
+        return ((DecimalType) schemaDataType).getPrecision();
+      } else if (schemaDataType == DataTypes.BYTE_ARRAY) {
+        return -1;
+      }
+      throw new UnsupportedOperationException();
     }
 
     @Override
@@ -170,8 +182,14 @@ public class TableSpec {
       out.writeUTF(fieldName);
       out.writeByte(schemaDataType.getId());
       out.writeByte(columnType.ordinal());
-      out.writeInt(scale);
-      out.writeInt(precision);
+      if (DataTypes.isDecimal(schemaDataType)) {
+        DecimalType decimalType = (DecimalType) schemaDataType;
+        out.writeInt(decimalType.getScale());
+        out.writeInt(decimalType.getPrecision());
+      } else {
+        out.writeInt(-1);
+        out.writeInt(-1);
+      }
     }
 
     @Override
@@ -179,8 +197,13 @@ public class TableSpec {
       this.fieldName = in.readUTF();
       this.schemaDataType = DataTypes.valueOf(in.readByte());
       this.columnType = ColumnType.valueOf(in.readByte());
-      this.scale = in.readInt();
-      this.precision = in.readInt();
+      int scale = in.readInt();
+      int precision = in.readInt();
+      if (DataTypes.isDecimal(this.schemaDataType)) {
+        DecimalType decimalType = (DecimalType) this.schemaDataType;
+        decimalType.setPrecision(precision);
+        decimalType.setScale(scale);
+      }
     }
   }
 
@@ -193,7 +216,7 @@ public class TableSpec {
     private boolean doInvertedIndex;
 
     DimensionSpec(ColumnType columnType, CarbonDimension dimension) {
-      super(dimension.getColName(), dimension.getDataType(), columnType, 0, 0);
+      super(dimension.getColName(), dimension.getDataType(), columnType);
       this.inSortColumns = dimension.isSortColumn();
       this.doInvertedIndex = dimension.isUseInvertedIndex();
     }
@@ -219,8 +242,8 @@ public class TableSpec {
 
   public class MeasureSpec extends ColumnSpec implements Writable {
 
-    MeasureSpec(String fieldName, DataType dataType, int scale, int precision) {
-      super(fieldName, dataType, ColumnType.MEASURE, scale, precision);
+    MeasureSpec(String fieldName, DataType dataType) {
+      super(fieldName, dataType, ColumnType.MEASURE);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/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 2ac200f..c93b771 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
@@ -669,7 +669,7 @@ public class SegmentProperties {
     int k = eachDimColumnValueSize.length + eachComplexDimColumnValueSize.length;
     for (int i = 0; i < measures.size(); i++) {
       DataType dataType = measures.get(i).getDataType();
-      if (dataType.equals(DataTypes.DECIMAL)) {
+      if (DataTypes.isDecimal(dataType)) {
         dimensionValueSize[k++] = -1;
       } else {
         dimensionValueSize[k++] = 8;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/main/java/org/apache/carbondata/core/datastore/page/ColumnPage.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/ColumnPage.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/ColumnPage.java
index 35bc560..58627d0 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/page/ColumnPage.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/ColumnPage.java
@@ -36,7 +36,6 @@ import org.apache.carbondata.core.util.CarbonProperties;
 
 import static org.apache.carbondata.core.metadata.datatype.DataTypes.BYTE;
 import static org.apache.carbondata.core.metadata.datatype.DataTypes.BYTE_ARRAY;
-import static org.apache.carbondata.core.metadata.datatype.DataTypes.DECIMAL;
 import static org.apache.carbondata.core.metadata.datatype.DataTypes.DOUBLE;
 import static org.apache.carbondata.core.metadata.datatype.DataTypes.FLOAT;
 import static org.apache.carbondata.core.metadata.datatype.DataTypes.INT;
@@ -96,13 +95,6 @@ public abstract class ColumnPage {
       return BYTE_ARRAY;
     }
 
-    @Override public int getScale() {
-      return 0;
-    }
-
-    @Override public int getPrecision() {
-      return 0;
-    }
   };
 
   public SimpleStatsResult getStatistics() {
@@ -163,7 +155,7 @@ public abstract class ColumnPage {
 
   private static ColumnPage createPage(TableSpec.ColumnSpec columnSpec, DataType dataType,
       int pageSize) {
-    if (dataType.equals(DECIMAL)) {
+    if (DataTypes.isDecimal(dataType)) {
       return createDecimalPage(columnSpec, dataType, pageSize);
     } else if (dataType.equals(BYTE_ARRAY)) {
       return createVarLengthPage(columnSpec, dataType, pageSize);
@@ -195,7 +187,7 @@ public abstract class ColumnPage {
           dataType == DataTypes.FLOAT ||
           dataType == DataTypes.DOUBLE) {
         instance = new UnsafeFixLengthColumnPage(columnSpec, dataType, pageSize);
-      } else if (dataType == DataTypes.DECIMAL) {
+      } else if (DataTypes.isDecimal(dataType)) {
         instance = new UnsafeDecimalColumnPage(columnSpec, dataType, pageSize);
       } else if (dataType == DataTypes.STRING || dataType == DataTypes.BYTE_ARRAY) {
         instance = new UnsafeVarLengthColumnPage(columnSpec, dataType, pageSize);
@@ -217,7 +209,7 @@ public abstract class ColumnPage {
         instance = newFloatPage(columnSpec, new float[pageSize]);
       } else if (dataType == DataTypes.DOUBLE) {
         instance = newDoublePage(columnSpec, new double[pageSize]);
-      } else if (dataType == DataTypes.DECIMAL) {
+      } else if (DataTypes.isDecimal(dataType)) {
         instance = newDecimalPage(columnSpec, new byte[pageSize][]);
       } else if (dataType == DataTypes.STRING || dataType == DataTypes.BYTE_ARRAY) {
         instance = new SafeVarLengthColumnPage(columnSpec, dataType, pageSize);
@@ -277,7 +269,8 @@ public abstract class ColumnPage {
   }
 
   private static ColumnPage newDecimalPage(TableSpec.ColumnSpec columnSpec, byte[][] byteArray) {
-    ColumnPage columnPage = createPage(columnSpec, DECIMAL, byteArray.length);
+    ColumnPage columnPage =
+        createPage(columnSpec, columnSpec.getSchemaDataType(), byteArray.length);
     columnPage.setByteArrayPage(byteArray);
     return columnPage;
   }
@@ -365,7 +358,7 @@ public abstract class ColumnPage {
     } else if (dataType == DataTypes.DOUBLE) {
       putDouble(rowId, (double) value);
       statsCollector.update((double) value);
-    } else if (dataType == DataTypes.DECIMAL) {
+    } else if (DataTypes.isDecimal(dataType)) {
       putDecimal(rowId, (BigDecimal) value);
       statsCollector.update((BigDecimal) value);
     } else if (dataType == DataTypes.STRING || dataType == DataTypes.BYTE_ARRAY) {
@@ -445,7 +438,7 @@ public abstract class ColumnPage {
       putLong(rowId, 0L);
     } else if (dataType == DataTypes.DOUBLE) {
       putDouble(rowId, 0.0);
-    } else if (dataType == DataTypes.DECIMAL) {
+    } else if (DataTypes.isDecimal(dataType)) {
       putDecimal(rowId, BigDecimal.ZERO);
     } else {
       throw new IllegalArgumentException("unsupported data type: " + dataType);
@@ -586,7 +579,7 @@ public abstract class ColumnPage {
       return compressor.compressFloat(getFloatPage());
     } else if (dataType == DataTypes.DOUBLE) {
       return compressor.compressDouble(getDoublePage());
-    } else if (dataType == DataTypes.DECIMAL) {
+    } else if (DataTypes.isDecimal(dataType)) {
       return compressor.compressByte(getDecimalPage());
     } else if (dataType == DataTypes.BYTE_ARRAY) {
       return compressor.compressByte(getLVFlattenedBytePage());

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/main/java/org/apache/carbondata/core/datastore/page/LazyColumnPage.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/LazyColumnPage.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/LazyColumnPage.java
index 11bdaca..f2cb860 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/page/LazyColumnPage.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/LazyColumnPage.java
@@ -108,7 +108,7 @@ public class LazyColumnPage extends ColumnPage {
       return decimalConverter.getDecimal(converter.decodeLong(columnPage.getShortInt(rowId)));
     } else if (dataType == DataTypes.INT) {
       return decimalConverter.getDecimal(converter.decodeLong(columnPage.getInt(rowId)));
-    } else if (dataType == DataTypes.LONG || dataType == DataTypes.DECIMAL) {
+    } else if (dataType == DataTypes.LONG || DataTypes.isDecimal(dataType)) {
       return columnPage.getDecimal(rowId);
     } else {
       throw new RuntimeException("internal error: " + this.toString());

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/main/java/org/apache/carbondata/core/datastore/page/UnsafeDecimalColumnPage.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/UnsafeDecimalColumnPage.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/UnsafeDecimalColumnPage.java
index b4f33b8..378b51f 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/page/UnsafeDecimalColumnPage.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/UnsafeDecimalColumnPage.java
@@ -53,20 +53,18 @@ public class UnsafeDecimalColumnPage extends DecimalColumnPage {
         dataType == DataTypes.LONG) {
       int size = pageSize << dataType.getSizeBits();
       memoryBlock = UnsafeMemoryManager.allocateMemoryWithRetry(taskId, size);
-      baseAddress = memoryBlock.getBaseObject();
-      baseOffset = memoryBlock.getBaseOffset();
     } else if (dataType == DataTypes.SHORT_INT) {
       int size = pageSize * 3;
       memoryBlock = UnsafeMemoryManager.allocateMemoryWithRetry(taskId, size);
-      baseAddress = memoryBlock.getBaseObject();
-      baseOffset = memoryBlock.getBaseOffset();
-    } else if (dataType == DataTypes.DECIMAL) {
+    } else if (DataTypes.isDecimal(dataType)) {
+      memoryBlock = UnsafeMemoryManager.allocateMemoryWithRetry(taskId, (long) (capacity));
+    } else if (dataType == DataTypes.BYTE_ARRAY) {
       memoryBlock = UnsafeMemoryManager.allocateMemoryWithRetry(taskId, (long) (capacity));
-      baseAddress = memoryBlock.getBaseObject();
-      baseOffset = memoryBlock.getBaseOffset();
     } else {
       throw new UnsupportedOperationException("invalid data type: " + dataType);
     }
+    baseAddress = memoryBlock.getBaseObject();
+    baseOffset = memoryBlock.getBaseOffset();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/main/java/org/apache/carbondata/core/datastore/page/UnsafeFixLengthColumnPage.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/UnsafeFixLengthColumnPage.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/UnsafeFixLengthColumnPage.java
index 5695b70..c88dc0b 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/page/UnsafeFixLengthColumnPage.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/UnsafeFixLengthColumnPage.java
@@ -72,7 +72,7 @@ public class UnsafeFixLengthColumnPage extends ColumnPage {
       memoryBlock = UnsafeMemoryManager.allocateMemoryWithRetry(taskId, size);
       baseAddress = memoryBlock.getBaseObject();
       baseOffset = memoryBlock.getBaseOffset();
-    } else if (dataType == DataTypes.DECIMAL || dataType == DataTypes.STRING) {
+    } else if (DataTypes.isDecimal(dataType) || dataType == DataTypes.STRING) {
       throw new UnsupportedOperationException("invalid data type: " + dataType);
     }
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/main/java/org/apache/carbondata/core/datastore/page/VarLengthColumnPageBase.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/VarLengthColumnPageBase.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/VarLengthColumnPageBase.java
index 60c7112..c6062c1 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/page/VarLengthColumnPageBase.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/VarLengthColumnPageBase.java
@@ -32,12 +32,9 @@ import org.apache.carbondata.core.metadata.datatype.DecimalConverterFactory;
 import org.apache.carbondata.core.util.ByteUtil;
 import org.apache.carbondata.core.util.ThreadLocalTaskInfo;
 
-import static org.apache.carbondata.core.metadata.datatype.DataTypes.BYTE;
-import static org.apache.carbondata.core.metadata.datatype.DataTypes.DECIMAL;
-
 public abstract class VarLengthColumnPageBase extends ColumnPage {
 
-  static final int byteBits = BYTE.getSizeBits();
+  static final int byteBits = DataTypes.BYTE.getSizeBits();
   static final int shortBits = DataTypes.SHORT.getSizeBits();
   static final int intBits = DataTypes.INT.getSizeBits();
   static final int longBits = DataTypes.LONG.getSizeBits();
@@ -65,6 +62,7 @@ public abstract class VarLengthColumnPageBase extends ColumnPage {
 
   // size of the allocated memory, in bytes
   int capacity;
+
   VarLengthColumnPageBase(TableSpec.ColumnSpec columnSpec, DataType dataType, int pageSize) {
     super(columnSpec, dataType, pageSize);
     rowOffset = new int[pageSize + 1];
@@ -116,7 +114,8 @@ public abstract class VarLengthColumnPageBase extends ColumnPage {
             columnSpec.getScale());
     int size = decimalConverter.getSize();
     if (size < 0) {
-      return getLVBytesColumnPage(columnSpec, lvEncodedBytes, DataTypes.DECIMAL);
+      return getLVBytesColumnPage(columnSpec, lvEncodedBytes,
+          DataTypes.createDecimalType(columnSpec.getPrecision(), columnSpec.getScale()));
     } else {
       // Here the size is always fixed.
       return getDecimalColumnPage(columnSpec, lvEncodedBytes, size);
@@ -144,9 +143,9 @@ public abstract class VarLengthColumnPageBase extends ColumnPage {
 
     VarLengthColumnPageBase page;
     if (unsafe) {
-      page = new UnsafeDecimalColumnPage(columnSpec, DECIMAL, rowId);
+      page = new UnsafeDecimalColumnPage(columnSpec, columnSpec.getSchemaDataType(), rowId);
     } else {
-      page = new SafeDecimalColumnPage(columnSpec, DECIMAL, rowId);
+      page = new SafeDecimalColumnPage(columnSpec, columnSpec.getSchemaDataType(), rowId);
     }
 
     // set total length and rowOffset in page
@@ -187,7 +186,7 @@ public abstract class VarLengthColumnPageBase extends ColumnPage {
     VarLengthColumnPageBase page;
     int inputDataLength = offset;
     if (unsafe) {
-      page = new UnsafeDecimalColumnPage(columnSpec, DECIMAL, numRows, inputDataLength);
+      page = new UnsafeDecimalColumnPage(columnSpec, dataType, numRows, inputDataLength);
     } else {
       page = new SafeDecimalColumnPage(columnSpec, dataType, numRows);
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/ColumnPageEncoder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/ColumnPageEncoder.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/ColumnPageEncoder.java
index 15e26e7..dfdca02 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/ColumnPageEncoder.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/ColumnPageEncoder.java
@@ -148,8 +148,8 @@ public abstract class ColumnPageEncoder {
 
   private static EncodedColumnPage encodeChildColumn(byte[][] data)
       throws IOException, MemoryException {
-    TableSpec.ColumnSpec spec =
-        new TableSpec.ColumnSpec("complex_inner_column", DataTypes.BYTE_ARRAY, ColumnType.COMPLEX);
+    TableSpec.ColumnSpec spec = TableSpec.ColumnSpec.newInstance("complex_inner_column",
+        DataTypes.BYTE_ARRAY, ColumnType.COMPLEX);
     ColumnPage page = ColumnPage.wrapByteArrayPage(spec, data);
     ColumnPageEncoder encoder = new DirectCompressCodec(DataTypes.BYTE_ARRAY).createEncoder(null);
     return encoder.encode(page);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/ColumnPageEncoderMeta.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/ColumnPageEncoderMeta.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/ColumnPageEncoderMeta.java
index a38da84..659feb0 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/ColumnPageEncoderMeta.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/ColumnPageEncoderMeta.java
@@ -28,6 +28,7 @@ import org.apache.carbondata.core.datastore.page.statistics.SimpleStatsResult;
 import org.apache.carbondata.core.metadata.ValueEncoderMeta;
 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.schema.table.Writable;
 import org.apache.carbondata.core.util.DataTypeUtil;
 
@@ -47,11 +48,6 @@ public class ColumnPageEncoderMeta extends ValueEncoderMeta implements Writable
   // compressor name for compressing and decompressing this column
   private String compressorName;
 
-  private int scale;
-  private int precision;
-
-
-
   public ColumnPageEncoderMeta() {
   }
 
@@ -74,8 +70,6 @@ public class ColumnPageEncoderMeta extends ValueEncoderMeta implements Writable
       setDecimal(stats.getDecimalCount());
       setMaxValue(stats.getMax());
       setMinValue(stats.getMin());
-      this.scale = stats.getScale();
-      this.precision = stats.getPrecision();
     }
   }
 
@@ -98,6 +92,12 @@ public class ColumnPageEncoderMeta extends ValueEncoderMeta implements Writable
     columnSpec = new TableSpec.ColumnSpec();
     columnSpec.readFields(in);
     storeDataType = DataTypes.valueOf(in.readByte());
+    if (DataTypes.isDecimal(storeDataType)) {
+      DecimalType decimalType = (DecimalType) storeDataType;
+      decimalType.setPrecision(columnSpec.getPrecision());
+      decimalType.setScale(columnSpec.getScale());
+    }
+
     setDecimal(in.readInt());
     setDataTypeSelected(in.readByte());
     readMinMax(in);
@@ -126,7 +126,7 @@ public class ColumnPageEncoderMeta extends ValueEncoderMeta implements Writable
       out.writeDouble((Double) getMaxValue());
       out.writeDouble((Double) getMinValue());
       out.writeDouble(0d); // unique value is obsoleted, maintain for compatibility
-    } else if (dataType == DataTypes.DECIMAL) {
+    } else if (DataTypes.isDecimal(dataType)) {
       byte[] maxAsBytes = getMaxAsBytes(columnSpec.getSchemaDataType());
       byte[] minAsBytes = getMinAsBytes(columnSpec.getSchemaDataType());
       byte[] unique = DataTypeUtil.bigDecimalToByte(BigDecimal.ZERO);
@@ -137,8 +137,14 @@ public class ColumnPageEncoderMeta extends ValueEncoderMeta implements Writable
       // unique value is obsoleted, maintain for compatibility
       out.writeShort((short) unique.length);
       out.write(unique);
-      out.writeInt(scale);
-      out.writeInt(precision);
+      if (DataTypes.isDecimal(dataType)) {
+        DecimalType decimalType = (DecimalType) dataType;
+        out.writeInt(decimalType.getScale());
+        out.writeInt(decimalType.getPrecision());
+      } else {
+        out.writeInt(-1);
+        out.writeInt(-1);
+      }
     } else if (dataType == DataTypes.BYTE_ARRAY) {
       // for complex type, it will come here, ignoring stats for complex type
       // TODO: support stats for complex type
@@ -169,7 +175,7 @@ public class ColumnPageEncoderMeta extends ValueEncoderMeta implements Writable
       this.setMaxValue(in.readDouble());
       this.setMinValue(in.readDouble());
       in.readDouble(); // for non exist value which is obsoleted, it is backward compatibility;
-    } else if (dataType == DataTypes.DECIMAL) {
+    } else if (DataTypes.isDecimal(dataType)) {
       byte[] max = new byte[in.readShort()];
       in.readFully(max);
       this.setMaxValue(DataTypeUtil.byteToBigDecimal(max));
@@ -179,8 +185,10 @@ public class ColumnPageEncoderMeta extends ValueEncoderMeta implements Writable
       // unique value is obsoleted, maintain for compatiability
       short uniqueLength = in.readShort();
       in.readFully(new byte[uniqueLength]);
-      this.scale = in.readInt();
-      this.precision = in.readInt();
+      // scale field is obsoleted. It is stored in the schema data type in columnSpec
+      in.readInt();
+      // precision field is obsoleted. It is stored in the schema data type in columnSpec
+      in.readInt();
     } else if (dataType == DataTypes.BYTE_ARRAY) {
       // for complex type, it will come here, ignoring stats for complex type
       // TODO: support stats for complex type
@@ -227,7 +235,7 @@ public class ColumnPageEncoderMeta extends ValueEncoderMeta implements Writable
       b.putDouble((double) value);
       b.flip();
       return b.array();
-    } else if (dataType == DataTypes.DECIMAL) {
+    } else if (DataTypes.isDecimal(dataType)) {
       return DataTypeUtil.bigDecimalToByte((BigDecimal) value);
     } else if (dataType == DataTypes.STRING || dataType == DataTypes.TIMESTAMP
         || dataType == DataTypes.DATE) {
@@ -238,11 +246,17 @@ public class ColumnPageEncoderMeta extends ValueEncoderMeta implements Writable
   }
 
   public int getScale() {
-    return scale;
+    if (DataTypes.isDecimal(columnSpec.getSchemaDataType())) {
+      return columnSpec.getScale();
+    }
+    throw new UnsupportedOperationException();
   }
 
   public int getPrecision() {
-    return precision;
+    if (DataTypes.isDecimal(columnSpec.getSchemaDataType())) {
+      return columnSpec.getPrecision();
+    }
+    throw new UnsupportedOperationException();
   }
 
   public TableSpec.ColumnSpec getColumnSpec() {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/DefaultEncodingFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/DefaultEncodingFactory.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/DefaultEncodingFactory.java
index 03af657..518573d 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/DefaultEncodingFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/DefaultEncodingFactory.java
@@ -121,7 +121,7 @@ public class DefaultEncodingFactory extends EncodingFactory {
         dataType == DataTypes.INT ||
         dataType == DataTypes.LONG) {
       return selectCodecByAlgorithmForIntegral(stats).createEncoder(null);
-    } else if (dataType == DataTypes.DECIMAL) {
+    } else if (DataTypes.isDecimal(dataType)) {
       return createEncoderForDecimalDataTypeMeasure(columnPage);
     } else if (dataType == DataTypes.FLOAT ||
         dataType == DataTypes.DOUBLE) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/EncodingFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/EncodingFactory.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/EncodingFactory.java
index c298525..0f45abb 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/EncodingFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/EncodingFactory.java
@@ -111,8 +111,8 @@ public abstract class EncodingFactory {
    */
   public ColumnPageDecoder createDecoderLegacy(ValueEncoderMeta metadata) {
     SimpleStatsResult stats = PrimitivePageStatsCollector.newInstance(metadata);
-    TableSpec.ColumnSpec spec = new TableSpec.ColumnSpec("legacy", stats.getDataType(),
-        ColumnType.MEASURE);
+    TableSpec.ColumnSpec spec =
+        TableSpec.ColumnSpec.newInstanceLegacy("legacy", stats.getDataType(), ColumnType.MEASURE);
     String compressor = "snappy";
     DataType dataType = DataType.getDataType(metadata.getType());
     if (dataType == DataTypes.BYTE ||
@@ -155,7 +155,7 @@ public abstract class EncodingFactory {
       } else {
         throw new RuntimeException("internal error");
       }
-    } else if (dataType == DataTypes.DECIMAL || dataType == DataTypes.BYTE_ARRAY) {
+    } else if (DataTypes.isDecimal(dataType) || dataType == DataTypes.BYTE_ARRAY) {
       // no dictionary dimension
       return new DirectCompressCodec(stats.getDataType())
           .createDecoder(new ColumnPageEncoderMeta(spec, stats.getDataType(), stats, compressor));

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveDeltaIntegralCodec.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveDeltaIntegralCodec.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveDeltaIntegralCodec.java
index 96f7b16..a543f7e 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveDeltaIntegralCodec.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveDeltaIntegralCodec.java
@@ -61,7 +61,7 @@ public class AdaptiveDeltaIntegralCodec extends AdaptiveCodec {
       this.max = (long) stats.getMax();
     } else if (srcDataType == DataTypes.DOUBLE) {
       this.max = (long) (double) stats.getMax();
-    } else if (srcDataType == DataTypes.DECIMAL) {
+    } else if (DataTypes.isDecimal(srcDataType)) {
       this.max = ((BigDecimal) stats.getMax()).unscaledValue().longValue();
     } else {
       // this codec is for integer type only
@@ -114,7 +114,7 @@ public class AdaptiveDeltaIntegralCodec extends AdaptiveCodec {
       @Override public ColumnPage decode(byte[] input, int offset, int length)
           throws MemoryException, IOException {
         ColumnPage page = null;
-        if (meta.getSchemaDataType() == DataTypes.DECIMAL) {
+        if (DataTypes.isDecimal(meta.getSchemaDataType())) {
           page = ColumnPage.decompressDecimalPage(meta, input, offset, length);
         } else {
           page = ColumnPage.decompress(meta, input, offset, length);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveIntegralCodec.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveIntegralCodec.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveIntegralCodec.java
index 907649d..b65296d 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveIntegralCodec.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveIntegralCodec.java
@@ -94,7 +94,7 @@ public class AdaptiveIntegralCodec extends AdaptiveCodec {
       public ColumnPage decode(byte[] input, int offset, int length)
           throws MemoryException, IOException {
         ColumnPage page = null;
-        if (meta.getSchemaDataType() == DataTypes.DECIMAL) {
+        if (DataTypes.isDecimal(meta.getSchemaDataType())) {
           page = ColumnPage.decompressDecimalPage(meta, input, offset, length);
         } else {
           page = ColumnPage.decompress(meta, input, offset, length);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/compress/DirectCompressCodec.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/compress/DirectCompressCodec.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/compress/DirectCompressCodec.java
index b3d282e..cfdf114 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/compress/DirectCompressCodec.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/compress/DirectCompressCodec.java
@@ -103,7 +103,7 @@ public class DirectCompressCodec implements ColumnPageCodec {
     @Override
     public ColumnPage decode(byte[] input, int offset, int length) throws MemoryException {
       ColumnPage decodedPage;
-      if (dataType == DataTypes.DECIMAL) {
+      if (DataTypes.isDecimal(dataType)) {
         decodedPage = ColumnPage.decompressDecimalPage(meta, input, offset, length);
       } else {
         decodedPage = ColumnPage.decompress(meta, input, offset, length);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/main/java/org/apache/carbondata/core/datastore/page/statistics/KeyPageStatsCollector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/statistics/KeyPageStatsCollector.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/statistics/KeyPageStatsCollector.java
index be47966..e6cf29e 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/page/statistics/KeyPageStatsCollector.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/statistics/KeyPageStatsCollector.java
@@ -106,13 +106,6 @@ public class KeyPageStatsCollector implements ColumnPageStatsCollector {
         return dataType;
       }
 
-      @Override public int getScale() {
-        return 0;
-      }
-
-      @Override public int getPrecision() {
-        return 0;
-      }
     };
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/main/java/org/apache/carbondata/core/datastore/page/statistics/LVStringStatsCollector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/statistics/LVStringStatsCollector.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/statistics/LVStringStatsCollector.java
index 20e10b8..61acec9 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/page/statistics/LVStringStatsCollector.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/statistics/LVStringStatsCollector.java
@@ -119,13 +119,6 @@ public class LVStringStatsCollector implements ColumnPageStatsCollector {
         return DataTypes.STRING;
       }
 
-      @Override public int getScale() {
-        return 0;
-      }
-
-      @Override public int getPrecision() {
-        return 0;
-      }
     };
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/main/java/org/apache/carbondata/core/datastore/page/statistics/PrimitivePageStatsCollector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/statistics/PrimitivePageStatsCollector.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/statistics/PrimitivePageStatsCollector.java
index ed92622..76cb002 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/page/statistics/PrimitivePageStatsCollector.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/statistics/PrimitivePageStatsCollector.java
@@ -37,7 +37,6 @@ public class PrimitivePageStatsCollector implements ColumnPageStatsCollector, Si
   private long minLong, maxLong;
   private double minDouble, maxDouble;
   private BigDecimal minDecimal, maxDecimal;
-  private int scale, precision;
 
   // scale of the double value, apply adaptive encoding if this is positive
   private int decimal;
@@ -46,15 +45,14 @@ public class PrimitivePageStatsCollector implements ColumnPageStatsCollector, Si
   private BigDecimal zeroDecimal;
 
   // this is for encode flow
-  public static PrimitivePageStatsCollector newInstance(DataType dataType,
-      int scale, int precision) {
-    return new PrimitivePageStatsCollector(dataType, scale, precision);
+  public static PrimitivePageStatsCollector newInstance(DataType dataType) {
+    return new PrimitivePageStatsCollector(dataType);
   }
 
   // this is for decode flow, create stats from encoder meta in carbondata file
   public static PrimitivePageStatsCollector newInstance(ColumnPageEncoderMeta meta) {
-    PrimitivePageStatsCollector instance = new PrimitivePageStatsCollector(meta.getSchemaDataType(),
-        meta.getScale(), meta.getPrecision());
+    PrimitivePageStatsCollector instance =
+        new PrimitivePageStatsCollector(meta.getSchemaDataType());
     // set min max from meta
     DataType dataType = meta.getSchemaDataType();
     if (dataType == DataTypes.BOOLEAN || dataType == DataTypes.BYTE) {
@@ -73,12 +71,10 @@ public class PrimitivePageStatsCollector implements ColumnPageStatsCollector, Si
       instance.minDouble = (double) meta.getMinValue();
       instance.maxDouble = (double) meta.getMaxValue();
       instance.decimal = meta.getDecimal();
-    } else if (dataType == DataTypes.DECIMAL) {
+    } else if (DataTypes.isDecimal(dataType)) {
       instance.minDecimal = (BigDecimal) meta.getMinValue();
       instance.maxDecimal = (BigDecimal) meta.getMaxValue();
       instance.decimal = meta.getDecimal();
-      instance.scale = meta.getScale();
-      instance.precision = meta.getPrecision();
     } else {
       throw new UnsupportedOperationException(
           "unsupported data type for stats collection: " + meta.getSchemaDataType());
@@ -88,7 +84,7 @@ public class PrimitivePageStatsCollector implements ColumnPageStatsCollector, Si
 
   public static PrimitivePageStatsCollector newInstance(ValueEncoderMeta meta) {
     PrimitivePageStatsCollector instance =
-        new PrimitivePageStatsCollector(DataType.getDataType(meta.getType()), -1, -1);
+        new PrimitivePageStatsCollector(DataType.getDataType(meta.getType()));
     // set min max from meta
     DataType dataType = DataType.getDataType(meta.getType());
     if (dataType == DataTypes.BOOLEAN || dataType == DataTypes.BYTE) {
@@ -107,12 +103,10 @@ public class PrimitivePageStatsCollector implements ColumnPageStatsCollector, Si
       instance.minDouble = (double) meta.getMinValue();
       instance.maxDouble = (double) meta.getMaxValue();
       instance.decimal = meta.getDecimal();
-    } else if (dataType == DataTypes.DECIMAL) {
+    } else if (DataTypes.isDecimal(dataType)) {
       instance.minDecimal = (BigDecimal) meta.getMinValue();
       instance.maxDecimal = (BigDecimal) meta.getMaxValue();
       instance.decimal = meta.getDecimal();
-      instance.scale = -1;
-      instance.precision = -1;
     } else {
       throw new UnsupportedOperationException(
           "unsupported data type for Stats collection: " + meta.getType());
@@ -120,7 +114,7 @@ public class PrimitivePageStatsCollector implements ColumnPageStatsCollector, Si
     return instance;
   }
 
-  private PrimitivePageStatsCollector(DataType dataType, int scale, int precision) {
+  private PrimitivePageStatsCollector(DataType dataType) {
     this.dataType = dataType;
     if (dataType == DataTypes.BOOLEAN) {
       minByte = TRUE_VALUE;
@@ -141,11 +135,9 @@ public class PrimitivePageStatsCollector implements ColumnPageStatsCollector, Si
       minDouble = Double.POSITIVE_INFINITY;
       maxDouble = Double.NEGATIVE_INFINITY;
       decimal = 0;
-    } else if (dataType == DataTypes.DECIMAL) {
+    } else if (DataTypes.isDecimal(dataType)) {
       this.zeroDecimal = BigDecimal.ZERO;
-      decimal = scale;
-      this.scale = scale;
-      this.precision = precision;
+      decimal = 0;
     } else {
       throw new UnsupportedOperationException(
           "unsupported data type for Stats collection: " + dataType);
@@ -165,7 +157,7 @@ public class PrimitivePageStatsCollector implements ColumnPageStatsCollector, Si
       update(value);
     } else if (dataType == DataTypes.DOUBLE) {
       update(0d);
-    } else if (dataType == DataTypes.DECIMAL) {
+    } else if (DataTypes.isDecimal(dataType)) {
       if (isFirst) {
         maxDecimal = zeroDecimal;
         minDecimal = zeroDecimal;
@@ -306,7 +298,7 @@ public class PrimitivePageStatsCollector implements ColumnPageStatsCollector, Si
       return minLong;
     } else if (dataType == DataTypes.DOUBLE) {
       return minDouble;
-    } else if (dataType == DataTypes.DECIMAL) {
+    } else if (DataTypes.isDecimal(dataType)) {
       return minDecimal;
     }
     return null;
@@ -324,7 +316,7 @@ public class PrimitivePageStatsCollector implements ColumnPageStatsCollector, Si
       return maxLong;
     } else if (dataType == DataTypes.DOUBLE) {
       return maxDouble;
-    } else if (dataType == DataTypes.DECIMAL) {
+    } else if (DataTypes.isDecimal(dataType)) {
       return maxDecimal;
     }
     return null;
@@ -340,14 +332,4 @@ public class PrimitivePageStatsCollector implements ColumnPageStatsCollector, Si
     return dataType;
   }
 
-  @Override
-  public int getScale() {
-    return scale;
-  }
-
-  @Override
-  public int getPrecision() {
-    return precision;
-  }
-
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/main/java/org/apache/carbondata/core/datastore/page/statistics/SimpleStatsResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/statistics/SimpleStatsResult.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/statistics/SimpleStatsResult.java
index 65cd40f..0e1f650 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/page/statistics/SimpleStatsResult.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/statistics/SimpleStatsResult.java
@@ -29,7 +29,4 @@ public interface SimpleStatsResult {
 
   DataType getDataType();
 
-  int getScale();
-
-  int getPrecision();
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/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 3e083cc..d5bd695 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
@@ -206,7 +206,7 @@ public class BlockletDataMap implements DataMap, Cacheable {
         } else if (dataType == DataTypes.LONG) {
           buffer.putLong(Long.MIN_VALUE);
           updatedValues[minValues.length + i] = buffer.array().clone();
-        } else if (dataType == DataTypes.DECIMAL) {
+        } else if (DataTypes.isDecimal(dataType)) {
           updatedValues[minValues.length + i] =
               DataTypeUtil.bigDecimalToByte(BigDecimal.valueOf(Long.MIN_VALUE));
         } else {
@@ -244,7 +244,7 @@ public class BlockletDataMap implements DataMap, Cacheable {
         } else if (dataType == DataTypes.LONG) {
           buffer.putLong(Long.MAX_VALUE);
           updatedValues[maxValues.length + i] = buffer.array().clone();
-        } else if (dataType == DataTypes.DECIMAL) {
+        } else if (DataTypes.isDecimal(dataType)) {
           updatedValues[maxValues.length + i] =
               DataTypeUtil.bigDecimalToByte(BigDecimal.valueOf(Long.MAX_VALUE));
         } else {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/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 7faa7e6..70a6e63 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
@@ -139,7 +139,7 @@ public class ThriftWrapperSchemaConverterImpl implements SchemaConverter {
       return org.apache.carbondata.format.DataType.LONG;
     } else if (dataType.getId() == DataTypes.DOUBLE.getId()) {
       return org.apache.carbondata.format.DataType.DOUBLE;
-    } else if (dataType.getId() == DataTypes.DECIMAL.getId()) {
+    } else if (DataTypes.isDecimal(dataType)) {
       return org.apache.carbondata.format.DataType.DECIMAL;
     } else if (dataType.getId() == DataTypes.DATE.getId()) {
       return org.apache.carbondata.format.DataType.DATE;
@@ -167,12 +167,21 @@ public class ThriftWrapperSchemaConverterImpl implements SchemaConverter {
     }
     org.apache.carbondata.format.ColumnSchema thriftColumnSchema =
         new org.apache.carbondata.format.ColumnSchema(
-            fromWrapperToExternalDataType(wrapperColumnSchema.getDataType()),
-            wrapperColumnSchema.getColumnName(), wrapperColumnSchema.getColumnUniqueId(),
-            wrapperColumnSchema.isColumnar(), encoders, wrapperColumnSchema.isDimensionColumn());
+            fromWrapperToExternalDataType(
+                wrapperColumnSchema.getDataType()),
+            wrapperColumnSchema.getColumnName(),
+            wrapperColumnSchema.getColumnUniqueId(),
+            wrapperColumnSchema.isColumnar(),
+            encoders,
+            wrapperColumnSchema.isDimensionColumn());
     thriftColumnSchema.setColumn_group_id(wrapperColumnSchema.getColumnGroupId());
-    thriftColumnSchema.setScale(wrapperColumnSchema.getScale());
-    thriftColumnSchema.setPrecision(wrapperColumnSchema.getPrecision());
+    if (DataTypes.isDecimal(wrapperColumnSchema.getDataType())) {
+      thriftColumnSchema.setScale(wrapperColumnSchema.getScale());
+      thriftColumnSchema.setPrecision(wrapperColumnSchema.getPrecision());
+    } else {
+      thriftColumnSchema.setScale(-1);
+      thriftColumnSchema.setPrecision(-1);
+    }
     thriftColumnSchema.setNum_child(wrapperColumnSchema.getNumberOfChild());
     thriftColumnSchema.setDefault_value(wrapperColumnSchema.getDefaultValue());
     thriftColumnSchema.setColumnProperties(wrapperColumnSchema.getColumnProperties());
@@ -358,7 +367,8 @@ public class ThriftWrapperSchemaConverterImpl implements SchemaConverter {
    * @param dataType
    * @return
    */
-  private DataType fromExternalToWrapperDataType(org.apache.carbondata.format.DataType dataType) {
+  private DataType fromExternalToWrapperDataType(org.apache.carbondata.format.DataType dataType,
+      int precision, int scale) {
     if (null == dataType) {
       return null;
     }
@@ -376,7 +386,7 @@ public class ThriftWrapperSchemaConverterImpl implements SchemaConverter {
       case DOUBLE:
         return DataTypes.DOUBLE;
       case DECIMAL:
-        return DataTypes.DECIMAL;
+        return DataTypes.createDecimalType(precision, scale);
       case TIMESTAMP:
         return DataTypes.TIMESTAMP;
       case DATE:
@@ -399,7 +409,11 @@ public class ThriftWrapperSchemaConverterImpl implements SchemaConverter {
     wrapperColumnSchema.setColumnUniqueId(externalColumnSchema.getColumn_id());
     wrapperColumnSchema.setColumnName(externalColumnSchema.getColumn_name());
     wrapperColumnSchema.setColumnar(externalColumnSchema.isColumnar());
-    wrapperColumnSchema.setDataType(fromExternalToWrapperDataType(externalColumnSchema.data_type));
+    wrapperColumnSchema.setDataType(
+        fromExternalToWrapperDataType(
+            externalColumnSchema.data_type,
+            externalColumnSchema.precision,
+            externalColumnSchema.scale));
     wrapperColumnSchema.setDimensionColumn(externalColumnSchema.isDimension());
     List<Encoding> encoders = new ArrayList<Encoding>();
     for (org.apache.carbondata.format.Encoding encoder : externalColumnSchema.getEncoders()) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/main/java/org/apache/carbondata/core/metadata/datatype/DataType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/DataType.java b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/DataType.java
index b3dd1bc..8e08436 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/DataType.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/DataType.java
@@ -85,7 +85,7 @@ public class DataType implements Serializable {
       return BIG_INT_MEASURE_CHAR;
     } else if (dataType == DataTypes.DOUBLE) {
       return DOUBLE_MEASURE_CHAR;
-    } else if (dataType == DataTypes.DECIMAL) {
+    } else if (DataTypes.isDecimal(dataType)) {
       return BIG_DECIMAL_MEASURE_CHAR;
     } else if (dataType == DataTypes.STRING) {
       return STRING_CHAR;
@@ -107,11 +107,12 @@ public class DataType implements Serializable {
       case DOUBLE_MEASURE_CHAR:
         return DataTypes.DOUBLE;
       case BIG_DECIMAL_MEASURE_CHAR:
-        return DataTypes.DECIMAL;
+        return DataTypes.createDefaultDecimalType();
       case 'l':
         return DataTypes.LEGACY_LONG;
       default:
         throw new RuntimeException("Unexpected type: " + type);
     }
   }
+
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/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 178f06a..43dad72 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
@@ -44,7 +44,6 @@ 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 DECIMAL = DecimalType.DECIMAL;
   public static final DataType ARRAY = ArrayType.ARRAY;
   public static final DataType STRUCT = StructType.STRUCT;
   public static final DataType MAP = MapType.MAP;
@@ -99,8 +98,8 @@ public class DataTypes {
       return DOUBLE;
     } else if (id == NULL.getId()) {
       return NULL;
-    } else if (id == DECIMAL.getId()) {
-      return DECIMAL;
+    } else if (id == DECIMAL_TYPE_ID) {
+      return createDefaultDecimalType();
     } else if (id == ARRAY.getId()) {
       return ARRAY;
     } else if (id == STRUCT.getId()) {
@@ -114,4 +113,22 @@ public class DataTypes {
     }
   }
 
+  /**
+   * create a decimal type object with specified precision and scale
+   */
+  public static DecimalType createDecimalType(int precision, int scale) {
+    return new DecimalType(precision, scale);
+  }
+
+  /**
+   * create a decimal type object with default precision = 10 and scale = 2
+   */
+  public static DecimalType createDefaultDecimalType() {
+    return new DecimalType(10, 2);
+  }
+
+  public static boolean isDecimal(DataType dataType) {
+    return dataType.getId() == DECIMAL_TYPE_ID;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/main/java/org/apache/carbondata/core/metadata/datatype/DecimalConverterFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/DecimalConverterFactory.java b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/DecimalConverterFactory.java
index 9dbc9b4..e4059c8 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/DecimalConverterFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/DecimalConverterFactory.java
@@ -20,7 +20,6 @@ import java.math.BigDecimal;
 import java.math.BigInteger;
 import java.util.Arrays;
 
-import org.apache.carbondata.core.scan.result.vector.CarbonColumnVector;
 import org.apache.carbondata.core.util.DataTypeUtil;
 
 /**
@@ -73,19 +72,17 @@ public final class DecimalConverterFactory {
 
     BigDecimal getDecimal(Object valueToBeConverted);
 
-    void writeToColumnVector(byte[] bytes, CarbonColumnVector vector, int rowId);
-
     int getSize();
 
     DecimalConverterType getDecimalConverterType();
 
   }
 
-  public class DecimalIntConverter implements DecimalConverter {
+  public static class DecimalIntConverter implements DecimalConverter {
 
     private int scale;
 
-    public DecimalIntConverter(int precision, int scale) {
+    DecimalIntConverter(int scale) {
       this.scale = scale;
     }
 
@@ -98,11 +95,6 @@ public final class DecimalConverterFactory {
       return BigDecimal.valueOf((Long) valueToBeConverted, scale);
     }
 
-    @Override public void writeToColumnVector(byte[] bytes, CarbonColumnVector vector, int rowId) {
-      long unscaled = getUnscaledLong(bytes);
-      vector.putInt(rowId, (int) unscaled);
-    }
-
     @Override public int getSize() {
       return 4;
     }
@@ -126,28 +118,22 @@ public final class DecimalConverterFactory {
     return unscaled;
   }
 
-  public class DecimalLongConverter implements DecimalConverter {
+  public static class DecimalLongConverter implements DecimalConverter {
 
     private int scale;
 
-    public DecimalLongConverter(int precision, int scale) {
+    DecimalLongConverter(int scale) {
       this.scale = scale;
     }
 
     @Override public Object convert(BigDecimal decimal) {
-      long longValue = decimal.unscaledValue().longValue();
-      return longValue;
+      return decimal.unscaledValue().longValue();
     }
 
     @Override public BigDecimal getDecimal(Object valueToBeConverted) {
       return BigDecimal.valueOf((Long) valueToBeConverted, scale);
     }
 
-    @Override public void writeToColumnVector(byte[] bytes, CarbonColumnVector vector, int rowId) {
-      long unscaled = getUnscaledLong(bytes);
-      vector.putLong(rowId, unscaled);
-    }
-
     @Override public int getSize() {
       return 8;
     }
@@ -159,14 +145,13 @@ public final class DecimalConverterFactory {
 
   public class DecimalUnscaledConverter implements DecimalConverter {
 
-
     private int scale;
 
     private int numBytes;
 
     private byte[] decimalBuffer = new byte[minBytesForPrecision[38]];
 
-    public DecimalUnscaledConverter(int precision, int scale) {
+    DecimalUnscaledConverter(int precision, int scale) {
       this.scale = scale;
       this.numBytes = minBytesForPrecision[precision];
     }
@@ -202,10 +187,6 @@ public final class DecimalConverterFactory {
       return new BigDecimal(bigInteger, scale);
     }
 
-    @Override public void writeToColumnVector(byte[] bytes, CarbonColumnVector vector, int rowId) {
-      vector.putBytes(rowId, bytes);
-    }
-
     @Override public int getSize() {
       return numBytes;
     }
@@ -227,10 +208,6 @@ public final class DecimalConverterFactory {
       return DataTypeUtil.byteToBigDecimal((byte[]) valueToBeConverted);
     }
 
-    @Override public void writeToColumnVector(byte[] bytes, CarbonColumnVector vector, int rowId) {
-      throw new UnsupportedOperationException("Unsupported in vector reading for legacy format");
-    }
-
     @Override public int getSize() {
       return -1;
     }
@@ -244,9 +221,9 @@ public final class DecimalConverterFactory {
     if (precision < 0) {
       return new LVBytesDecimalConverter();
     } else if (precision <= 9) {
-      return new DecimalIntConverter(precision, scale);
+      return new DecimalIntConverter(scale);
     } else if (precision <= 18) {
-      return new DecimalLongConverter(precision, scale);
+      return new DecimalLongConverter(scale);
     } else {
       return new DecimalUnscaledConverter(precision, scale);
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/main/java/org/apache/carbondata/core/metadata/datatype/DecimalType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/DecimalType.java b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/DecimalType.java
index 0c78e50..b2acd21 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/datatype/DecimalType.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/datatype/DecimalType.java
@@ -19,15 +19,29 @@ package org.apache.carbondata.core.metadata.datatype;
 
 public class DecimalType extends DataType {
 
-  public static final DataType DECIMAL =
-      new DecimalType(DataTypes.DECIMAL_TYPE_ID, 8, "DECIMAL", -1);
+  private int precision;
+  private int scale;
 
-  private DecimalType(int id, int precedenceOrder, String name, int sizeInBytes) {
-    super(id, precedenceOrder, name, sizeInBytes);
+  // create a decimal type object with specified precision and scale
+  DecimalType(int precision, int scale) {
+    super(DataTypes.DECIMAL_TYPE_ID, 8, "DECIMAL", -1);
+    this.precision = precision;
+    this.scale = scale;
   }
 
-  // this function is needed to ensure singleton pattern while supporting java serialization
-  private Object readResolve() {
-    return DataTypes.DECIMAL;
+  public int getPrecision() {
+    return precision;
+  }
+
+  public void setPrecision(int precision) {
+    this.precision = precision;
+  }
+
+  public int getScale() {
+    return scale;
+  }
+
+  public void setScale(int scale) {
+    this.scale = scale;
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/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 bd246a4..cad3dd6 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
@@ -27,6 +27,7 @@ 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.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.Writable;
 import org.apache.carbondata.core.metadata.schema.table.WritableUtil;
@@ -212,6 +213,16 @@ public class ColumnSchema implements Serializable, Writable {
   }
 
   /**
+   * Set the scale if it is decimal type
+   */
+  public void setScale(int scale) {
+    this.scale = scale;
+    if (DataTypes.isDecimal(dataType)) {
+      ((DecimalType) dataType).setScale(scale);
+    }
+  }
+
+  /**
    * @return the scale
    */
   public int getScale() {
@@ -219,10 +230,13 @@ public class ColumnSchema implements Serializable, Writable {
   }
 
   /**
-   * @param scale the scale to set
+   * Set the precision if it is decimal type
    */
-  public void setScale(int scale) {
-    this.scale = scale;
+  public void setPrecision(int precision) {
+    this.precision = precision;
+    if (DataTypes.isDecimal(dataType)) {
+      ((DecimalType) dataType).setPrecision(precision);
+    }
   }
 
   /**
@@ -233,13 +247,6 @@ public class ColumnSchema implements Serializable, Writable {
   }
 
   /**
-   * @param precision the precision to set
-   */
-  public void setPrecision(int precision) {
-    this.precision = precision;
-  }
-
-  /**
    * @return the getNumberOfChild
    */
   public int getNumberOfChild() {
@@ -423,8 +430,13 @@ public class ColumnSchema implements Serializable, Writable {
       }
     }
     out.writeBoolean(isDimensionColumn);
-    out.writeInt(scale);
-    out.writeInt(precision);
+    if (DataTypes.isDecimal(dataType)) {
+      out.writeInt(((DecimalType) dataType).getScale());
+      out.writeInt(((DecimalType) dataType).getPrecision());
+    } else {
+      out.writeInt(-1);
+      out.writeInt(-1);
+    }
     out.writeInt(schemaOrdinal);
     out.writeInt(numberOfChild);
     WritableUtil.writeByteArray(out, defaultValue);
@@ -457,6 +469,11 @@ public class ColumnSchema implements Serializable, Writable {
     this.isDimensionColumn = in.readBoolean();
     this.scale = in.readInt();
     this.precision = in.readInt();
+    if (DataTypes.isDecimal(dataType)) {
+      DecimalType decimalType = (DecimalType) dataType;
+      decimalType.setPrecision(precision);
+      decimalType.setScale(scale);
+    }
     this.schemaOrdinal = in.readInt();
     this.numberOfChild = in.readInt();
     this.defaultValue = WritableUtil.readByteArray(in);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/AbstractScannedResultCollector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/AbstractScannedResultCollector.java b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/AbstractScannedResultCollector.java
index 7e24413..dc78ac6 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/AbstractScannedResultCollector.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/AbstractScannedResultCollector.java
@@ -78,7 +78,7 @@ public abstract class AbstractScannedResultCollector implements ScannedResultCol
       } else {
         // if not then get the default value and use that value in aggregation
         Object defaultValue = measureInfo.getDefaultValues()[i];
-        if (null != defaultValue && measureInfo.getMeasureDataTypes()[i] == DataTypes.DECIMAL) {
+        if (null != defaultValue && DataTypes.isDecimal(measureInfo.getMeasureDataTypes()[i])) {
           // convert data type as per the computing engine
           defaultValue = DataTypeUtil.getDataTypeConverter().convertToDecimal(defaultValue);
         }
@@ -99,7 +99,7 @@ public abstract class AbstractScannedResultCollector implements ScannedResultCol
         return (int) dataChunk.getLong(index);
       } else if (dataType == DataTypes.LONG) {
         return dataChunk.getLong(index);
-      } else if (dataType == DataTypes.DECIMAL) {
+      } else if (DataTypes.isDecimal(dataType)) {
         BigDecimal bigDecimalMsrValue = dataChunk.getDecimal(index);
         if (null != bigDecimalMsrValue && carbonMeasure.getScale() > bigDecimalMsrValue.scale()) {
           bigDecimalMsrValue =

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedVectorResultCollector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedVectorResultCollector.java b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedVectorResultCollector.java
index cbde2e1..d9b7b23 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedVectorResultCollector.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedVectorResultCollector.java
@@ -236,7 +236,7 @@ public class RestructureBasedVectorResultCollector extends DictionaryBasedVector
           } else if (dataType == DataTypes.LONG) {
             vector.putLongs(columnVectorInfo.vectorOffset, columnVectorInfo.size,
                 (long) defaultValue);
-          } else if (dataType == DataTypes.DECIMAL) {
+          } else if (DataTypes.isDecimal(dataType)) {
             vector.putDecimals(columnVectorInfo.vectorOffset, columnVectorInfo.size,
                 (Decimal) defaultValue, measure.getPrecision());
           } else {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/main/java/org/apache/carbondata/core/scan/executor/util/RestructureUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/executor/util/RestructureUtil.java b/core/src/main/java/org/apache/carbondata/core/scan/executor/util/RestructureUtil.java
index ddc75ff..b3a77b8 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/executor/util/RestructureUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/executor/util/RestructureUtil.java
@@ -292,7 +292,7 @@ public class RestructureUtil {
       if (dataType == DataTypes.SHORT || dataType == DataTypes.INT || dataType == DataTypes.LONG) {
         value = new String(defaultValue, Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
         measureDefaultValue = Long.parseLong(value);
-      } else if (dataType == DataTypes.DECIMAL) {
+      } else if (DataTypes.isDecimal(dataType)) {
         BigDecimal decimal = DataTypeUtil.byteToBigDecimal(defaultValue);
         if (columnSchema.getScale() > decimal.scale()) {
           decimal = decimal.setScale(columnSchema.getScale(), RoundingMode.HALF_UP);
@@ -331,7 +331,7 @@ public class RestructureUtil {
       } else if (dataType == DataTypes.LONG) {
         value = new String(defaultValue, Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
         measureDefaultValue = Long.parseLong(value);
-      } else if (dataType == DataTypes.DECIMAL) {
+      } else if (DataTypes.isDecimal(dataType)) {
         BigDecimal decimal = DataTypeUtil.byteToBigDecimal(defaultValue);
         if (columnSchema.getScale() > decimal.scale()) {
           decimal = decimal.setScale(columnSchema.getScale(), RoundingMode.HALF_UP);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/main/java/org/apache/carbondata/core/scan/expression/ExpressionResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/expression/ExpressionResult.java b/core/src/main/java/org/apache/carbondata/core/scan/expression/ExpressionResult.java
index e3892be..262904f 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/expression/ExpressionResult.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/expression/ExpressionResult.java
@@ -308,7 +308,7 @@ public class ExpressionResult implements Comparable<ExpressionResult> {
         return new BigDecimal((int) value);
       } else if (dataType == DataTypes.LONG) {
         return new BigDecimal((long) value);
-      } else if (dataType == DataTypes.DOUBLE || dataType == DataTypes.DECIMAL) {
+      } else if (dataType == DataTypes.DOUBLE || DataTypes.isDecimal(dataType)) {
         return new BigDecimal(value.toString());
       } else if (dataType == DataTypes.DATE) {
         if (value instanceof java.sql.Date) {
@@ -494,7 +494,7 @@ public class ExpressionResult implements Comparable<ExpressionResult> {
         result = this.getLong().equals(objToCompare.getLong());
       } else if (dataType == DataTypes.DOUBLE) {
         result = this.getDouble().equals(objToCompare.getDouble());
-      } else if (dataType == DataTypes.DECIMAL) {
+      } else if (DataTypes.isDecimal(dataType)) {
         result = this.getDecimal().equals(objToCompare.getDecimal());
       }
     } catch (FilterIllegalMemberException ex) {
@@ -518,7 +518,7 @@ public class ExpressionResult implements Comparable<ExpressionResult> {
         Double d1 = this.getDouble();
         Double d2 = o.getDouble();
         return d1.compareTo(d2);
-      } else if (type == DataTypes.DECIMAL) {
+      } else if (DataTypes.isDecimal(type)) {
         java.math.BigDecimal val1 = this.getDecimal();
         java.math.BigDecimal val2 = o.getDecimal();
         return val1.compareTo(val2);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/EqualToExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/EqualToExpression.java b/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/EqualToExpression.java
index f143189..faf5bb1 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/EqualToExpression.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/EqualToExpression.java
@@ -81,7 +81,7 @@ public class EqualToExpression extends BinaryConditionalExpression {
       result = val1.getTime().equals(val2.getTime());
     } else if (dataType == DataTypes.LONG) {
       result = val1.getLong().equals(val2.getLong());
-    } else if (dataType == DataTypes.DECIMAL) {
+    } else if (DataTypes.isDecimal(dataType)) {
       result = val1.getDecimal().compareTo(val2.getDecimal()) == 0;
     } else {
       throw new FilterUnsupportedException(

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/GreaterThanEqualToExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/GreaterThanEqualToExpression.java b/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/GreaterThanEqualToExpression.java
index 1472959..24575d2 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/GreaterThanEqualToExpression.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/GreaterThanEqualToExpression.java
@@ -64,7 +64,7 @@ public class GreaterThanEqualToExpression extends BinaryConditionalExpression {
       result = elRes.getTime() >= (erRes.getTime());
     } else if (dataType == DataTypes.LONG) {
       result = elRes.getLong() >= (erRes.getLong());
-    } else if (dataType == DataTypes.DECIMAL) {
+    } else if (DataTypes.isDecimal(dataType)) {
       result = elRes.getDecimal().compareTo(erRes.getDecimal()) >= 0;
     } else  {
       throw new FilterUnsupportedException(

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/GreaterThanExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/GreaterThanExpression.java b/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/GreaterThanExpression.java
index b8a8a7c..ddc3d30 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/GreaterThanExpression.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/GreaterThanExpression.java
@@ -66,7 +66,7 @@ public class GreaterThanExpression extends BinaryConditionalExpression {
       result = exprLeftRes.getTime() > (exprRightRes.getTime());
     } else if (dataType == DataTypes.LONG) {
       result = exprLeftRes.getLong() > (exprRightRes.getLong());
-    } else if (dataType == DataTypes.DECIMAL) {
+    } else if (DataTypes.isDecimal(dataType)) {
       result = exprLeftRes.getDecimal().compareTo(exprRightRes.getDecimal()) > 0;
     } else {
       throw new FilterUnsupportedException(

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/InExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/InExpression.java b/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/InExpression.java
index 7243741..a560cc3 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/InExpression.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/InExpression.java
@@ -68,7 +68,7 @@ public class InExpression extends BinaryConditionalExpression {
           val = new ExpressionResult(val.getDataType(), expressionResVal.getLong());
         } else if (dataType == DataTypes.DATE || dataType == DataTypes.TIMESTAMP) {
           val = new ExpressionResult(val.getDataType(), expressionResVal.getTime());
-        } else if (dataType == DataTypes.DECIMAL) {
+        } else if (DataTypes.isDecimal(dataType)) {
           val = new ExpressionResult(val.getDataType(), expressionResVal.getDecimal());
         } else {
           throw new FilterUnsupportedException(

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/LessThanEqualToExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/LessThanEqualToExpression.java b/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/LessThanEqualToExpression.java
index 6a9fc3c..df7d791 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/LessThanEqualToExpression.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/LessThanEqualToExpression.java
@@ -64,7 +64,7 @@ public class LessThanEqualToExpression extends BinaryConditionalExpression {
       result = elRes.getTime() <= (erRes.getTime());
     } else if (dataType == DataTypes.LONG) {
       result = elRes.getLong() <= (erRes.getLong());
-    } else if (dataType == DataTypes.DECIMAL) {
+    } else if (DataTypes.isDecimal(dataType)) {
       result = elRes.getDecimal().compareTo(erRes.getDecimal()) <= 0;
     } else {
       throw new FilterUnsupportedException("DataType: " + exprResValue1.getDataType()

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/LessThanExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/LessThanExpression.java b/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/LessThanExpression.java
index 4283d83..f4b7f7c 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/LessThanExpression.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/LessThanExpression.java
@@ -68,7 +68,7 @@ public class LessThanExpression extends BinaryConditionalExpression {
       result = elRes.getTime() < (erRes.getTime());
     } else if (dataType == DataTypes.LONG) {
       result = elRes.getLong() < (erRes.getLong());
-    } else if (dataType == DataTypes.DECIMAL) {
+    } else if (DataTypes.isDecimal(dataType)) {
       result = elRes.getDecimal().compareTo(erRes.getDecimal()) < 0;
     } else {
       throw new FilterUnsupportedException("DataType: " + val1.getDataType() +

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/NotEqualsExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/NotEqualsExpression.java b/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/NotEqualsExpression.java
index 5055caf..5046722 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/NotEqualsExpression.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/NotEqualsExpression.java
@@ -77,7 +77,7 @@ public class NotEqualsExpression extends BinaryConditionalExpression {
       result = val1.getTime().longValue() != val2.getTime().longValue();
     } else if (dataType == DataTypes.LONG) {
       result = elRes.getLong().longValue() != (erRes.getLong()).longValue();
-    } else if (dataType == DataTypes.DECIMAL) {
+    } else if (DataTypes.isDecimal(dataType)) {
       result = elRes.getDecimal().compareTo(erRes.getDecimal()) != 0;
     } else {
       throw new FilterUnsupportedException(

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/NotInExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/NotInExpression.java b/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/NotInExpression.java
index 89a0374..c4a2fc8 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/NotInExpression.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/NotInExpression.java
@@ -89,7 +89,7 @@ public class NotInExpression extends BinaryConditionalExpression {
           val = new ExpressionResult(val.getDataType(), exprResVal.getTime());
         } else if (dataType == DataTypes.LONG) {
           val = new ExpressionResult(val.getDataType(), exprResVal.getLong());
-        } else if (dataType == DataTypes.DECIMAL) {
+        } else if (DataTypes.isDecimal(dataType)) {
           val = new ExpressionResult(val.getDataType(), exprResVal.getDecimal());
         } else {
           throw new FilterUnsupportedException(

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f209e8ee/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterUtil.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterUtil.java
index ecac617..6943b8b 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterUtil.java
@@ -1318,7 +1318,7 @@ public final class FilterUtil {
         dateToStr = parser.parse(memberVal);
         dictionaryDate = parser.parse(dictionaryVal);
         return dictionaryDate.compareTo(dateToStr);
-      } else if (dataType == DataTypes.DECIMAL) {
+      } else if (DataTypes.isDecimal(dataType)) {
         java.math.BigDecimal javaDecValForDictVal = new java.math.BigDecimal(dictionaryVal);
         java.math.BigDecimal javaDecValForMemberVal = new java.math.BigDecimal(memberVal);
         return javaDecValForDictVal.compareTo(javaDecValForMemberVal);
@@ -1430,7 +1430,7 @@ public final class FilterUtil {
         Double d1 = Double.parseDouble(filterMember1);
         Double d2 = Double.parseDouble(filterMember2);
         return d1.compareTo(d2);
-      } else if (dataType == DataTypes.DECIMAL) {
+      } else if (DataTypes.isDecimal(dataType)) {
         if (CarbonCommonConstants.MEMBER_DEFAULT_VAL.equals(filterMember1)) {
           return 1;
         }


[37/49] 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/fgdatamap
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
 


[28/49] carbondata git commit: [CARBONDATA-1689] Fix parent pom issues and correct CI link of README

Posted by ra...@apache.org.
[CARBONDATA-1689] Fix parent pom issues and correct CI link of README

This closes #1478


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

Branch: refs/heads/fgdatamap
Commit: 2de24063227329a2ba058adbf79ec78abae919e7
Parents: 9e9d689
Author: chenliang613 <ch...@huawei.com>
Authored: Thu Nov 9 20:34:35 2017 +0800
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Nov 9 23:49:14 2017 +0530

----------------------------------------------------------------------
 README.md | 36 +++++++++++++++++-------------------
 pom.xml   |  6 ++++--
 2 files changed, 21 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/2de24063/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index 297cece..f2b4e15 100644
--- a/README.md
+++ b/README.md
@@ -1,21 +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">
 
@@ -28,7 +26,7 @@ You can find the latest CarbonData document and learn more at:
 
 ## Status
 Spark2.1:
-[![Build Status](https://builds.apache.org/buildStatus/icon?job=carbondata-master-spark-2.1)](https://builds.apache.org/view/CarbonData/job/carbondata-master-spark-2.1/)
+[![Build Status](https://builds.apache.org/buildStatus/icon?job=carbondata-master-spark-2.1)](https://builds.apache.org/view/A-D/view/CarbonData/job/carbondata-master-spark-2.1/badge/icon)
 
 ## Features
 CarbonData file format is a columnar store in HDFS, it has many features that a modern columnar format has, such as splittable, compression schema ,complex data type etc, and CarbonData has following unique features:

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2de24063/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 8b1212d..342cb2b 100644
--- a/pom.xml
+++ b/pom.xml
@@ -108,6 +108,8 @@
     <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
     <snappy.version>1.1.2.6</snappy.version>
     <hadoop.version>2.7.2</hadoop.version>
+    <scala.binary.version>2.11</scala.binary.version>
+    <scala.version>2.11.8</scala.version>
     <hadoop.deps.scope>compile</hadoop.deps.scope>
     <spark.deps.scope>compile</spark.deps.scope>
     <scala.deps.scope>compile</scala.deps.scope>
@@ -427,8 +429,6 @@
       <id>build-all</id>
       <properties>
         <spark.version>2.1.0</spark.version>
-        <scala.binary.version>2.11</scala.binary.version>
-        <scala.version>2.11.8</scala.version>
         <maven.test.skip>true</maven.test.skip>
       </properties>
       <modules>
@@ -503,6 +503,8 @@
                 <sourceDirectory>${basedir}/integration/hive/src/main/java</sourceDirectory>
                 <sourceDirectory>${basedir}/integration/presto/src/main/scala</sourceDirectory>
                 <sourceDirectory>${basedir}/integration/presto/src/main/java</sourceDirectory>
+                <sourceDirectory>${basedir}/streaming/src/main/java</sourceDirectory>
+                <sourceDirectory>${basedir}/streaming/src/main/java</sourceDirectory>
               </sourceDirectories>
             </configuration>
           </plugin>