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

[01/24] carbondata git commit: [CARBONDATA-1444] Update documentation about Boolean data type [Forced Update!]

Repository: carbondata
Updated Branches:
  refs/heads/pre-aggregate 33d11997a -> 7e1d40c0b (forced update)


[CARBONDATA-1444] Update documentation about Boolean data type

Add BOOLEAN data type into supported-data-types-in-carbondata documentation

This closes #1430


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

Branch: refs/heads/pre-aggregate
Commit: 40eaa443e73a7ce40b249251bb6f8431914cdf35
Parents: 5a67c98
Author: xubo245 <60...@qq.com>
Authored: Tue Oct 24 11:52:16 2017 +0800
Committer: chenliang613 <ch...@apache.org>
Committed: Fri Oct 27 16:56:15 2017 +0530

----------------------------------------------------------------------
 docs/supported-data-types-in-carbondata.md | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/40eaa443/docs/supported-data-types-in-carbondata.md
----------------------------------------------------------------------
diff --git a/docs/supported-data-types-in-carbondata.md b/docs/supported-data-types-in-carbondata.md
index 561248c..4ef9987 100644
--- a/docs/supported-data-types-in-carbondata.md
+++ b/docs/supported-data-types-in-carbondata.md
@@ -40,3 +40,6 @@
   * Complex Types
     * arrays: ARRAY``<data_type>``
     * structs: STRUCT``<col_name : data_type COMMENT col_comment, ...>``
+
+  * Other Types
+    * BOOLEAN
\ No newline at end of file


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

Posted by ja...@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/2235ed6e
Tree: http://git-wip-us.apache.org/repos/asf/carbondata/tree/2235ed6e
Diff: http://git-wip-us.apache.org/repos/asf/carbondata/diff/2235ed6e

Branch: refs/heads/pre-aggregate
Commit: 2235ed6ed4007818efc316d46ba355b0b788dc9c
Parents: 74bd52b
Author: kumarvishal <ku...@gmail.com>
Authored: Sat Oct 21 16:31:15 2017 +0530
Committer: Jacky Li <ja...@qq.com>
Committed: Wed Nov 8 11:02:15 2017 +0800

----------------------------------------------------------------------
 .../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   |  80 +++++++++-
 .../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, 577 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/2235ed6e/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/2235ed6e/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..86ce7e9 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/2235ed6e/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..389e32e 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
@@ -212,7 +212,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 =
@@ -226,8 +226,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++;
             }
@@ -236,8 +236,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 {
@@ -245,15 +245,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());
@@ -286,14 +286,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 =
@@ -301,8 +301,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/2235ed6e/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/2235ed6e/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/2235ed6e/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..5f6178f 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,10 +81,15 @@ 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;
+
   // table comment
   private String tableComment;
 
   public TableInfo() {
+    dataMapSchemaList = new ArrayList<>();
   }
 
   /**
@@ -97,6 +104,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);
   }
 
   /**
@@ -171,6 +195,14 @@ public class TableInfo implements Serializable, Writable {
     this.tableComment = tableComment;
   }
 
+  public List<DataMapSchema> getDataMapSchemaList() {
+    return dataMapSchemaList;
+  }
+
+  public void setDataMapSchemaList(List<DataMapSchema> dataMapSchemaList) {
+    this.dataMapSchemaList = dataMapSchemaList;
+  }
+
   /**
    * to generate the hash code
    */
@@ -239,10 +271,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();
@@ -250,6 +299,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() {
@@ -273,4 +342,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/2235ed6e/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/2235ed6e/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/2235ed6e/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..bb83c85 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/2235ed6e/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
 }


[18/24] carbondata git commit: [CARBONDATA-1669] Clean up code in CarbonDataRDDFactory

Posted by ja...@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() {


[19/24] carbondata git commit: [CARBONDATA-1669] Clean up code in CarbonDataRDDFactory

Posted by ja...@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.")
+      }
+    }
+  }
 }


[06/24] carbondata git commit: [CARBONDATA-1659] Remove spark 1.x info

Posted by ja...@apache.org.
[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/pre-aggregate
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>


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

Posted by ja...@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/pre-aggregate
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)


[04/24] carbondata git commit: [CARBONDATA-1628][Streaming] Re-factory LoadTableCommand to reuse code for streaming ingest in the future

Posted by ja...@apache.org.
[CARBONDATA-1628][Streaming] Re-factory LoadTableCommand to reuse code for streaming ingest in the future

Re-factory LoadTableCommand to reuse code for streaming ingest in the future

This closes #1439


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

Branch: refs/heads/pre-aggregate
Commit: 5936e7fb25c9d5125a5863aa7648cffe5a1d8f7c
Parents: 4d70a21
Author: QiangCai <qi...@qq.com>
Authored: Fri Oct 27 16:06:06 2017 +0800
Committer: Jacky Li <ja...@qq.com>
Committed: Mon Oct 30 14:55:00 2017 +0530

----------------------------------------------------------------------
 .../carbondata/spark/util/DataLoadingUtil.scala | 300 ++++++++++++
 .../command/management/LoadTableCommand.scala   | 464 ++++++-------------
 2 files changed, 452 insertions(+), 312 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/5936e7fb/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
new file mode 100644
index 0000000..445fdbb
--- /dev/null
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataLoadingUtil.scala
@@ -0,0 +1,300 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.spark.util
+
+import scala.collection.immutable
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.commons.lang3.StringUtils
+
+import org.apache.carbondata.common.constants.LoggerAction
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.constants.{CarbonCommonConstants, CarbonLoadOptionConstants}
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
+import org.apache.carbondata.processing.loading.constants.DataLoadProcessorConstants
+import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, CarbonLoadModel}
+import org.apache.carbondata.processing.util.TableOptionConstant
+import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
+import org.apache.carbondata.spark.load.ValidateUtil
+
+/**
+ * the util object of data loading
+ */
+object DataLoadingUtil {
+
+  /**
+   * get data loading options and initialise default value
+   */
+  def getDataLoadingOptions(
+      carbonProperty: CarbonProperties,
+      options: immutable.Map[String, String]): mutable.Map[String, String] = {
+    val optionsFinal = scala.collection.mutable.Map[String, String]()
+    optionsFinal.put("delimiter", options.getOrElse("delimiter", ","))
+    optionsFinal.put("quotechar", options.getOrElse("quotechar", "\""))
+    optionsFinal.put("fileheader", options.getOrElse("fileheader", ""))
+    optionsFinal.put("escapechar", options.getOrElse("escapechar", "\\"))
+    optionsFinal.put("commentchar", options.getOrElse("commentchar", "#"))
+    optionsFinal.put("columndict", options.getOrElse("columndict", null))
+
+    optionsFinal.put(
+      "serialization_null_format",
+      options.getOrElse("serialization_null_format", "\\N"))
+
+    optionsFinal.put(
+      "bad_records_logger_enable",
+      options.getOrElse(
+        "bad_records_logger_enable",
+        carbonProperty.getProperty(
+          CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORDS_LOGGER_ENABLE,
+          CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORDS_LOGGER_ENABLE_DEFAULT)))
+
+    val badRecordActionValue = carbonProperty.getProperty(
+      CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION,
+      CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION_DEFAULT)
+
+    optionsFinal.put(
+      "bad_records_action",
+      options.getOrElse(
+        "bad_records_action",
+        carbonProperty.getProperty(
+          CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORDS_ACTION,
+          badRecordActionValue)))
+
+    optionsFinal.put(
+      "is_empty_data_bad_record",
+      options.getOrElse(
+        "is_empty_data_bad_record",
+        carbonProperty.getProperty(
+          CarbonLoadOptionConstants.CARBON_OPTIONS_IS_EMPTY_DATA_BAD_RECORD,
+          CarbonLoadOptionConstants.CARBON_OPTIONS_IS_EMPTY_DATA_BAD_RECORD_DEFAULT)))
+
+    optionsFinal.put("all_dictionary_path", options.getOrElse("all_dictionary_path", ""))
+
+    optionsFinal.put(
+      "complex_delimiter_level_1",
+      options.getOrElse("complex_delimiter_level_1", "\\$"))
+
+    optionsFinal.put(
+      "complex_delimiter_level_2",
+      options.getOrElse("complex_delimiter_level_2", "\\:"))
+
+    optionsFinal.put(
+      "dateformat",
+      options.getOrElse(
+        "dateformat",
+        carbonProperty.getProperty(
+          CarbonLoadOptionConstants.CARBON_OPTIONS_DATEFORMAT,
+          CarbonLoadOptionConstants.CARBON_OPTIONS_DATEFORMAT_DEFAULT)))
+
+    optionsFinal.put(
+      "global_sort_partitions",
+      options.getOrElse(
+        "global_sort_partitions",
+        carbonProperty.getProperty(
+          CarbonLoadOptionConstants.CARBON_OPTIONS_GLOBAL_SORT_PARTITIONS,
+          null)))
+
+    optionsFinal.put("maxcolumns", options.getOrElse("maxcolumns", null))
+
+    optionsFinal.put(
+      "batch_sort_size_inmb",
+      options.getOrElse(
+        "batch_sort_size_inmb",
+        carbonProperty.getProperty(
+          CarbonLoadOptionConstants.CARBON_OPTIONS_BATCH_SORT_SIZE_INMB,
+          carbonProperty.getProperty(
+            CarbonCommonConstants.LOAD_BATCH_SORT_SIZE_INMB,
+            CarbonCommonConstants.LOAD_BATCH_SORT_SIZE_INMB_DEFAULT))))
+
+    optionsFinal.put(
+      "bad_record_path",
+      options.getOrElse(
+        "bad_record_path",
+        carbonProperty.getProperty(
+          CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORD_PATH,
+          carbonProperty.getProperty(
+            CarbonCommonConstants.CARBON_BADRECORDS_LOC,
+            CarbonCommonConstants.CARBON_BADRECORDS_LOC_DEFAULT_VAL))))
+
+    val useOnePass = options.getOrElse(
+      "single_pass",
+      carbonProperty.getProperty(
+        CarbonLoadOptionConstants.CARBON_OPTIONS_SINGLE_PASS,
+        CarbonLoadOptionConstants.CARBON_OPTIONS_SINGLE_PASS_DEFAULT)).trim.toLowerCase match {
+      case "true" =>
+        true
+      case "false" =>
+        // when single_pass = false  and if either alldictionarypath
+        // or columnDict is configured the do not allow load
+        if (StringUtils.isNotEmpty(optionsFinal("all_dictionary_path")) ||
+            StringUtils.isNotEmpty(optionsFinal("columndict"))) {
+          throw new MalformedCarbonCommandException(
+            "Can not use all_dictionary_path or columndict without single_pass.")
+        } else {
+          false
+        }
+      case illegal =>
+        val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+        LOGGER.error(s"Can't use single_pass, because illegal syntax found: [$illegal] " +
+                     "Please set it as 'true' or 'false'")
+        false
+    }
+    optionsFinal.put("single_pass", useOnePass.toString)
+    optionsFinal
+  }
+
+  /**
+   * check whether using default value or not
+   */
+  private def checkDefaultValue(value: String, default: String) = {
+    if (StringUtils.isEmpty(value)) {
+      default
+    } else {
+      value
+    }
+  }
+
+  /**
+   * build CarbonLoadModel for data loading
+   */
+  def buildCarbonLoadModel(
+      table: CarbonTable,
+      carbonProperty: CarbonProperties,
+      options: immutable.Map[String, String],
+      optionsFinal: mutable.Map[String, String],
+      carbonLoadModel: CarbonLoadModel): Unit = {
+    carbonLoadModel.setTableName(table.getFactTableName)
+    carbonLoadModel.setDatabaseName(table.getDatabaseName)
+    carbonLoadModel.setStorePath(table.getStorePath)
+    carbonLoadModel.setTableName(table.getFactTableName)
+    val dataLoadSchema = new CarbonDataLoadSchema(table)
+    // Need to fill dimension relation
+    carbonLoadModel.setCarbonDataLoadSchema(dataLoadSchema)
+    val sort_scope = optionsFinal("sort_scope")
+    val single_pass = optionsFinal("single_pass")
+    val bad_records_logger_enable = optionsFinal("bad_records_logger_enable")
+    val bad_records_action = optionsFinal("bad_records_action")
+    val bad_record_path = optionsFinal("bad_record_path")
+    val global_sort_partitions = optionsFinal("global_sort_partitions")
+    val dateFormat = optionsFinal("dateformat")
+    val delimeter = optionsFinal("delimiter")
+    val complex_delimeter_level1 = optionsFinal("complex_delimiter_level_1")
+    val complex_delimeter_level2 = optionsFinal("complex_delimiter_level_2")
+    val all_dictionary_path = optionsFinal("all_dictionary_path")
+    val column_dict = optionsFinal("columndict")
+    ValidateUtil.validateDateFormat(dateFormat, table, table.getFactTableName)
+    ValidateUtil.validateSortScope(table, sort_scope)
+
+    if (bad_records_logger_enable.toBoolean ||
+        LoggerAction.REDIRECT.name().equalsIgnoreCase(bad_records_action)) {
+      if (!CarbonUtil.isValidBadStorePath(bad_record_path)) {
+        sys.error("Invalid bad records location.")
+      }
+    }
+    carbonLoadModel.setBadRecordsLocation(bad_record_path)
+
+    ValidateUtil.validateGlobalSortPartitions(global_sort_partitions)
+    carbonLoadModel.setEscapeChar(checkDefaultValue(optionsFinal("escapechar"), "\\"))
+    carbonLoadModel.setQuoteChar(checkDefaultValue(optionsFinal("quotechar"), "\""))
+    carbonLoadModel.setCommentChar(checkDefaultValue(optionsFinal("commentchar"), "#"))
+
+    // if there isn't file header in csv file and load sql doesn't provide FILEHEADER option,
+    // we should use table schema to generate file header.
+    var fileHeader = optionsFinal("fileheader")
+    val headerOption = options.get("header")
+    if (headerOption.isDefined) {
+      // whether the csv file has file header
+      // the default value is true
+      val header = try {
+        headerOption.get.toBoolean
+      } catch {
+        case ex: IllegalArgumentException =>
+          throw new MalformedCarbonCommandException(
+            "'header' option should be either 'true' or 'false'. " + ex.getMessage)
+      }
+      if (header) {
+        if (fileHeader.nonEmpty) {
+          throw new MalformedCarbonCommandException(
+            "When 'header' option is true, 'fileheader' option is not required.")
+        }
+      } else {
+        if (fileHeader.isEmpty) {
+          fileHeader = table.getCreateOrderColumn(table.getFactTableName)
+            .asScala.map(_.getColName).mkString(",")
+        }
+      }
+    }
+
+    carbonLoadModel.setDateFormat(dateFormat)
+    carbonLoadModel.setDefaultTimestampFormat(carbonProperty.getProperty(
+      CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
+      CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT))
+
+    carbonLoadModel.setDefaultDateFormat(carbonProperty.getProperty(
+      CarbonCommonConstants.CARBON_DATE_FORMAT,
+      CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT))
+
+    carbonLoadModel.setSerializationNullFormat(
+        TableOptionConstant.SERIALIZATION_NULL_FORMAT.getName + "," +
+        optionsFinal("serialization_null_format"))
+
+    carbonLoadModel.setBadRecordsLoggerEnable(
+        TableOptionConstant.BAD_RECORDS_LOGGER_ENABLE.getName + "," + bad_records_logger_enable)
+
+    carbonLoadModel.setBadRecordsAction(
+        TableOptionConstant.BAD_RECORDS_ACTION.getName + "," + bad_records_action)
+
+    carbonLoadModel.setIsEmptyDataBadRecord(
+        DataLoadProcessorConstants.IS_EMPTY_DATA_BAD_RECORD + "," +
+        optionsFinal("is_empty_data_bad_record"))
+
+    carbonLoadModel.setSortScope(sort_scope)
+    carbonLoadModel.setBatchSortSizeInMb(optionsFinal("batch_sort_size_inmb"))
+    carbonLoadModel.setGlobalSortPartitions(global_sort_partitions)
+    carbonLoadModel.setUseOnePass(single_pass.toBoolean)
+
+    if (delimeter.equalsIgnoreCase(complex_delimeter_level1) ||
+        complex_delimeter_level1.equalsIgnoreCase(complex_delimeter_level2) ||
+        delimeter.equalsIgnoreCase(complex_delimeter_level2)) {
+      sys.error(s"Field Delimiter & Complex types delimiter are same")
+    } else {
+      carbonLoadModel.setComplexDelimiterLevel1(
+        CarbonUtil.delimiterConverter(complex_delimeter_level1))
+      carbonLoadModel.setComplexDelimiterLevel2(
+        CarbonUtil.delimiterConverter(complex_delimeter_level2))
+    }
+    // set local dictionary path, and dictionary file extension
+    carbonLoadModel.setAllDictPath(all_dictionary_path)
+    carbonLoadModel.setCsvDelimiter(CarbonUtil.unescapeChar(delimeter))
+    carbonLoadModel.setCsvHeader(fileHeader)
+    carbonLoadModel.setColDictFilePath(column_dict)
+    carbonLoadModel.setDirectLoad(true)
+    carbonLoadModel.setCsvHeaderColumns(CommonUtil.getCsvHeaderColumns(carbonLoadModel))
+
+    val validatedMaxColumns = CommonUtil.validateMaxColumns(
+      carbonLoadModel.getCsvHeaderColumns,
+      optionsFinal("maxcolumns"))
+
+    carbonLoadModel.setMaxColumns(validatedMaxColumns.toString)
+    if (null == carbonLoadModel.getLoadMetadataDetails) {
+      CommonUtil.readLoadMetadataDetails(carbonLoadModel)
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5936e7fb/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 9018f7b..630ee27 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
@@ -27,7 +27,6 @@ import org.apache.spark.sql.execution.command.{DataLoadTableFileMapping, DataPro
 import org.apache.spark.sql.hive.CarbonRelation
 import org.apache.spark.util.{CausedBy, FileUtils}
 
-import org.apache.carbondata.common.constants.LoggerAction
 import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
 import org.apache.carbondata.core.constants.{CarbonCommonConstants, CarbonLoadOptionConstants}
 import org.apache.carbondata.core.datastore.impl.FileFactory
@@ -39,14 +38,11 @@ import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
 import org.apache.carbondata.core.util.path.CarbonStorePath
 import org.apache.carbondata.format
 import org.apache.carbondata.processing.exception.DataLoadingException
-import org.apache.carbondata.processing.loading.constants.DataLoadProcessorConstants
 import org.apache.carbondata.processing.loading.exception.NoRetryException
 import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, CarbonLoadModel}
-import org.apache.carbondata.processing.util.TableOptionConstant
 import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
-import org.apache.carbondata.spark.load.ValidateUtil
 import org.apache.carbondata.spark.rdd.{CarbonDataRDDFactory, DictionaryLoadModel}
-import org.apache.carbondata.spark.util.{CommonUtil, GlobalDictionaryUtil}
+import org.apache.carbondata.spark.util.{CommonUtil, DataLoadingUtil, GlobalDictionaryUtil}
 
 case class LoadTableCommand(
     databaseNameOp: Option[String],
@@ -60,89 +56,6 @@ case class LoadTableCommand(
     updateModel: Option[UpdateTableModel] = None)
   extends RunnableCommand with DataProcessCommand {
 
-  private def getFinalOptions(carbonProperty: CarbonProperties):
-  scala.collection.mutable.Map[String, String] = {
-    val optionsFinal = scala.collection.mutable.Map[String, String]()
-    optionsFinal.put("delimiter", options.getOrElse("delimiter", ","))
-    optionsFinal.put("quotechar", options.getOrElse("quotechar", "\""))
-    optionsFinal.put("fileheader", options.getOrElse("fileheader", ""))
-    optionsFinal.put("escapechar", options.getOrElse("escapechar", "\\"))
-    optionsFinal.put("commentchar", options.getOrElse("commentchar", "#"))
-    optionsFinal.put("columndict", options.getOrElse("columndict", null))
-    optionsFinal
-      .put("serialization_null_format", options.getOrElse("serialization_null_format", "\\N"))
-    optionsFinal.put("bad_records_logger_enable", options.getOrElse("bad_records_logger_enable",
-      carbonProperty
-        .getProperty(CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORDS_LOGGER_ENABLE,
-          CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORDS_LOGGER_ENABLE_DEFAULT)))
-    val badRecordActionValue = carbonProperty
-      .getProperty(CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION,
-        CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION_DEFAULT)
-    optionsFinal.put("bad_records_action", options.getOrElse("bad_records_action", carbonProperty
-      .getProperty(CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORDS_ACTION,
-        badRecordActionValue)))
-    optionsFinal
-      .put("is_empty_data_bad_record", options.getOrElse("is_empty_data_bad_record", carbonProperty
-        .getProperty(CarbonLoadOptionConstants.CARBON_OPTIONS_IS_EMPTY_DATA_BAD_RECORD,
-          CarbonLoadOptionConstants.CARBON_OPTIONS_IS_EMPTY_DATA_BAD_RECORD_DEFAULT)))
-    optionsFinal.put("all_dictionary_path", options.getOrElse("all_dictionary_path", ""))
-    optionsFinal
-      .put("complex_delimiter_level_1", options.getOrElse("complex_delimiter_level_1", "\\$"))
-    optionsFinal
-      .put("complex_delimiter_level_2", options.getOrElse("complex_delimiter_level_2", "\\:"))
-    optionsFinal.put("dateformat", options.getOrElse("dateformat",
-      carbonProperty.getProperty(CarbonLoadOptionConstants.CARBON_OPTIONS_DATEFORMAT,
-        CarbonLoadOptionConstants.CARBON_OPTIONS_DATEFORMAT_DEFAULT)))
-
-    optionsFinal.put("global_sort_partitions", options.getOrElse("global_sort_partitions",
-      carbonProperty
-        .getProperty(CarbonLoadOptionConstants.CARBON_OPTIONS_GLOBAL_SORT_PARTITIONS, null)))
-
-    optionsFinal.put("maxcolumns", options.getOrElse("maxcolumns", null))
-
-    optionsFinal.put("batch_sort_size_inmb", options.getOrElse("batch_sort_size_inmb",
-      carbonProperty.getProperty(CarbonLoadOptionConstants.CARBON_OPTIONS_BATCH_SORT_SIZE_INMB,
-        carbonProperty.getProperty(CarbonCommonConstants.LOAD_BATCH_SORT_SIZE_INMB,
-          CarbonCommonConstants.LOAD_BATCH_SORT_SIZE_INMB_DEFAULT))))
-
-    optionsFinal.put("bad_record_path", options.getOrElse("bad_record_path",
-      carbonProperty.getProperty(CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORD_PATH,
-        carbonProperty.getProperty(CarbonCommonConstants.CARBON_BADRECORDS_LOC,
-          CarbonCommonConstants.CARBON_BADRECORDS_LOC_DEFAULT_VAL))))
-
-    val useOnePass = options.getOrElse("single_pass",
-      carbonProperty.getProperty(CarbonLoadOptionConstants.CARBON_OPTIONS_SINGLE_PASS,
-        CarbonLoadOptionConstants.CARBON_OPTIONS_SINGLE_PASS_DEFAULT)).trim.toLowerCase match {
-      case "true" =>
-        true
-      case "false" =>
-        // when single_pass = false  and if either alldictionarypath
-        // or columnDict is configured the do not allow load
-        if (StringUtils.isNotEmpty(optionsFinal("all_dictionary_path")) ||
-            StringUtils.isNotEmpty(optionsFinal("columndict"))) {
-          throw new MalformedCarbonCommandException(
-            "Can not use all_dictionary_path or columndict without single_pass.")
-        } else {
-          false
-        }
-      case illegal =>
-        val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
-        LOGGER.error(s"Can't use single_pass, because illegal syntax found: [" + illegal + "] " +
-                     "Please set it as 'true' or 'false'")
-        false
-    }
-    optionsFinal.put("single_pass", useOnePass.toString)
-    optionsFinal
-  }
-
-  private def checkDefaultValue(value: String, default: String) = {
-    if (StringUtils.isEmpty(value)) {
-      default
-    } else {
-      value
-    }
-  }
-
   override def run(sparkSession: SparkSession): Seq[Row] = {
     processData(sparkSession)
   }
@@ -158,7 +71,6 @@ case class LoadTableCommand(
     }
 
     val dbName = databaseNameOp.getOrElse(sparkSession.catalog.currentDatabase)
-
     val relation = CarbonEnv.getInstance(sparkSession).carbonMetastore
       .lookupRelation(Option(dbName), tableName)(sparkSession).asInstanceOf[CarbonRelation]
     if (relation == null) {
@@ -172,7 +84,7 @@ case class LoadTableCommand(
 
     val carbonProperty: CarbonProperties = CarbonProperties.getInstance()
     carbonProperty.addProperty("zookeeper.enable.lock", "false")
-    val optionsFinal = getFinalOptions(carbonProperty)
+    val optionsFinal = DataLoadingUtil.getDataLoadingOptions(carbonProperty, options)
 
     val tableProperties = relation.tableMeta.carbonTable.getTableInfo
       .getFactTable.getTableProperties
@@ -183,133 +95,26 @@ case class LoadTableCommand(
           CarbonCommonConstants.LOAD_SORT_SCOPE_DEFAULT))))
 
     try {
+      val table = relation.tableMeta.carbonTable
+      val carbonLoadModel = new CarbonLoadModel()
       val factPath = if (dataFrame.isDefined) {
         ""
       } else {
         FileUtils.getPaths(
           CarbonUtil.checkAndAppendHDFSUrl(factPathFromUser))
       }
-      val carbonLoadModel = new CarbonLoadModel()
-      carbonLoadModel.setTableName(relation.tableMeta.carbonTableIdentifier.getTableName)
-      carbonLoadModel.setDatabaseName(relation.tableMeta.carbonTableIdentifier.getDatabaseName)
-      carbonLoadModel.setStorePath(relation.tableMeta.carbonTable.getStorePath)
-
-      val table = relation.tableMeta.carbonTable
-      carbonLoadModel.setTableName(table.getFactTableName)
-      val dataLoadSchema = new CarbonDataLoadSchema(table)
-      // Need to fill dimension relation
-      carbonLoadModel.setCarbonDataLoadSchema(dataLoadSchema)
-
-      val partitionLocation = relation.tableMeta.storePath + "/partition/" +
-                              relation.tableMeta.carbonTableIdentifier.getDatabaseName + "/" +
-                              relation.tableMeta.carbonTableIdentifier.getTableName + "/"
-      val columnar = sparkSession.conf.get("carbon.is.columnar.storage", "true").toBoolean
-      val sort_scope = optionsFinal("sort_scope")
-      val single_pass = optionsFinal("single_pass")
-      val bad_records_logger_enable = optionsFinal("bad_records_logger_enable")
-      val bad_records_action = optionsFinal("bad_records_action")
-      val bad_record_path = optionsFinal("bad_record_path")
-      val global_sort_partitions = optionsFinal("global_sort_partitions")
-      val dateFormat = optionsFinal("dateformat")
-      val delimeter = optionsFinal("delimiter")
-      val complex_delimeter_level1 = optionsFinal("complex_delimiter_level_1")
-      val complex_delimeter_level2 = optionsFinal("complex_delimiter_level_2")
-      val all_dictionary_path = optionsFinal("all_dictionary_path")
-      val column_dict = optionsFinal("columndict")
-      ValidateUtil.validateDateFormat(dateFormat, table, tableName)
-      ValidateUtil.validateSortScope(table, sort_scope)
-
-      if (bad_records_logger_enable.toBoolean ||
-          LoggerAction.REDIRECT.name().equalsIgnoreCase(bad_records_action)) {
-        if (!CarbonUtil.isValidBadStorePath(bad_record_path)) {
-          sys.error("Invalid bad records location.")
-        }
-      }
-      carbonLoadModel.setBadRecordsLocation(bad_record_path)
-
-      ValidateUtil.validateGlobalSortPartitions(global_sort_partitions)
-      carbonLoadModel.setEscapeChar(checkDefaultValue(optionsFinal("escapechar"), "\\"))
-      carbonLoadModel.setQuoteChar(checkDefaultValue(optionsFinal("quotechar"), "\""))
-      carbonLoadModel.setCommentChar(checkDefaultValue(optionsFinal("commentchar"), "#"))
-
-      // if there isn't file header in csv file and load sql doesn't provide FILEHEADER option,
-      // we should use table schema to generate file header.
-      var fileHeader = optionsFinal("fileheader")
-      val headerOption = options.get("header")
-      if (headerOption.isDefined) {
-        // whether the csv file has file header
-        // the default value is true
-        val header = try {
-          headerOption.get.toBoolean
-        } catch {
-          case ex: IllegalArgumentException =>
-            throw new MalformedCarbonCommandException(
-              "'header' option should be either 'true' or 'false'. " + ex.getMessage)
-        }
-        if (header) {
-          if (fileHeader.nonEmpty) {
-            throw new MalformedCarbonCommandException(
-              "When 'header' option is true, 'fileheader' option is not required.")
-          }
-        } else {
-          if (fileHeader.isEmpty) {
-            fileHeader = table.getCreateOrderColumn(table.getFactTableName)
-              .asScala.map(_.getColName).mkString(",")
-          }
-        }
-      }
-
-      carbonLoadModel.setDateFormat(dateFormat)
-      carbonLoadModel.setDefaultTimestampFormat(carbonProperty.getProperty(
-        CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
-        CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT))
-      carbonLoadModel.setDefaultDateFormat(carbonProperty.getProperty(
-        CarbonCommonConstants.CARBON_DATE_FORMAT,
-        CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT))
-      carbonLoadModel
-        .setSerializationNullFormat(
-          TableOptionConstant.SERIALIZATION_NULL_FORMAT.getName + "," +
-          optionsFinal("serialization_null_format"))
-      carbonLoadModel
-        .setBadRecordsLoggerEnable(
-          TableOptionConstant.BAD_RECORDS_LOGGER_ENABLE.getName + "," + bad_records_logger_enable)
-      carbonLoadModel
-        .setBadRecordsAction(
-          TableOptionConstant.BAD_RECORDS_ACTION.getName + "," + bad_records_action)
-      carbonLoadModel
-        .setIsEmptyDataBadRecord(
-          DataLoadProcessorConstants.IS_EMPTY_DATA_BAD_RECORD + "," +
-          optionsFinal("is_empty_data_bad_record"))
-      carbonLoadModel.setSortScope(sort_scope)
-      carbonLoadModel.setBatchSortSizeInMb(optionsFinal("batch_sort_size_inmb"))
-      carbonLoadModel.setGlobalSortPartitions(global_sort_partitions)
-      carbonLoadModel.setUseOnePass(single_pass.toBoolean)
-      if (delimeter.equalsIgnoreCase(complex_delimeter_level1) ||
-          complex_delimeter_level1.equalsIgnoreCase(complex_delimeter_level2) ||
-          delimeter.equalsIgnoreCase(complex_delimeter_level2)) {
-        sys.error(s"Field Delimiter & Complex types delimiter are same")
-      } else {
-        carbonLoadModel.setComplexDelimiterLevel1(
-          CarbonUtil.delimiterConverter(complex_delimeter_level1))
-        carbonLoadModel.setComplexDelimiterLevel2(
-          CarbonUtil.delimiterConverter(complex_delimeter_level2))
-      }
-      // set local dictionary path, and dictionary file extension
-      carbonLoadModel.setAllDictPath(all_dictionary_path)
-
-      val partitionStatus = CarbonCommonConstants.STORE_LOADSTATUS_SUCCESS
-      try {
+      carbonLoadModel.setFactFilePath(factPath)
+      DataLoadingUtil.buildCarbonLoadModel(
+        table,
+        carbonProperty,
+        options,
+        optionsFinal,
+        carbonLoadModel
+      )
+
+      try{
         // First system has to partition the data first and then call the load data
         LOGGER.info(s"Initiating Direct Load for the Table : ($dbName.$tableName)")
-        carbonLoadModel.setFactFilePath(factPath)
-        carbonLoadModel.setCsvDelimiter(CarbonUtil.unescapeChar(delimeter))
-        carbonLoadModel.setCsvHeader(fileHeader)
-        carbonLoadModel.setColDictFilePath(column_dict)
-        carbonLoadModel.setDirectLoad(true)
-        carbonLoadModel.setCsvHeaderColumns(CommonUtil.getCsvHeaderColumns(carbonLoadModel))
-        val validatedMaxColumns = CommonUtil.validateMaxColumns(carbonLoadModel.getCsvHeaderColumns,
-          optionsFinal("maxcolumns"))
-        carbonLoadModel.setMaxColumns(validatedMaxColumns.toString)
         GlobalDictionaryUtil.updateTableMetadataFunc = updateTableMetadata
         val storePath = relation.tableMeta.storePath
         // add the start entry for the new load in the table status file
@@ -320,11 +125,9 @@ case class LoadTableCommand(
         if (isOverwriteTable) {
           LOGGER.info(s"Overwrite of carbon table with $dbName.$tableName is in progress")
         }
-        if (null == carbonLoadModel.getLoadMetadataDetails) {
-          CommonUtil.readLoadMetadataDetails(carbonLoadModel)
-        }
         if (carbonLoadModel.getLoadMetadataDetails.isEmpty && carbonLoadModel.getUseOnePass &&
-            StringUtils.isEmpty(column_dict) && StringUtils.isEmpty(all_dictionary_path)) {
+            StringUtils.isEmpty(carbonLoadModel.getColDictFilePath) &&
+            StringUtils.isEmpty(carbonLoadModel.getAllDictPath)) {
           LOGGER.info(s"Cannot use single_pass=true for $dbName.$tableName during the first load")
           LOGGER.audit(s"Cannot use single_pass=true for $dbName.$tableName during the first load")
           carbonLoadModel.setUseOnePass(false)
@@ -337,111 +140,21 @@ case class LoadTableCommand(
         if (!FileFactory.isFileExist(metadataDirectoryPath, fileType)) {
           FileFactory.mkdirs(metadataDirectoryPath, fileType)
         }
+        val partitionStatus = CarbonCommonConstants.STORE_LOADSTATUS_SUCCESS
+        val columnar = sparkSession.conf.get("carbon.is.columnar.storage", "true").toBoolean
         if (carbonLoadModel.getUseOnePass) {
-          val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
-          val carbonTableIdentifier = carbonTable.getAbsoluteTableIdentifier
-            .getCarbonTableIdentifier
-          val carbonTablePath = CarbonStorePath
-            .getCarbonTablePath(storePath, carbonTableIdentifier)
-          val dictFolderPath = carbonTablePath.getMetadataDirectoryPath
-          val dimensions = carbonTable.getDimensionByTableName(
-            carbonTable.getFactTableName).asScala.toArray
-          val colDictFilePath = carbonLoadModel.getColDictFilePath
-          if (!StringUtils.isEmpty(colDictFilePath)) {
-            carbonLoadModel.initPredefDictMap()
-            // generate predefined dictionary
-            GlobalDictionaryUtil
-              .generatePredefinedColDictionary(colDictFilePath, carbonTableIdentifier,
-                dimensions, carbonLoadModel, sparkSession.sqlContext, storePath, dictFolderPath)
-          }
-          if (!StringUtils.isEmpty(all_dictionary_path)) {
-            carbonLoadModel.initPredefDictMap()
-            GlobalDictionaryUtil
-              .generateDictionaryFromDictionaryFiles(sparkSession.sqlContext,
-                carbonLoadModel,
-                storePath,
-                carbonTableIdentifier,
-                dictFolderPath,
-                dimensions,
-                all_dictionary_path)
-          }
-          // dictionaryServerClient dictionary generator
-          val dictionaryServerPort = carbonProperty
-            .getProperty(CarbonCommonConstants.DICTIONARY_SERVER_PORT,
-              CarbonCommonConstants.DICTIONARY_SERVER_PORT_DEFAULT)
-          val sparkDriverHost = sparkSession.sqlContext.sparkContext.
-            getConf.get("spark.driver.host")
-          carbonLoadModel.setDictionaryServerHost(sparkDriverHost)
-          // start dictionary server when use one pass load and dimension with DICTIONARY
-          // encoding is present.
-          val allDimensions = table.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(dictionaryServerPort.toInt, carbonTable)
-            carbonLoadModel.setDictionaryServerPort(dictionaryServer.getPort)
-            sparkSession.sparkContext.addSparkListener(new SparkListener() {
-              override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) {
-                dictionaryServer.shutdown()
-              }
-            })
-            Some(dictionaryServer)
-          } else {
-            None
-          }
-          CarbonDataRDDFactory.loadCarbonData(sparkSession.sqlContext,
+          loadDataUsingOnePass(
+            sparkSession,
+            carbonProperty,
             carbonLoadModel,
-            relation.tableMeta.storePath,
             columnar,
-            partitionStatus,
-            server,
-            isOverwriteTable,
-            dataFrame,
-            updateModel)
+            partitionStatus)
         } else {
-          val (dictionaryDataFrame, loadDataFrame) = if (updateModel.isDefined) {
-            val fields = dataFrame.get.schema.fields
-            import org.apache.spark.sql.functions.udf
-            // extracting only segment from tupleId
-            val getSegIdUDF = udf((tupleId: String) =>
-              CarbonUpdateUtil.getRequiredFieldFromTID(tupleId, TupleIdEnum.SEGMENT_ID))
-            // getting all fields except tupleId field as it is not required in the value
-            var otherFields = fields.toSeq
-              .filter(field => !field.name
-                .equalsIgnoreCase(CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID))
-              .map(field => new Column(field.name))
-
-            // extract tupleId field which will be used as a key
-            val segIdColumn = getSegIdUDF(new Column(UnresolvedAttribute
-              .quotedString(CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID))).
-              as(CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_SEGMENTID)
-            // use dataFrameWithoutTupleId as dictionaryDataFrame
-            val dataFrameWithoutTupleId = dataFrame.get.select(otherFields: _*)
-            otherFields = otherFields :+ segIdColumn
-            // use dataFrameWithTupleId as loadDataFrame
-            val dataFrameWithTupleId = dataFrame.get.select(otherFields: _*)
-            (Some(dataFrameWithoutTupleId), Some(dataFrameWithTupleId))
-          } else {
-            (dataFrame, dataFrame)
-          }
-
-          GlobalDictionaryUtil.generateGlobalDictionary(
-            sparkSession.sqlContext,
-            carbonLoadModel,
-            relation.tableMeta.storePath,
-            dictionaryDataFrame)
-          CarbonDataRDDFactory.loadCarbonData(sparkSession.sqlContext,
+          loadData(
+            sparkSession,
             carbonLoadModel,
-            relation.tableMeta.storePath,
             columnar,
-            partitionStatus,
-            None,
-            isOverwriteTable,
-            loadDataFrame,
-            updateModel)
+            partitionStatus)
         }
       } catch {
         case CausedBy(ex: NoRetryException) =>
@@ -454,6 +167,9 @@ case class LoadTableCommand(
       } finally {
         // Once the data load is successful delete the unwanted partition files
         try {
+          val partitionLocation = table.getStorePath + "/partition/" +
+                                  table.getDatabaseName + "/" +
+                                  table.getFactTableName + "/"
           val fileType = FileFactory.getFileType(partitionLocation)
           if (FileFactory.isFileExist(partitionLocation, fileType)) {
             val file = FileFactory
@@ -480,6 +196,130 @@ case class LoadTableCommand(
     Seq.empty
   }
 
+  private def loadDataUsingOnePass(
+      sparkSession: SparkSession,
+      carbonProperty: CarbonProperties,
+      carbonLoadModel: CarbonLoadModel,
+      columnar: Boolean,
+      partitionStatus: String): Unit = {
+    val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
+    val carbonTableIdentifier = carbonTable.getAbsoluteTableIdentifier
+      .getCarbonTableIdentifier
+    val carbonTablePath = CarbonStorePath
+      .getCarbonTablePath(carbonLoadModel.getStorePath, carbonTableIdentifier)
+    val dictFolderPath = carbonTablePath.getMetadataDirectoryPath
+    val dimensions = carbonTable.getDimensionByTableName(
+      carbonTable.getFactTableName).asScala.toArray
+    val colDictFilePath = carbonLoadModel.getColDictFilePath
+    if (!StringUtils.isEmpty(colDictFilePath)) {
+      carbonLoadModel.initPredefDictMap()
+      // generate predefined dictionary
+      GlobalDictionaryUtil.generatePredefinedColDictionary(
+        colDictFilePath,
+        carbonTableIdentifier,
+        dimensions,
+        carbonLoadModel,
+        sparkSession.sqlContext,
+        carbonLoadModel.getStorePath,
+        dictFolderPath)
+    }
+    if (!StringUtils.isEmpty(carbonLoadModel.getAllDictPath)) {
+      carbonLoadModel.initPredefDictMap()
+      GlobalDictionaryUtil
+        .generateDictionaryFromDictionaryFiles(sparkSession.sqlContext,
+          carbonLoadModel,
+          carbonLoadModel.getStorePath,
+          carbonTableIdentifier,
+          dictFolderPath,
+          dimensions,
+          carbonLoadModel.getAllDictPath)
+    }
+    // dictionaryServerClient dictionary generator
+    val dictionaryServerPort = carbonProperty
+      .getProperty(CarbonCommonConstants.DICTIONARY_SERVER_PORT,
+        CarbonCommonConstants.DICTIONARY_SERVER_PORT_DEFAULT)
+    val sparkDriverHost = sparkSession.sqlContext.sparkContext.
+      getConf.get("spark.driver.host")
+    carbonLoadModel.setDictionaryServerHost(sparkDriverHost)
+    // start dictionary server when use one pass load and dimension with DICTIONARY
+    // encoding is present.
+    val allDimensions =
+      carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable.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(dictionaryServerPort.toInt, carbonTable)
+      carbonLoadModel.setDictionaryServerPort(dictionaryServer.getPort)
+      sparkSession.sparkContext.addSparkListener(new SparkListener() {
+        override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) {
+          dictionaryServer.shutdown()
+        }
+      })
+      Some(dictionaryServer)
+    } else {
+      None
+    }
+    CarbonDataRDDFactory.loadCarbonData(sparkSession.sqlContext,
+      carbonLoadModel,
+      carbonLoadModel.getStorePath,
+      columnar,
+      partitionStatus,
+      server,
+      isOverwriteTable,
+      dataFrame,
+      updateModel)
+  }
+
+  private def loadData(
+      sparkSession: SparkSession,
+      carbonLoadModel: CarbonLoadModel,
+      columnar: Boolean,
+      partitionStatus: String): Unit = {
+    val (dictionaryDataFrame, loadDataFrame) = if (updateModel.isDefined) {
+      val fields = dataFrame.get.schema.fields
+      import org.apache.spark.sql.functions.udf
+      // extracting only segment from tupleId
+      val getSegIdUDF = udf((tupleId: String) =>
+        CarbonUpdateUtil.getRequiredFieldFromTID(tupleId, TupleIdEnum.SEGMENT_ID))
+      // getting all fields except tupleId field as it is not required in the value
+      var otherFields = fields.toSeq
+        .filter(field => !field.name
+          .equalsIgnoreCase(CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID))
+        .map(field => new Column(field.name))
+
+      // extract tupleId field which will be used as a key
+      val segIdColumn = getSegIdUDF(new Column(UnresolvedAttribute
+        .quotedString(CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID))).
+        as(CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_SEGMENTID)
+      // use dataFrameWithoutTupleId as dictionaryDataFrame
+      val dataFrameWithoutTupleId = dataFrame.get.select(otherFields: _*)
+      otherFields = otherFields :+ segIdColumn
+      // use dataFrameWithTupleId as loadDataFrame
+      val dataFrameWithTupleId = dataFrame.get.select(otherFields: _*)
+      (Some(dataFrameWithoutTupleId), Some(dataFrameWithTupleId))
+    } else {
+      (dataFrame, dataFrame)
+    }
+
+    GlobalDictionaryUtil.generateGlobalDictionary(
+      sparkSession.sqlContext,
+      carbonLoadModel,
+      carbonLoadModel.getStorePath,
+      dictionaryDataFrame)
+    CarbonDataRDDFactory.loadCarbonData(sparkSession.sqlContext,
+      carbonLoadModel,
+      carbonLoadModel.getStorePath,
+      columnar,
+      partitionStatus,
+      None,
+      isOverwriteTable,
+      loadDataFrame,
+      updateModel)
+  }
+
   private def updateTableMetadata(
       carbonLoadModel: CarbonLoadModel,
       sqlContext: SQLContext,


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

Posted by ja...@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/pre-aggregate
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;
   }


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

Posted by ja...@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


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

Posted by ja...@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();


[20/24] carbondata git commit: [CARBONDATA-1669] Clean up code in CarbonDataRDDFactory

Posted by ja...@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/pre-aggregate
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,


[14/24] carbondata git commit: [Tests] Fix BTreeBlockFinderTest variable mistake

Posted by ja...@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/pre-aggregate
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);


[16/24] carbondata git commit: [CARBONDATA-1668] Remove isTableSplitPartition in data loading

Posted by ja...@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/pre-aggregate
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 = {


[02/24] carbondata git commit: [CARBONDATA-1652] Add examples for Carbon usage when integrating with Spark

Posted by ja...@apache.org.
[CARBONDATA-1652] Add examples for Carbon usage when integrating with Spark

This closes #1442


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

Branch: refs/heads/pre-aggregate
Commit: 93a604dd668b4bb84eacfecb4142be24c17a704f
Parents: 40eaa44
Author: Jacky Li <ja...@qq.com>
Authored: Sat Oct 28 00:09:01 2017 +0530
Committer: chenliang613 <ch...@apache.org>
Committed: Sat Oct 28 19:46:28 2017 +0530

----------------------------------------------------------------------
 .../spark2/src/main/resources/complexdata.csv   | 101 +++++++++++++++++
 .../spark2/src/main/resources/dataSample.csv    |  11 ++
 .../spark2/src/main/resources/dimSample.csv     |  21 ++++
 .../spark2/src/main/resources/factSample.csv    |  51 +++++++++
 .../examples/AllDictionaryExample.scala         |  73 +++++++++++++
 .../carbondata/examples/AllDictionaryUtil.scala | 109 +++++++++++++++++++
 .../carbondata/examples/AlluxioExample.scala    |  64 +++++++++++
 .../examples/CarbonSessionExample.scala         |  28 +----
 .../examples/CaseClassDataFrameAPIExample.scala |  47 ++++++++
 .../examples/DataFrameAPIExample.scala          |  49 +++++++++
 .../carbondata/examples/ExampleUtils.scala      | 107 ++++++++++++++++++
 .../carbondata/examples/HadoopFileExample.scala |  52 +++++++++
 12 files changed, 688 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/93a604dd/examples/spark2/src/main/resources/complexdata.csv
----------------------------------------------------------------------
diff --git a/examples/spark2/src/main/resources/complexdata.csv b/examples/spark2/src/main/resources/complexdata.csv
new file mode 100644
index 0000000..23a3949
--- /dev/null
+++ b/examples/spark2/src/main/resources/complexdata.csv
@@ -0,0 +1,101 @@
+deviceInformationId,channelsId,ROMSize,purchasedate,mobile,MAC,locationinfo,proddate,gamePointId,contractNumber
+1,109,4ROM size,29-11-2015,1AA1$2BB1,MAC1$MAC2$MAC3,7:Chinese:Hubei Province:yichang:yichang:yichang$7:India:New Delhi:delhi:delhi:delhi,29-11-2015$29-11-2015:29-11-2015,109,2738.562
+10,93,1ROM size,29-11-2015,1AA10$2BB10,MAC4$MAC5$MAC6,7:Chinese:Hubei Province:yichang:yichang:yichang$7:India:New Delhi:delhi:delhi:delhi,30-11-2015$30-11-2015:30-11-2015,93,1714.635
+100,2591,2ROM size,29-11-2015,1AA100$2BB100,MAC7$MAC8$MAC9,4:Chinese:Hunan Province:xiangtan:xiangtan:jianshelu$4:India:Hunan Province:xiangtan:xiangtan:jianshelu,01-12-2015$01-12-2015:01-12-2015,2591,1271
+1000,2531,2ROM size,29-11-2015,1AA1000$2BB1000,MAC10$$MAC12,6:Chinese:Hubei Province:wuhan:hongshan:hongshan$6:India:New Delhi:wuhan:hongshan:hongshan,02-12-2015$02-12-2015:02-12-2015,2531,692
+10000,2408,0ROM size,29-11-2015,1AA10000$2BB10000,MAC13$$MAC15,2:Chinese:Guangdong Province:guangzhou:longhua:mingzhi$2:India:Guangdong Province:guangzhou:longhua:mingzhi,03-12-2015$03-12-2015:03-12-2015,2408,2175
+100000,1815,0ROM size,29-11-2015,1AA100000$2BB100000,MAC16$$MAC18,6:Chinese:Hubei Province:wuhan:hongshan:hongshan$6:India:New Delhi:wuhan:hongshan:hongshan,04-12-2015$04-12-2015:04-12-2015,1815,136
+1000000,2479,4ROM size,29-11-2015,1AA1000000$2BB1000000,MAC19$$MAC21,7:Chinese:Hubei Province:yichang:yichang:yichang$7:India:New Delhi:delhi:delhi:delhi,05-12-2015$05-12-2015:05-12-2015,2479,1600
+100001,1845,7ROM size,29-11-2015,1AA100001$,MAC22$$MAC24,4:Chinese:Hunan Province:xiangtan:xiangtan:jianshelu$4:India:Hunan Province:xiangtan:xiangtan:jianshelu,06-12-2015$06-12-2015:06-12-2015,1845,505
+100002,2008,1ROM size,29-11-2015,1AA100002$,MAC25$$MAC27,3:Chinese:Hunan Province:changsha:yuhua:shazitang$3:India:Hunan Province:changsha:yuhua:shazitang,07-12-2015$07-12-2015:07-12-2015,2008,1341
+100003,1121,5ROM size,29-11-2015,1AA100003$,MAC28$$MAC30,5:Chinese:Hunan Province:zhuzhou:tianyuan:tianyua$5:India:Hunan Province:zhuzhou:tianyuan:tianyua,08-12-2015$08-12-2015:08-12-2015,1121,2239
+100004,1511,8ROM size,29-11-2015,1AA100004$,MAC31$$MAC33,7:Chinese:Hubei Province:yichang:yichang:yichang$7:India:New Delhi:delhi:delhi:delhi,09-12-2015$09-12-2015:09-12-2015,1511,2970
+100005,2759,0ROM size,29-11-2015,1AA100005$,MAC34$$MAC36,7:Chinese:Hubei Province:yichang:yichang:yichang$7:India:New Delhi:delhi:delhi:delhi,10-12-2015$10-12-2015:10-12-2015,2759,2593
+100006,2069,7ROM size,29-11-2015,1AA100006$,MAC37$$MAC39,3:Chinese:Hunan Province:changsha:yuhua:shazitang$3:India:Hunan Province:changsha:yuhua:shazitang,11-12-2015$11-12-2015:11-12-2015,2069,2572
+100007,396,7ROM size,29-11-2015,1AA100007$,MAC40$$MAC42,3:Chinese:Hunan Province:changsha:yuhua:shazitang$3:India:Hunan Province:changsha:yuhua:shazitang,12-12-2015$12-12-2015:12-12-2015,396,1991
+100008,104,2ROM size,29-11-2015,1AA100008$,MAC43$$MAC45,3:Chinese:Hunan Province:changsha:yuhua:shazitang$3:India:Hunan Province:changsha:yuhua:shazitang,13-12-2015$13-12-2015:13-12-2015,104,1442
+100009,477,3ROM size,29-11-2015,1AA100009$,MAC46$$MAC48,7:Chinese:Hubei Province:yichang:yichang:yichang$7:India:New Delhi:delhi:delhi:delhi,14-12-2015$14-12-2015:14-12-2015,477,1841
+10001,546,8ROM size,29-11-2015,1AA10001$2,MAC49$$MAC51,3:Chinese:Hunan Province:changsha:yuhua:shazitang$3:India:Hunan Province:changsha:yuhua:shazitang,15-12-2015$15-12-2015:15-12-2015,546,298
+100010,2696,3ROM size,29-11-2015,1AA100010$2BB100010,MAC52$$MAC54,5:Chinese:Hunan Province:zhuzhou:tianyuan:tianyua$5:India:Hunan Province:zhuzhou:tianyuan:tianyua,16-12-2015$16-12-2015:16-12-2015,2696,79
+100011,466,2ROM size,29-11-2015,1AA100011$2BB100011,MAC55$$MAC57,2:Chinese:Guangdong Province:guangzhou:longhua:mingzhi$2:India:Guangdong Province:guangzhou:longhua:mingzhi,17-12-2015$17-12-2015:17-12-2015,466,202
+100012,2644,2ROM size,29-11-2015,1AA100012$2BB100012,MAC58$$MAC60,4:Chinese:Hunan Province:xiangtan:xiangtan:jianshelu$4:India:Hunan Province:xiangtan:xiangtan:jianshelu,18-12-2015$18-12-2015:18-12-2015,2644,568
+100013,2167,3ROM size,29-11-2015,1AA100013$2BB100013,MAC61$MAC62,3:Chinese:Hunan Province:changsha:yuhua:shazitang$3:India:Hunan Province:changsha:yuhua:shazitang,19-12-2015$19-12-2015:19-12-2015,2167,355
+100014,1069,7ROM size,29-11-2015,1AA100014$2BB100014,MAC64$MAC65,5:Chinese:Hunan Province:zhuzhou:tianyuan:tianyua$5:India:Hunan Province:zhuzhou:tianyuan:tianyua,20-12-2015$20-12-2015:20-12-2015,1069,151
+100015,1447,9ROM size,29-11-2015,1AA100015$2BB100015,MAC67$MAC68,4:Chinese:Hunan Province:xiangtan:xiangtan:jianshelu$4:India:Hunan Province:xiangtan:xiangtan:jianshelu,21-12-2015$21-12-2015:21-12-2015,1447,2863
+100016,2963,3ROM size,29-11-2015,1AA100016$2BB100016,MAC70$MAC71,3:Chinese:Hunan Province:changsha:yuhua:shazitang$3:India:Hunan Province:changsha:yuhua:shazitang,22-12-2015$22-12-2015:22-12-2015,2963,1873
+100017,1580,5ROM size,29-11-2015,1AA100017$2BB100017,MAC73$MAC74,4:Chinese:Hunan Province:xiangtan:xiangtan:jianshelu$4:India:Hunan Province:xiangtan:xiangtan:jianshelu,23-12-2015$23-12-2015:23-12-2015,1580,2205
+100018,446,2ROM size,29-11-2015,1AA100018$2BB100018,MAC76$MAC77,7:Chinese:Hubei Province:yichang:yichang:yichang$7:India:New Delhi:delhi:delhi:delhi,24-12-2015$24-12-2015:24-12-2015,446,441
+100019,2151,7ROM size,29-11-2015,1AA100019$2BB100019,MAC79$MAC80,5:Chinese:Hunan Province:zhuzhou:tianyuan:tianyua$5:India:Hunan Province:zhuzhou:tianyuan:tianyua,25-12-2015$25-12-2015:25-12-2015,2151,2194
+10002,2201,1ROM size,29-11-2015,2BB10002,MAC82$MAC83,6:Chinese:Hubei Province:wuhan:hongshan:hongshan$6:India:New Delhi:wuhan:hongshan:hongshan,26-12-2015$26-12-2015:26-12-2015,2201,2972
+100020,2574,5ROM size,29-11-2015,$2BB100020,MAC85$MAC86,1:Chinese:Guangdong Province:shenzhen:longgang:matishan$1:India:Guangdong Province:shenzhen:longgang:matishan,27-12-2015$27-12-2015:27-12-2015,2574,256
+100021,1734,4ROM size,29-11-2015,$2BB100021,MAC88$MAC89,3:Chinese:Hunan Province:changsha:yuhua:shazitang$3:India:Hunan Province:changsha:yuhua:shazitang,28-12-2015$28-12-2015:28-12-2015,1734,1778
+100022,155,3ROM size,29-11-2015,$2BB100022,MAC91$MAC92,5:Chinese:Hunan Province:zhuzhou:tianyuan:tianyua$5:India:Hunan Province:zhuzhou:tianyuan:tianyua,29-12-2015$29-12-2015:29-12-2015,155,1999
+100023,1386,8ROM size,29-11-2015,$2BB100023,MAC94$MAC95,2:Chinese:Guangdong Province:guangzhou:longhua:mingzhi$2:India:Guangdong Province:guangzhou:longhua:mingzhi,30-12-2015$30-12-2015:30-12-2015,1386,2194
+100024,1017,9ROM size,29-11-2015,$2BB100024,MAC97$MAC98,3:Chinese:Hunan Province:changsha:yuhua:shazitang$3:India:Hunan Province:changsha:yuhua:shazitang,,1017,2483
+100025,47,2ROM size,29-11-2015,$2BB100025,$MAC101$MAC102,2:Chinese:Guangdong Province:guangzhou:longhua:mingzhi$2:India:Guangdong Province:guangzhou:longhua:mingzhi,,47,1724
+100026,2930,7ROM size,29-11-2015,$2BB100026,$MAC104$MAC105,7:Chinese:Hubei Province:yichang:yichang:yichang$7:India:New Delhi:delhi:delhi:delhi,,2930,1768
+100027,2940,0ROM size,29-11-2015,$2BB100027,$MAC107$MAC108,5:Chinese:Hunan Province:zhuzhou:tianyuan:tianyua$5:India:Hunan Province:zhuzhou:tianyuan:tianyua,,2940,2436
+100028,297,5ROM size,29-11-2015,$2BB100028,$MAC110$MAC111,5:Chinese:Hunan Province:zhuzhou:tianyuan:tianyua$5:India:Hunan Province:zhuzhou:tianyuan:tianyua,,297,2849
+100029,1695,2ROM size,29-11-2015,$2BB100029,$MAC113$MAC114,4:Chinese:Hunan Province:xiangtan:xiangtan:jianshelu$4:India:Hunan Province:xiangtan:xiangtan:jianshelu,,1695,1691
+10003,1326,7ROM size,29-11-2015,2BB10003,$MAC116$MAC117,4:Chinese:Hunan Province:xiangtan:xiangtan:jianshelu$4:India:Hunan Province:xiangtan:xiangtan:jianshelu,,1326,2071
+100030,513,7ROM size,29-11-2015,$2BB100030,$MAC119$MAC120,5:Chinese:Hunan Province:zhuzhou:tianyuan:tianyua$5:India:Hunan Province:zhuzhou:tianyuan:tianyua,07-01-2016$07-01-2016:,513,1333
+100031,1741,1ROM size,29-11-2015,$2BB100031,$MAC122$MAC123,7:Chinese:Hubei Province:yichang:yichang:yichang$7:India:New Delhi:delhi:delhi:delhi,08-01-2016$08-01-2016:,1741,1080
+100032,1198,0ROM size,29-11-2015,$2BB100032,$MAC125$MAC126,1:Chinese:Guangdong Province:shenzhen:longgang:matishan$1:India:Guangdong Province:shenzhen:longgang:matishan,09-01-2016$09-01-2016:,1198,1053
+100033,273,9ROM size,29-11-2015,$2BB100033,$MAC128$MAC129,6:Chinese:Hubei Province:wuhan:hongshan:hongshan$6:India:New Delhi:wuhan:hongshan:hongshan,10-01-2016$10-01-2016:,273,760
+100034,1234,6ROM size,29-11-2015,$2BB100034,$MAC131$MAC132,2:Chinese:Guangdong Province:guangzhou:longhua:mingzhi$2:India:Guangdong Province:guangzhou:longhua:mingzhi,11-01-2016$11-01-2016:,1234,2061
+100035,1619,1ROM size,29-11-2015,$2BB100035,$MAC134$MAC135,3:Chinese:Hunan Province:changsha:yuhua:shazitang$3:India:Hunan Province:changsha:yuhua:shazitang,12-01-2016$12-01-2016:,1619,2142
+100036,2415,2ROM size,29-11-2015,$2BB100036,$MAC137$MAC138,3:Chinese:Hunan Province:changsha:yuhua:shazitang$3:India:Hunan Province:changsha:yuhua:shazitang,13-01-2016$13-01-2016:,2415,2224
+100037,2381,2ROM size,29-11-2015,$2BB100037,$MAC140$MAC141,4:Chinese:Hunan Province:xiangtan:xiangtan:jianshelu$4:India:Hunan Province:xiangtan:xiangtan:jianshelu,14-01-2016$14-01-2016:,2381,1015
+100038,872,7ROM size,29-11-2015,1AA100038$2BB100038,$MAC143$MAC144,1:Chinese:Guangdong Province:shenzhen:longgang:matishan$1:India:Guangdong Province:shenzhen:longgang:matishan,15-01-2016$15-01-2016,872,1229
+100039,1835,9ROM size,29-11-2015,1AA100039$2BB100039,$$MAC147,1:Chinese:Guangdong Province:shenzhen:longgang:matishan$1:India:Guangdong Province:shenzhen:longgang:matishan,16-01-2016$16-01-2016,1835,1750
+10004,2597,1ROM size,29-11-2015,1AA10004$2BB10004,$$MAC150,2:Chinese:Guangdong Province:guangzhou:longhua:mingzhi$2:India:Guangdong Province:guangzhou:longhua:mingzhi,17-01-2016$17-01-2016,2597,1717
+100040,1969,9ROM size,29-11-2015,1AA100040$2BB100040,$$MAC153,,18-01-2016$18-01-2016,1969,2078
+100041,2133,8ROM size,29-11-2015,$,$$MAC156,,19-01-2016$19-01-2016,2133,2734
+100042,631,9ROM size,29-11-2015,$,$$MAC159,,20-01-2016$20-01-2016,631,2745
+100043,187,4ROM size,29-11-2015,$,$$MAC162,2:Chinese::guangzhou:longhua:mingzhi$2:India::guangzhou:longhua:mingzhi,21-01-2016$21-01-2016,187,571
+100044,1232,5ROM size,29-11-2015,$,$$MAC165,2::Guangdong Province:guangzhou:longhua:mingzhi$2::Guangdong Province:guangzhou:longhua:mingzhi,22-01-2016$22-01-2016,1232,1697
+100045,1602,6ROM size,29-11-2015,$,$$MAC168,4:Chinese:Hunan Province::xiangtan:jianshelu$4:India:Hunan Province::xiangtan:jianshelu,23-01-2016$23-01-2016,1602,2553
+100046,2319,9ROM size,29-11-2015,$,$$MAC171,2:Chinese:Guangdong Province:guangzhou::mingzhi$2:India:Guangdong Province:guangzhou::mingzhi,24-01-2016$24-01-2016,2319,1077
+100047,839,4ROM size,29-11-2015,$,$$MAC174,5:Chinese:Hunan Province:zhuzhou:tianyuan:$5:India:Hunan Province:zhuzhou:tianyuan:,25-01-2016$25-01-2016,839,1823
+100048,1184,2ROM size,29-11-2015,$,$$MAC177,2:Chinese:Guangdong Province:guangzhou:longhua:mingzhi$2:India:Guangdong Province:guangzhou:longhua:mingzhi,26-01-2016$:,1184,2399
+100049,2705,2ROM size,29-11-2015,$,$$MAC180,2:Chinese:Guangdong Province$2:India:Guangdong Province,27-01-2016$:,2705,2890
+10005,1185,1ROM size,29-11-2015,,$$MAC183,4:Chinese:Hunan Province:xiangtan:xiangtan:jianshelu$4:India:Hunan Province:xiangtan:xiangtan:jianshelu,28-01-2016$:,1185,1608
+100050,2457,9ROM size,29-11-2015,,$$MAC186,7:Chinese:Hubei Province:yichang:yichang:yichang$7:India:New Delhi:delhi:delhi:delhi,29-01-2016$:,2457,29
+100051,2320,8ROM size,29-11-2015,,$$MAC189,2:Chinese:Guangdong Province:guangzhou:longhua:mingzhi$2:India:Guangdong Province:guangzhou:longhua:mingzhi,30-01-2016$:,2320,1407
+100052,2300,0ROM size,29-11-2015,,$$,5:Chinese:Hunan Province:zhuzhou:tianyuan:tianyua$5:India:Hunan Province:zhuzhou:tianyuan:tianyua,31-01-2016$:,2300,845
+100053,1210,4ROM size,29-11-2015,,$$,6:Chinese:Hubei Province:wuhan:hongshan:hongshan$6:India:New Delhi:wuhan:hongshan:hongshan,01-02-2016$:,1210,1655
+100054,1689,8ROM size,29-11-2015,,$$,1:Chinese:Guangdong Province:shenzhen:longgang:matishan$1:India:Guangdong Province:shenzhen:longgang:matishan,02-02-2016$:,1689,1368
+100055,2823,2ROM size,29-11-2015,,$$,7:Chinese:Hubei Province:yichang:yichang:yichang$7:India:New Delhi:delhi:delhi:delhi,$03-02-2016:03-02-2016,2823,1728
+100056,68,6ROM size,29-11-2015,,$$,6:Chinese:Hubei Province:wuhan:hongshan:hongshan$6:India:New Delhi:wuhan:hongshan:hongshan,$04-02-2016:04-02-2016,68,750
+100057,716,0ROM size,29-11-2015,,$$,5:Chinese:Hunan Province:zhuzhou:tianyuan:tianyua$5:India:Hunan Province:zhuzhou:tianyuan:tianyua,$05-02-2016:05-02-2016,716,2288
+100058,864,6ROM size,29-11-2015,,$$,2:Chinese:Guangdong Province:guangzhou:longhua:mingzhi$2:India:Guangdong Province:guangzhou:longhua:mingzhi,$06-02-2016:06-02-2016,864,2635
+100059,499,6ROM size,29-11-2015,,$$,1:Chinese:Guangdong Province:shenzhen:longgang:matishan$1:India:Guangdong Province:shenzhen:longgang:matishan,$07-02-2016:07-02-2016,499,1337
+10006,1429,3ROM size,29-11-2015,,$$,:Chinese:Guangdong Province:guangzhou:longhua:mingzhi$:India:Guangdong Province:guangzhou:longhua:mingzhi,$08-02-2016:08-02-2016,1429,2478
+100060,2176,2ROM size,29-11-2015,,$$,:Chinese:Hunan Province:xiangtan:xiangtan:jianshelu$:India:Hunan Province:xiangtan:xiangtan:jianshelu,$09-02-2016:09-02-2016,2176,538
+100061,2563,7ROM size,29-11-2015,,,3:Chinese:Hunan Province:changsha:yuhua:shazitang$3:India:Hunan Province:changsha:yuhua:shazitang,$10-02-2016:10-02-2016,2563,1407
+100062,2594,3ROM size,29-11-2015,,,7:Chinese:Hubei Province:yichang:yichang:yichang$7:India:New Delhi:delhi:delhi:delhi,$11-02-2016:11-02-2016,2594,2952
+100063,2142,1ROM size,29-11-2015,,,7:Chinese:Hubei Province:yichang:yichang:yichang$7:India:New Delhi:delhi:delhi:delhi,$12-02-2016:12-02-2016,2142,1226
+100064,138,0ROM size,29-11-2015,1AA100064$2BB100064,,5:Chinese:Hunan Province:zhuzhou:tianyuan:tianyua$5:India:Hunan Province:zhuzhou:tianyuan:tianyua,$13-02-2016:13-02-2016,138,865
+100065,1168,6ROM size,29-11-2015,1AA100065$2BB100065,,4:Chinese:Hunan Province:xiangtan:xiangtan:jianshelu$4:India:Hunan Province:xiangtan:xiangtan:jianshelu,$14-02-2016:14-02-2016,1168,901
+100066,2828,5ROM size,29-11-2015,1AA100066$2BB100066,,5:Chinese:Hunan Province:zhuzhou:tianyuan:tianyua$5:India:Hunan Province:zhuzhou:tianyuan:tianyua,$:,2828,1864
+100067,1160,0ROM size,29-11-2015,1AA100067$2BB100067,,6:Chinese:Hubei Province:wuhan:hongshan:hongshan$6:India:New Delhi:wuhan:hongshan:hongshan,$:,1160,572
+100068,1890,6ROM size,29-11-2015,1AA100068$2BB100068,,2:Chinese:Guangdong Province:guangzhou:longhua:mingzhi$2:India:Guangdong Province:guangzhou:longhua:mingzhi,$:,1890,412
+100069,1195,4ROM size,29-11-2015,1AA100069$2BB100069,,4:Chinese:Hunan Province:xiangtan:xiangtan:jianshelu$4:India:Hunan Province:xiangtan:xiangtan:jianshelu,$:,1195,1491
+10007,2797,9ROM size,29-11-2015,1AA10007$2BB10007,,4:Chinese:Hunan Province:xiangtan:xiangtan:jianshelu$4:India:Hunan Province:xiangtan:xiangtan:jianshelu,$:,2797,1350
+100070,44,5ROM size,29-11-2015,1AA100070$2BB100070,,2:Chinese:Guangdong Province:guangzhou:longhua:mingzhi$2:India:Guangdong Province:guangzhou:longhua:mingzhi,$:,44,1567
+100071,1683,6ROM size,29-11-2015,1AA100071$2BB100071,,2:Chinese:Guangdong Province:guangzhou:longhua:mingzhi$2:India:Guangdong Province:guangzhou:longhua:mingzhi,$:,1683,1973
+100072,1085,1ROM size,29-11-2015,1AA100072$2BB100072,,3:Chinese:Hunan Province:changsha:yuhua:shazitang$3:India:Hunan Province:changsha:yuhua:shazitang,22-02-2016$22-02-2016:22-02-2016,1085,448
+100073,776,7ROM size,29-11-2015,1AA100073$2BB100073,,5:Chinese:Hunan Province:zhuzhou:tianyuan:tianyua$5:India:Hunan Province:zhuzhou:tianyuan:tianyua,23-02-2016$23-02-2016:23-02-2016,776,2488
+100074,2074,9ROM size,29-11-2015,1AA100074$2BB100074,MAC262$MAC263$,6:Chinese:Hubei Province:wuhan:hongshan:hongshan$6:India:New Delhi:wuhan:hongshan:hongshan,24-02-2016$24-02-2016:24-02-2016,2074,907
+100075,1062,2ROM size,29-11-2015,1AA100075$2BB100075,MAC265$MAC266$,1:Chinese:Guangdong Province:shenzhen:longgang:matishan$1:India:Guangdong Province:shenzhen:longgang:matishan,25-02-2016$25-02-2016:25-02-2016,1062,2507
+100076,987,7ROM size,29-11-2015,1AA100076$2BB100076,MAC268$MAC269$,6:Chinese:Hubei Province:wuhan:hongshan:hongshan$6:India:New Delhi:wuhan:hongshan:hongshan,26-02-2016$26-02-2016:26-02-2016,987,732
+100077,2799,9ROM size,29-11-2015,1AA100077$2BB100077,MAC271$MAC272$,7:Chinese:Hubei Province:yichang:yichang:yichang$7:India:New Delhi:delhi:delhi:delhi,27-02-2016$27-02-2016:27-02-2016,2799,2077
+100078,2765,1ROM size,29-11-2015,1AA100078$2BB100078,MAC274$MAC275$,7:Chinese:Hubei Province:yichang:yichang:yichang$7:India:New Delhi:delhi:delhi:delhi,28-02-2016$28-02-2016:28-02-2016,2765,1434
+100079,2164,1ROM size,29-11-2015,1AA100079$2BB100079,MAC277$MAC278$,4:Chinese:Hunan Province:xiangtan:xiangtan:jianshelu$4:India:Hunan Province:xiangtan:xiangtan:jianshelu,29-02-2016$29-02-2016:29-02-2016,2164,1098
+10008,1624,6ROM size,29-11-2015,1AA10008$2BB10008,MAC280$MAC281$,1:Chinese:Guangdong Province:shenzhen:longgang:matishan$1:India:Guangdong Province:shenzhen:longgang:matishan,01-03-2016$01-03-2016:01-03-2016,1624,813
+100080,2355,1ROM size,29-11-2015,1AA100080$2BB100080,MAC283$MAC284$MAC285,1:Chinese:Guangdong Province:shenzhen:longgang:matishan$1:India:Guangdong Province:shenzhen:longgang:matishan,02-03-2016$02-03-2016:02-03-2016,2355,954
+100081,1650,6ROM size,29-11-2015,1AA100081$2BB100081,MAC286$MAC287$MAC288,1:Chinese:Guangdong Province:shenzhen:longgang:matishan$1:India:Guangdong Province:shenzhen:longgang:matishan,03-03-2016$03-03-2016:03-03-2016,1650,613
+100082,2761,3ROM size,29-11-2015,1AA100082$2BB100082,MAC289$MAC290$MAC291,4:Chinese:Hunan Province:xiangtan:xiangtan:jianshelu$4:India:Hunan Province:xiangtan:xiangtan:jianshelu,04-03-2016$04-03-2016:04-03-2016,2761,2348
+100083,1856,3ROM size,29-11-2015,1AA100083$2BB100083,MAC292$MAC293$MAC294,5:Chinese:Hunan Province:zhuzhou:tianyuan:tianyua$5:India:Hunan Province:zhuzhou:tianyuan:tianyua,05-03-2016$05-03-2016:05-03-2016,1856,2192
+100084,1841,7ROM size,29-11-2015,1AA100084$2BB100084,MAC295$MAC296$MAC297,2:Chinese:Guangdong Province:guangzhou:longhua:mingzhi$2:India:Guangdong Province:guangzhou:longhua:mingzhi,06-03-2016$06-03-2016:06-03-2016,1841,2826
+100085,1841,7ROM size,29-11-2015,1AA100084$2BB100084,MAC295$MAC296$MAC297,2:Chinese:Guangdong Province:guangzhou:longhua:mingzhi$2:India:Guangdong Province:guangzhou:longhua:mingzhi,06-03-2016$06-03-2016:06-03-2016,1841,2826

http://git-wip-us.apache.org/repos/asf/carbondata/blob/93a604dd/examples/spark2/src/main/resources/dataSample.csv
----------------------------------------------------------------------
diff --git a/examples/spark2/src/main/resources/dataSample.csv b/examples/spark2/src/main/resources/dataSample.csv
new file mode 100644
index 0000000..5696978
--- /dev/null
+++ b/examples/spark2/src/main/resources/dataSample.csv
@@ -0,0 +1,11 @@
+ID,date,country,name,phonetype,serialname,salary,floatField
+1,2015/7/23,china,aaa1,phone197,ASD69643,15000,2.34
+2,2015/7/24,china,aaa2,phone756,ASD42892,15001,2.34
+3,2015/7/25,china,aaa3,phone1904,ASD37014,15002,2.34
+4,2015/7/26,china,aaa4,phone2435,ASD66902,15003,2.34
+5,2015/7/27,china,aaa5,phone2441,ASD90633,15004,2.34
+6,2015/7/28,china,aaa6,phone294,ASD59961,15005,3.5
+7,2015/7/29,china,aaa7,phone610,ASD14875,15006,2.34
+8,2015/7/30,china,aaa8,phone1848,ASD57308,15007,2.34
+9,2015/7/18,china,aaa9,phone706,ASD86717,15008,2.34
+10,2015/7/19,usa,aaa10,phone685,ASD30505,15009,2.34
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/93a604dd/examples/spark2/src/main/resources/dimSample.csv
----------------------------------------------------------------------
diff --git a/examples/spark2/src/main/resources/dimSample.csv b/examples/spark2/src/main/resources/dimSample.csv
new file mode 100644
index 0000000..0c8f27a
--- /dev/null
+++ b/examples/spark2/src/main/resources/dimSample.csv
@@ -0,0 +1,21 @@
+id,name,city
+1,David,Beijing
+2,Mark,Paris
+3,Bill,NewYork
+4,Sara,Tokyo
+5,John,Beijing
+6,Michel,Chicago
+7,Robert,Houston
+8,Sunny,Boston
+9,Mary,Tokyo
+10,Edward,Paris
+11,James,Washington
+12,Maria,Berlin
+13,Adam,Athens
+14,Peter,Boston
+15,George,Paris
+16,Paul,Shanghai
+17,Lisa,Hangzhou
+18,Angel,Beijing
+19,Emily,Bangalore
+20,Kevin,Singapore
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/93a604dd/examples/spark2/src/main/resources/factSample.csv
----------------------------------------------------------------------
diff --git a/examples/spark2/src/main/resources/factSample.csv b/examples/spark2/src/main/resources/factSample.csv
new file mode 100644
index 0000000..9693156
--- /dev/null
+++ b/examples/spark2/src/main/resources/factSample.csv
@@ -0,0 +1,51 @@
+id,name,city,salary
+1,David,Beijing,15000
+1,David,Tokyo,20000
+1,David,Hangzhou,18000
+2,Mark,Paris,12000
+2,Mark,Boston,15000
+2,Mark,Chicago,18000
+3,Bill,NewYork,20000
+3,Bill,Boston,23000
+4,Sara,Tokyo,11000
+4,Sara,Paris,15000
+4,Sara,Chicago,21000
+4,Sara,Hangzhou,17000
+5,John,Beijing,15000
+5,John,Shanghai,16000
+6,Michel,Chicago,11000
+6,Michel,Boston,12000
+6,Michel,Tokyo,11000
+8,Sunny,Boston,14000
+8,Sunny,Beijing,22000
+8,Sunny,Tokyo,20000
+9,Mary,Tokyo,13000
+9,Mary,NewYork,18000
+9,Mary,Paris,16000
+9,Mary,Washington,20000
+9,Mary,Boston,17000
+10,Edward,Paris,20000
+10,Edward,Beijing,12000
+10,Edward,Berlin,15000
+11,James,Washington,16000
+12,Maria,Berlin,15000
+12,Maria,Beijing,16000
+13,Adam,Athens,21000
+13,Adam,Berlin,18000
+13,Adam,Hangzhou,17000
+14,Peter,Boston,20000
+14,Peter,Berlin,21000
+14,Peter,Shanghai,18000
+15,George,Paris,17000
+15,George,Tokyo,12000
+15,George,Beijing,15000
+15,George,Berlin,18000
+16,Paul,Shanghai,22000
+16,Paul,Tokyo,19000
+16,Paul,Paris,24000
+16,Paul,Hangzhou,22000
+18,Angel,Beijing,22000
+18,Angel,NewYork,25000
+18,Angel,Tokyo,22000
+20,Kevin,Singapore,18000
+20,Kevin,Bangalore,16000
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/93a604dd/examples/spark2/src/main/scala/org/apache/carbondata/examples/AllDictionaryExample.scala
----------------------------------------------------------------------
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/AllDictionaryExample.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/AllDictionaryExample.scala
new file mode 100644
index 0000000..2f337f4
--- /dev/null
+++ b/examples/spark2/src/main/scala/org/apache/carbondata/examples/AllDictionaryExample.scala
@@ -0,0 +1,73 @@
+/*
+ * 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 org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.CarbonProperties
+
+object AllDictionaryExample {
+
+  def main(args: Array[String]) {
+    val spark = ExampleUtils.createCarbonSession("AllDictionaryExample")
+    val testData = ExampleUtils.currentPath + "/src/main/resources/dataSample.csv"
+    val csvHeader = "ID,date,country,name,phonetype,serialname,salary"
+    val dictCol = "|date|country|name|phonetype|serialname|"
+    val allDictFile = ExampleUtils.currentPath + "/src/main/resources/data.dictionary"
+    // extract all dictionary files from source data
+    AllDictionaryUtil.extractDictionary(spark.sparkContext,
+      testData, allDictFile, csvHeader, dictCol)
+    // Specify date format based on raw data
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_DATE_FORMAT, "yyyy/MM/dd")
+
+    spark.sql("DROP TABLE IF EXISTS t3")
+
+    spark.sql(
+      s"""
+         | CREATE TABLE IF NOT EXISTS t3(
+         | ID Int,
+         | date Date,
+         | country String,
+         | name String,
+         | phonetype String,
+         | serialname String,
+         | salary Int,
+         | floatField float
+         | ) STORED BY 'carbondata'
+       """.stripMargin)
+
+    spark.sql(s"""
+           LOAD DATA LOCAL INPATH '$testData' into table t3
+           options('ALL_DICTIONARY_PATH'='$allDictFile', 'SINGLE_PASS'='true')
+           """)
+
+    spark.sql("""
+           SELECT * FROM t3
+           """).show()
+
+    spark.sql("""
+           SELECT * FROM t3 where floatField=3.5
+           """).show()
+
+    spark.sql("DROP TABLE IF EXISTS t3")
+
+    // clean local dictionary files
+    AllDictionaryUtil.cleanDictionary(allDictFile)
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/93a604dd/examples/spark2/src/main/scala/org/apache/carbondata/examples/AllDictionaryUtil.scala
----------------------------------------------------------------------
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/AllDictionaryUtil.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/AllDictionaryUtil.scala
new file mode 100644
index 0000000..50d26aa
--- /dev/null
+++ b/examples/spark2/src/main/scala/org/apache/carbondata/examples/AllDictionaryUtil.scala
@@ -0,0 +1,109 @@
+/*
+ * 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.DataOutputStream
+
+import scala.collection.mutable.{ArrayBuffer, HashSet}
+
+import org.apache.spark.SparkContext
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.datastore.impl.FileFactory
+
+object AllDictionaryUtil {
+  private val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+  def extractDictionary(sc: SparkContext,
+                        srcData: String,
+                        outputPath: String,
+                        fileHeader: String,
+                        dictCol: String): Unit = {
+    val fileHeaderArr = fileHeader.split(",")
+    val isDictCol = new Array[Boolean](fileHeaderArr.length)
+    for (i <- 0 until fileHeaderArr.length) {
+      if (dictCol.contains("|" + fileHeaderArr(i).toLowerCase() + "|")) {
+        isDictCol(i) = true
+      } else {
+        isDictCol(i) = false
+      }
+    }
+    val dictionaryRdd = sc.textFile(srcData).flatMap(x => {
+      val tokens = x.split(",")
+      val result = new ArrayBuffer[(Int, String)]()
+      for (i <- 0 until isDictCol.length) {
+        if (isDictCol(i)) {
+          try {
+            result += ((i, tokens(i)))
+          } catch {
+            case ex: ArrayIndexOutOfBoundsException =>
+              LOGGER.error("Read a bad record: " + x)
+          }
+        }
+      }
+      result
+    }).groupByKey().flatMap(x => {
+      val distinctValues = new HashSet[(Int, String)]()
+      for (value <- x._2) {
+        distinctValues.add(x._1, value)
+      }
+      distinctValues
+    })
+    val dictionaryValues = dictionaryRdd.map(x => x._1 + "," + x._2).collect()
+    saveToFile(dictionaryValues, outputPath)
+  }
+
+  def cleanDictionary(outputPath: String): Unit = {
+    try {
+      val fileType = FileFactory.getFileType(outputPath)
+      val file = FileFactory.getCarbonFile(outputPath, fileType)
+      if (file.exists()) {
+        file.delete()
+      }
+    } catch {
+      case ex: Exception =>
+        LOGGER.error("Clean dictionary catching exception:" + ex)
+    }
+  }
+
+  def saveToFile(contents: Array[String], outputPath: String): Unit = {
+    var writer: DataOutputStream = null
+    try {
+      val fileType = FileFactory.getFileType(outputPath)
+      val file = FileFactory.getCarbonFile(outputPath, fileType)
+      if (!file.exists()) {
+        file.createNewFile()
+      }
+      writer = FileFactory.getDataOutputStream(outputPath, fileType)
+      for (content <- contents) {
+        writer.writeBytes(content + "\n")
+      }
+    } catch {
+      case ex: Exception =>
+        LOGGER.error("Save dictionary to file catching exception:" + ex)
+    } finally {
+      if (writer != null) {
+        try {
+          writer.close()
+        } catch {
+          case ex: Exception =>
+            LOGGER.error("Close output stream catching exception:" + ex)
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/93a604dd/examples/spark2/src/main/scala/org/apache/carbondata/examples/AlluxioExample.scala
----------------------------------------------------------------------
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/AlluxioExample.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/AlluxioExample.scala
new file mode 100644
index 0000000..6c183a5
--- /dev/null
+++ b/examples/spark2/src/main/scala/org/apache/carbondata/examples/AlluxioExample.scala
@@ -0,0 +1,64 @@
+/*
+ * 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 org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.util.CarbonProperties
+
+/**
+ * configure alluxio:
+ * 1.start alluxio
+ * 2.upload the jar :"/alluxio_path/core/client/target/
+ * alluxio-core-client-YOUR-VERSION-jar-with-dependencies.jar"
+ * 3.Get more detail at:http://www.alluxio.org/docs/master/en/Running-Spark-on-Alluxio.html
+ */
+
+object AlluxioExample {
+  def main(args: Array[String]) {
+    val spark = ExampleUtils.createCarbonSession("AlluxioExample")
+    spark.sparkContext.hadoopConfiguration.set("fs.alluxio.impl", "alluxio.hadoop.FileSystem")
+    FileFactory.getConfiguration.set("fs.alluxio.impl", "alluxio.hadoop.FileSystem")
+
+    // Specify date format based on raw data
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_DATE_FORMAT, "yyyy/MM/dd")
+
+    spark.sql("DROP TABLE IF EXISTS t3")
+
+    spark.sql("""
+           CREATE TABLE IF NOT EXISTS t3
+           (ID Int, date Date, country String,
+           name String, phonetype String, serialname String, salary Int)
+           STORED BY 'carbondata'
+           """)
+
+    spark.sql(s"""
+           LOAD DATA LOCAL INPATH 'alluxio://localhost:19998/data.csv' into table t3
+           """)
+
+    spark.sql("""
+           SELECT country, count(salary) AS amount
+           FROM t3
+           WHERE country IN ('china','france')
+           GROUP BY country
+           """).show()
+
+    spark.sql("DROP TABLE IF EXISTS t3")
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/93a604dd/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonSessionExample.scala
----------------------------------------------------------------------
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonSessionExample.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonSessionExample.scala
index c0429b5..a42b366 100644
--- a/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonSessionExample.scala
+++ b/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonSessionExample.scala
@@ -19,34 +19,10 @@ package org.apache.carbondata.examples
 
 import java.io.File
 
-import org.apache.spark.sql.SparkSession
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.util.CarbonProperties
-
 object CarbonSessionExample {
 
   def main(args: Array[String]) {
-    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"
-
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/MM/dd HH:mm:ss")
-      .addProperty(CarbonCommonConstants.CARBON_DATE_FORMAT, "yyyy/MM/dd")
-      .addProperty(CarbonCommonConstants.ENABLE_UNSAFE_COLUMN_PAGE_LOADING, "true")
-
-    import org.apache.spark.sql.CarbonSession._
-    val spark = SparkSession
-      .builder()
-      .master("local")
-      .appName("CarbonSessionExample")
-      .config("spark.sql.warehouse.dir", warehouse)
-      .config("spark.driver.host", "localhost")
-      .getOrCreateCarbonSession(storeLocation)
-
+    val spark = ExampleUtils.createCarbonSession("CarbonSessionExample")
     spark.sparkContext.setLogLevel("WARN")
 
     spark.sql("DROP TABLE IF EXISTS carbon_table")
@@ -71,6 +47,8 @@ object CarbonSessionExample {
          | TBLPROPERTIES('SORT_COLUMNS'='', 'DICTIONARY_INCLUDE'='dateField, charField')
        """.stripMargin)
 
+    val rootPath = new File(this.getClass.getResource("/").getPath
+                            + "../../../..").getCanonicalPath
     val path = s"$rootPath/examples/spark2/src/main/resources/data.csv"
 
     // scalastyle:off

http://git-wip-us.apache.org/repos/asf/carbondata/blob/93a604dd/examples/spark2/src/main/scala/org/apache/carbondata/examples/CaseClassDataFrameAPIExample.scala
----------------------------------------------------------------------
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/CaseClassDataFrameAPIExample.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/CaseClassDataFrameAPIExample.scala
new file mode 100644
index 0000000..c926817
--- /dev/null
+++ b/examples/spark2/src/main/scala/org/apache/carbondata/examples/CaseClassDataFrameAPIExample.scala
@@ -0,0 +1,47 @@
+/*
+ * 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 org.apache.spark.rdd.RDD
+import org.apache.spark.sql.{DataFrame, SaveMode}
+
+case class People(name: String, occupation: String, id: Int)
+
+object CaseClassDataFrameAPIExample {
+
+  def main(args: Array[String]) {
+    val spark = ExampleUtils.createCarbonSession("CaseClassDataFrameAPIExample")
+
+    val people = List(People("sangeeta", "engineer", 1), People("pallavi", "consultant", 2))
+    val peopleRDD: RDD[People] = spark.sparkContext.parallelize(people)
+    import spark.implicits._
+    val peopleDF: DataFrame = peopleRDD.toDF("name", "occupation", "id")
+
+    // writing data to carbon table
+    peopleDF.write
+      .format("carbondata")
+      .option("tableName", "carbon2")
+      .option("compress", "true")
+      .mode(SaveMode.Overwrite)
+      .save()
+
+    spark.sql("SELECT * FROM carbon2").show()
+
+    spark.sql("DROP TABLE IF EXISTS carbon2")
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/93a604dd/examples/spark2/src/main/scala/org/apache/carbondata/examples/DataFrameAPIExample.scala
----------------------------------------------------------------------
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/DataFrameAPIExample.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/DataFrameAPIExample.scala
new file mode 100644
index 0000000..7a7e74a
--- /dev/null
+++ b/examples/spark2/src/main/scala/org/apache/carbondata/examples/DataFrameAPIExample.scala
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.examples
+
+// scalastyle:off println
+object DataFrameAPIExample {
+
+  def main(args: Array[String]) {
+    val spark = ExampleUtils.createCarbonSession("DataFrameAPIExample")
+    ExampleUtils.writeSampleCarbonFile(spark, "carbon1", 1000)
+
+    import spark.implicits._
+
+    // use datasource api to read
+    val in = spark.read
+      .format("carbondata")
+      .option("tableName", "carbon1")
+      .load()
+    var count = in.where($"c3" > 500).select($"*").count()
+    println(s"count after 1 load: $count")
+
+    // append new data, query answer should be 1000
+    ExampleUtils.appendSampleCarbonFile(spark, "carbon1")
+    count = in.where($"c3" > 500).select($"*").count()
+    println(s"count after 2 load: $count")
+
+    // use SQL to read
+    spark.sql("SELECT c1, count(c3) FROM carbon1 where c3 > 500 group by c1 limit 10").show
+
+    // delete carbondata file
+    ExampleUtils.cleanSampleCarbonFile(spark, "carbon1")
+  }
+}
+// scalastyle:on println

http://git-wip-us.apache.org/repos/asf/carbondata/blob/93a604dd/examples/spark2/src/main/scala/org/apache/carbondata/examples/ExampleUtils.scala
----------------------------------------------------------------------
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/ExampleUtils.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/ExampleUtils.scala
new file mode 100644
index 0000000..a48ed6a
--- /dev/null
+++ b/examples/spark2/src/main/scala/org/apache/carbondata/examples/ExampleUtils.scala
@@ -0,0 +1,107 @@
+/*
+ * 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
+
+import org.apache.spark.sql.{SaveMode, SparkSession}
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.CarbonProperties
+
+// scalastyle:off println
+
+object ExampleUtils {
+
+  def currentPath: String = new File(this.getClass.getResource("/").getPath + "../../")
+      .getCanonicalPath
+  val storeLocation: String = currentPath + "/target/store"
+
+  def createCarbonSession(appName: String): SparkSession = {
+    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"
+
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/MM/dd HH:mm:ss")
+      .addProperty(CarbonCommonConstants.CARBON_DATE_FORMAT, "yyyy/MM/dd")
+      .addProperty(CarbonCommonConstants.ENABLE_UNSAFE_COLUMN_PAGE_LOADING, "true")
+
+    import org.apache.spark.sql.CarbonSession._
+    val spark = SparkSession
+      .builder()
+      .master("local")
+      .appName("CarbonSessionExample")
+      .config("spark.sql.warehouse.dir", warehouse)
+      .config("spark.driver.host", "localhost")
+      .getOrCreateCarbonSession(storeLocation)
+    spark.sparkContext.setLogLevel("WARN")
+    spark
+  }
+
+  /**
+   * This func will write a sample CarbonData file containing following schema:
+   * c1: String, c2: String, c3: Double
+   * Returns table path
+   */
+  def writeSampleCarbonFile(spark: SparkSession, tableName: String, numRows: Int = 1000): String = {
+    spark.sql(s"DROP TABLE IF EXISTS $tableName")
+    writeDataframe(spark, tableName, numRows, SaveMode.Overwrite)
+    s"$storeLocation/default/$tableName"
+  }
+
+  /**
+   * This func will append data to the CarbonData file
+   * Returns table path
+   */
+  def appendSampleCarbonFile(
+      spark: SparkSession, tableName: String, numRows: Int = 1000): String = {
+    writeDataframe(spark, tableName, numRows, SaveMode.Append)
+    s"$storeLocation/default/$tableName"
+  }
+
+  /**
+   * create a new dataframe and write to CarbonData file, based on save mode
+   */
+  private def writeDataframe(
+      spark: SparkSession, tableName: String, numRows: Int, mode: SaveMode): Unit = {
+    // use CarbonContext to write CarbonData files
+    import spark.implicits._
+    val sc = spark.sparkContext
+    val df = sc.parallelize(1 to numRows, 2)
+        .map(x => ("a", "b", x))
+        .toDF("c1", "c2", "c3")
+
+    // save dataframe directl to carbon file without tempCSV
+    df.write
+      .format("carbondata")
+      .option("tableName", tableName)
+      .option("compress", "true")
+      .option("tempCSV", "false")
+      .mode(mode)
+      .save()
+  }
+
+  def cleanSampleCarbonFile(spark: SparkSession, tableName: String): Unit = {
+    spark.sql(s"DROP TABLE IF EXISTS $tableName")
+  }
+}
+// scalastyle:on println
+

http://git-wip-us.apache.org/repos/asf/carbondata/blob/93a604dd/examples/spark2/src/main/scala/org/apache/carbondata/examples/HadoopFileExample.scala
----------------------------------------------------------------------
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/HadoopFileExample.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/HadoopFileExample.scala
new file mode 100644
index 0000000..d14469a
--- /dev/null
+++ b/examples/spark2/src/main/scala/org/apache/carbondata/examples/HadoopFileExample.scala
@@ -0,0 +1,52 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration
+
+import org.apache.carbondata.hadoop.{CarbonInputFormat, CarbonProjection}
+
+// scalastyle:off println
+object HadoopFileExample {
+
+  def main(args: Array[String]): Unit = {
+    val spark = ExampleUtils.createCarbonSession("HadoopFileExample")
+    ExampleUtils.writeSampleCarbonFile(spark, "carbon1")
+
+    // read two columns
+    val projection = new CarbonProjection
+    projection.addColumn("c1")  // column c1
+    projection.addColumn("c3")  // column c3
+    val conf = new Configuration()
+    CarbonInputFormat.setColumnProjection(conf, projection)
+
+    val sc = spark.sparkContext
+    val input = sc.newAPIHadoopFile(s"${ExampleUtils.storeLocation}/default/carbon1",
+      classOf[CarbonInputFormat[Array[Object]]],
+      classOf[Void],
+      classOf[Array[Object]],
+      conf)
+    val result = input.map(x => x._2.toList).collect
+    result.foreach(x => println(x.mkString(", ")))
+
+    // delete carbondata file
+    ExampleUtils.cleanSampleCarbonFile(spark, "carbon1")
+  }
+}
+// scalastyle:on println
+


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

Posted by ja...@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/pre-aggregate
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")
+  }
+}


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

Posted by ja...@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/7e1d40c0
Tree: http://git-wip-us.apache.org/repos/asf/carbondata/tree/7e1d40c0
Diff: http://git-wip-us.apache.org/repos/asf/carbondata/diff/7e1d40c0

Branch: refs/heads/pre-aggregate
Commit: 7e1d40c0b8a04a469335c987b6e46a072b1e3b7b
Parents: 2235ed6
Author: kumarvishal <ku...@gmail.com>
Authored: Sun Oct 15 18:05:55 2017 +0530
Committer: Jacky Li <ja...@qq.com>
Committed: Wed Nov 8 11:25:21 2017 +0800

----------------------------------------------------------------------
 .../table/column/ParentColumnTableRelation.java |  71 +++
 .../ThriftWrapperSchemaConverterImplTest.java   |  28 +-
 .../preaggregate/TestPreAggCreateCommand.scala  | 148 +++++++
 .../carbondata/spark/util/CommonUtil.scala      |   9 +
 .../spark/sql/catalyst/CarbonDDLSqlParser.scala |  22 +-
 .../command/carbonTableSchemaCommon.scala       | 170 +++++---
 .../command/management/LoadTableCommand.scala   |   2 +-
 .../CreatePreAggregateTableCommand.scala        | 136 ++++++
 .../preaaggregate/PreAggregateUtil.scala        | 431 +++++++++++++++++++
 .../schema/AlterTableRenameTableCommand.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 |  37 +-
 .../org/apache/spark/util/AlterTableUtil.scala  |  10 +-
 16 files changed, 1109 insertions(+), 93 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/7e1d40c0/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/7e1d40c0/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..0fddc25 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
@@ -31,6 +31,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;
@@ -84,6 +85,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<>();
@@ -421,6 +423,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>() {
@@ -483,6 +486,8 @@ public class ThriftWrapperSchemaConverterImplTest {
       @Mock public String getColumnReferenceId() {
         return "1";
       }
+
+      @Mock public String getAggFunction() {return "" ;}
     };
 
     org.apache.carbondata.format.ColumnSchema actualResult =
@@ -496,7 +501,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;
@@ -571,7 +576,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;
@@ -645,6 +650,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;
@@ -718,7 +724,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;
@@ -792,6 +798,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() {
@@ -866,6 +873,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;
@@ -926,6 +934,10 @@ public class ThriftWrapperSchemaConverterImplTest {
       @Mock public String getColumnReferenceId() {
         return "1";
       }
+
+      @Mock public String getAggFunction() {
+        return "";
+      }
     };
 
     ColumnSchema wrapperColumnSchema = new ColumnSchema();
@@ -940,6 +952,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() {
@@ -1014,7 +1027,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;
@@ -1088,6 +1101,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() {
@@ -1168,6 +1182,7 @@ public class ThriftWrapperSchemaConverterImplTest {
             encoders,
             true);
     thriftColumnSchema.setSchemaOrdinal(1);
+    thriftColumnSchema.setAggregate_function("");
 
     new MockUp<ColumnSchema>() {
       @Mock public List<Encoding> getEncodingList() {
@@ -1318,7 +1333,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);
@@ -1506,6 +1521,8 @@ public class ThriftWrapperSchemaConverterImplTest {
       @Mock public String getColumnReferenceId() {
         return "1";
       }
+
+      @Mock public String getAggFunction() { return "";}
     };
 
     new MockUp<TableInfo>() {
@@ -1542,6 +1559,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/7e1d40c0/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/7e1d40c0/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/7e1d40c0/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..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.
    *
@@ -242,11 +232,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) =>
@@ -1070,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/7e1d40c0/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..3fa26ab 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/7e1d40c0/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..7aed442 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
@@ -344,7 +344,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/7e1d40c0/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/7e1d40c0/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/7e1d40c0/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
index af361d5..2be3c7c 100644
--- 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
@@ -105,7 +105,7 @@ private[sql] case class AlterTableRenameTableCommand(
       }
       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/7e1d40c0/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..6c29f83 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.format
 import org.apache.carbondata.format.{SchemaEvolutionEntry, TableInfo}
 import org.apache.carbondata.processing.merger.TableMeta
 import org.apache.carbondata.spark.util.CarbonSparkUtil
@@ -110,6 +111,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)
@@ -205,7 +222,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,
@@ -242,7 +259,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)
@@ -262,7 +279,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/7e1d40c0/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/7e1d40c0/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/7e1d40c0/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..e015c4f 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/7e1d40c0/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..1518c15 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,16 +18,17 @@ package org.apache.spark.sql.parser
 
 import scala.collection.mutable
 
-import org.apache.spark.sql.{CarbonEnv, SparkSession}
+import org.apache.spark.sql.{CarbonEnv, DataFrame, Dataset, SparkSession}
 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._
+import org.apache.spark.sql.execution.command.preaaggregate.{CreatePreAggregateTableCommand, PreAggregateUtil}
 import org.apache.spark.sql.internal.{SQLConf, VariableSubstitution}
 
-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
@@ -38,7 +39,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
 
@@ -120,7 +122,7 @@ class CarbonSqlAstBuilder(conf: SQLConf) extends SparkSqlAstBuilder(conf) {
 
       val tableProperties = mutable.Map[String, String]()
       properties.foreach{property => tableProperties.put(property._1, property._2)}
-
+      val isAggTable = tableProperties.get("parent").isDefined
       // validate partition clause
       if (partitionerFields.nonEmpty) {
         if (!CommonUtil.validatePartitionColumns(tableProperties, partitionerFields)) {
@@ -130,10 +132,20 @@ class CarbonSqlAstBuilder(conf: SQLConf) extends SparkSqlAstBuilder(conf) {
         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)
+          badPartCols.map("\"" + _ + "\"").mkString("[", ",", "]"), ctx)
         }
       }
-      val fields = parser.getFields(cols ++ partitionByStructFields)
+      var fields = parser.getFields(cols ++ 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 options = new CarbonOption(properties)
       // validate tblProperties
       val bucketFields = parser.getBucketFields(tableProperties, fields, options)
@@ -149,7 +161,14 @@ class CarbonSqlAstBuilder(conf: SQLConf) extends SparkSqlAstBuilder(conf) {
         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/7e1d40c0/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..a82823d 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
@@ -132,7 +132,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,
@@ -207,7 +207,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)
       }
@@ -239,7 +239,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)
     }
   }
@@ -274,7 +274,7 @@ object AlterTableUtil {
         }
       }
       metastore
-        .revertTableSchema(carbonTable.getCarbonTableIdentifier,
+        .revertTableSchemaInAlterFailure(carbonTable.getCarbonTableIdentifier,
           thriftTable, carbonTable.getAbsoluteTableIdentifier.getTablePath)(sparkSession)
     }
   }
@@ -312,7 +312,7 @@ object AlterTableUtil {
         }
       }
       metastore
-        .revertTableSchema(carbonTable.getCarbonTableIdentifier,
+        .revertTableSchemaInAlterFailure(carbonTable.getCarbonTableIdentifier,
           thriftTable, carbonTable.getAbsoluteTableIdentifier.getTablePath)(sparkSession)
     }
   }


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

Posted by ja...@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/pre-aggregate
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 {


[09/24] carbondata git commit: [CARBONDATA-1617] Merging carbonindex files within segment

Posted by ja...@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/pre-aggregate
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] = ???
 


[15/24] carbondata git commit: [CARBONDATA-1667] Remove direct load related code

Posted by ja...@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/pre-aggregate
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;


[05/24] carbondata git commit: [CARBONDATA-1599] Update pull request checklist

Posted by ja...@apache.org.
[CARBONDATA-1599] Update pull request checklist

Remove title and description items from pull request checklist, add them to How to contribute document

This closes #1451


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

Branch: refs/heads/pre-aggregate
Commit: 8e6b0a7ce9ec9b5c2e9dc8e288f148d567224e28
Parents: 5936e7f
Author: chenliang613 <ch...@apache.org>
Authored: Mon Oct 30 01:00:56 2017 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Tue Oct 31 12:16:02 2017 +0530

----------------------------------------------------------------------
 .github/PULL_REQUEST_TEMPLATE.md               | 18 ++++--------------
 docs/How-to-contribute-to-Apache-CarbonData.md |  8 ++++++--
 2 files changed, 10 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/8e6b0a7c/.github/PULL_REQUEST_TEMPLATE.md
----------------------------------------------------------------------
diff --git a/.github/PULL_REQUEST_TEMPLATE.md b/.github/PULL_REQUEST_TEMPLATE.md
index 0d9f60c..d80ff46 100644
--- a/.github/PULL_REQUEST_TEMPLATE.md
+++ b/.github/PULL_REQUEST_TEMPLATE.md
@@ -1,13 +1,5 @@
-Be sure to do all of the following to help us incorporate your contribution
-quickly and easily:
-
- - [ ] Make sure the PR title is formatted like:
-   `[CARBONDATA-<Jira issue #>] Description of pull request`
-   
- - [ ] Make sure to add PR description including
-        
-        - the root cause/problem statement
-        - What is the implemented solution
+Be sure to do all of the following checklist to help us incorporate 
+your contribution quickly and easily:
 
  - [ ] Any interfaces changed?
  
@@ -16,13 +8,11 @@ quickly and easily:
  - [ ] Document update required?
 
  - [ ] Testing done
- 
         Please provide details on 
         - Whether new unit test cases have been added or why no new tests are required?
         - How it is tested? Please attach test report.
         - Is it a performance related change? Please attach the performance test report.
         - Any additional information to help reviewers in testing this change.
-         
+       
  - [ ] For large changes, please consider breaking it into sub-tasks under an umbrella JIRA. 
-                 
----
+

http://git-wip-us.apache.org/repos/asf/carbondata/blob/8e6b0a7c/docs/How-to-contribute-to-Apache-CarbonData.md
----------------------------------------------------------------------
diff --git a/docs/How-to-contribute-to-Apache-CarbonData.md b/docs/How-to-contribute-to-Apache-CarbonData.md
index 9048021..f57642d 100644
--- a/docs/How-to-contribute-to-Apache-CarbonData.md
+++ b/docs/How-to-contribute-to-Apache-CarbonData.md
@@ -152,9 +152,13 @@ $ git rebase -i origin/master
 Navigate to the CarbonData GitHub mirror to create a pull request. The title of the pull request
 should be strictly in the following format:
 ```
-[CARBONDATA-issue number>] Title of the pull request
+[CARBONDATA-JiraTicketNumer][FeatureName] Description of pull request    
 ```
-Please include a descriptive pull request message to help make the reviewer’s job easier.
+Please include a descriptive pull request message to help make the reviewer’s job easier:
+```
+ - The root cause/problem statement
+ - What is the implemented solution
+ ```
 
 If you know a good committer to review your pull request, please make a comment like the following.
 If not, don’t worry, a committer will pick it up.


[03/24] carbondata git commit: [CARBONDATA-1653] Rename aggType to measureDataType

Posted by ja...@apache.org.
[CARBONDATA-1653] Rename aggType to measureDataType

This closes #1444


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

Branch: refs/heads/pre-aggregate
Commit: 4d70a2118fb26936e0b7eee9a0ce2147cb0501c9
Parents: 93a604d
Author: Jacky Li <ja...@qq.com>
Authored: Sat Oct 28 16:26:38 2017 +0530
Committer: chenliang613 <ch...@apache.org>
Committed: Sat Oct 28 23:39:26 2017 +0530

----------------------------------------------------------------------
 .../scan/executor/impl/AbstractQueryExecutor.java  |  4 ++--
 .../core/scan/executor/util/QueryUtil.java         | 17 -----------------
 .../core/scan/executor/util/QueryUtilTest.java     |  9 ---------
 .../merger/CompactionResultSortProcessor.java      |  8 +-------
 .../sort/sortdata/IntermediateFileMerger.java      |  6 +++---
 .../sort/sortdata/SortTempFileChunkHolder.java     | 12 ++++++------
 .../store/CarbonFactDataHandlerModel.java          |  6 +++---
 7 files changed, 15 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d70a211/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 b2c1a2b..56ed575 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
@@ -161,10 +161,10 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
         .addStatistics(QueryStatisticsConstants.LOAD_BLOCKS_EXECUTOR, System.currentTimeMillis());
     queryProperties.queryStatisticsRecorder.recordStatistics(queryStatistic);
     // calculating the total number of aggeragted columns
-    int aggTypeCount = queryModel.getQueryMeasures().size();
+    int measureCount = queryModel.getQueryMeasures().size();
 
     int currentIndex = 0;
-    DataType[] dataTypes = new DataType[aggTypeCount];
+    DataType[] dataTypes = new DataType[measureCount];
 
     for (QueryMeasure carbonMeasure : queryModel.getQueryMeasures()) {
       // adding the data type and aggregation type of all the measure this

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d70a211/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 59387b3..6aab8a0 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
@@ -678,23 +678,6 @@ public class QueryUtil {
   }
 
   /**
-   * below method will be used to get the actual type aggregator
-   *
-   * @param aggType
-   * @return index in aggrgetor
-   */
-  public static int[] getActualTypeIndex(List<String> aggType) {
-    List<Integer> indexList = new ArrayList<Integer>();
-    for (int i = 0; i < aggType.size(); i++) {
-      if (!CarbonCommonConstants.SUM.equals(aggType.get(i)) && !CarbonCommonConstants.AVERAGE
-          .equals(aggType.get(i))) {
-        indexList.add(i);
-      }
-    }
-    return ArrayUtils.toPrimitive(indexList.toArray(new Integer[indexList.size()]));
-  }
-
-  /**
    * Below method will be used to get the key structure for the column group
    *
    * @param segmentProperties      segment properties

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d70a211/core/src/test/java/org/apache/carbondata/core/scan/executor/util/QueryUtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/executor/util/QueryUtilTest.java b/core/src/test/java/org/apache/carbondata/core/scan/executor/util/QueryUtilTest.java
index 23504d3..b764569 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/executor/util/QueryUtilTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/executor/util/QueryUtilTest.java
@@ -221,15 +221,6 @@ public class QueryUtilTest extends TestCase {
     assertArrayEquals(expectedValue, actualValue);
   }
 
-  @Test public void testGetActualTypeIndex() {
-    List<String> dummyList = new ArrayList<>();
-    dummyList.add("test1");
-    dummyList.add("test2");
-    int[] actualValue = QueryUtil.getActualTypeIndex(dummyList);
-    int[] expectedValue = { 0, 1 };
-    assertArrayEquals(expectedValue, actualValue);
-  }
-
   @AfterClass public void tearDown() {
     segmentProperties = null;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d70a211/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 0e20ef5..a361f3a 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
@@ -155,7 +155,7 @@ public class CompactionResultSortProcessor extends AbstractResultProcessor {
     try {
       initTempStoreLocation();
       initSortDataRows();
-      initAggType();
+      dataTypes = CarbonDataProcessorUtil.initDataType(carbonTable, tableName, measureCount);
       processResult(resultIteratorList);
       // After delete command, if no records are fetched from one split,
       // below steps are not required to be initialized.
@@ -406,10 +406,4 @@ public class CompactionResultSortProcessor extends AbstractResultProcessor {
             true, false);
   }
 
-  /**
-   * initialise aggregation type for measures for their storage format
-   */
-  private void initAggType() {
-    dataTypes = CarbonDataProcessorUtil.initDataType(carbonTable, tableName, measureCount);
-  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d70a211/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 fa50f1c..ebc811c 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
@@ -326,7 +326,7 @@ public class IntermediateFileMerger implements Callable<Void> {
       return;
     }
     try {
-      DataType[] aggType = mergerParameters.getMeasureDataType();
+      DataType[] measureDataType = mergerParameters.getMeasureDataType();
       int[] mdkArray = (int[]) row[0];
       byte[][] nonDictArray = (byte[][]) row[1];
       int mdkIndex = 0;
@@ -346,7 +346,7 @@ public class IntermediateFileMerger implements Callable<Void> {
       for (int counter = 0; counter < mergerParameters.getMeasureColCount(); counter++) {
         if (null != NonDictionaryUtil.getMeasure(fieldIndex, row)) {
           stream.write((byte) 1);
-          DataType dataType = aggType[counter];
+          DataType dataType = measureDataType[counter];
           if (dataType == DataTypes.BOOLEAN) {
             stream.writeBoolean((boolean)NonDictionaryUtil.getMeasure(fieldIndex, row));
           } else if (dataType == DataTypes.SHORT) {
@@ -362,7 +362,7 @@ public class IntermediateFileMerger implements Callable<Void> {
             stream.writeInt(bigDecimalInBytes.length);
             stream.write(bigDecimalInBytes);
           } else {
-            throw new IllegalArgumentException("unsupported data type:" + aggType[counter]);
+            throw new IllegalArgumentException("unsupported data type:" + measureDataType[counter]);
           }
         } else {
           stream.write((byte) 0);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d70a211/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 afa4cd7..91bc83c 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
@@ -129,7 +129,7 @@ public class SortTempFileChunkHolder implements Comparable<SortTempFileChunkHold
 
   private int noDictionaryCount;
 
-  private DataType[] aggType;
+  private DataType[] measureDataTypes;
 
   /**
    * to store whether dimension is of dictionary type or not
@@ -150,11 +150,11 @@ public class SortTempFileChunkHolder implements Comparable<SortTempFileChunkHold
    * @param measureCount
    * @param fileBufferSize
    * @param noDictionaryCount
-   * @param aggType
+   * @param measureDataTypes
    * @param isNoDictionaryDimensionColumn
    */
   public SortTempFileChunkHolder(File tempFile, int dimensionCount, int complexDimensionCount,
-      int measureCount, int fileBufferSize, int noDictionaryCount, DataType[] aggType,
+      int measureCount, int fileBufferSize, int noDictionaryCount, DataType[] measureDataTypes,
       boolean[] isNoDictionaryDimensionColumn, boolean[] isNoDictionarySortColumn,
       String tableName) {
     // set temp file
@@ -170,7 +170,7 @@ public class SortTempFileChunkHolder implements Comparable<SortTempFileChunkHold
     this.fileBufferSize = fileBufferSize;
     this.executorService = Executors
         .newFixedThreadPool(1, new CarbonThreadFactory("SafeSortTempChunkHolderPool:" + tableName));
-    this.aggType = aggType;
+    this.measureDataTypes = measureDataTypes;
 
     this.isNoDictionaryDimensionColumn = isNoDictionaryDimensionColumn;
     this.isNoDictionarySortColumn = isNoDictionarySortColumn;
@@ -344,7 +344,7 @@ public class SortTempFileChunkHolder implements Comparable<SortTempFileChunkHold
       // read measure values
       for (int i = 0; i < this.measureCount; i++) {
         if (stream.readByte() == 1) {
-          DataType dataType = aggType[i];
+          DataType dataType = measureDataTypes[i];
           if (dataType == DataTypes.BOOLEAN) {
             measures[index++] = stream.readBoolean();
           } else if (dataType == DataTypes.SHORT) {
@@ -361,7 +361,7 @@ public class SortTempFileChunkHolder implements Comparable<SortTempFileChunkHold
             stream.readFully(buff);
             measures[index++] = DataTypeUtil.byteToBigDecimal(buff);
           } else {
-            throw new IllegalArgumentException("unsupported data type:" + aggType[i]);
+            throw new IllegalArgumentException("unsupported data type:" + measureDataTypes[i]);
           }
         } else {
           measures[index++] = null;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d70a211/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java
index 2c346b2..b6ac19d 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java
@@ -299,12 +299,12 @@ public class CarbonFactDataHandlerModel {
     Map<Integer, GenericDataType> complexIndexMap =
         new HashMap<Integer, GenericDataType>(segmentProperties.getComplexDimensions().size());
     carbonFactDataHandlerModel.setComplexIndexMap(complexIndexMap);
-    DataType[] aggType = new DataType[segmentProperties.getMeasures().size()];
+    DataType[] measureDataTypes = new DataType[segmentProperties.getMeasures().size()];
     int i = 0;
     for (CarbonMeasure msr : segmentProperties.getMeasures()) {
-      aggType[i++] = msr.getDataType();
+      measureDataTypes[i++] = msr.getDataType();
     }
-    carbonFactDataHandlerModel.setMeasureDataType(aggType);
+    carbonFactDataHandlerModel.setMeasureDataType(measureDataTypes);
     String carbonDataDirectoryPath = CarbonDataProcessorUtil
         .checkAndCreateCarbonStoreLocation(loadModel.getStorePath(), loadModel.getDatabaseName(),
             tableName, loadModel.getPartitionId(), loadModel.getSegmentId());


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

Posted by ja...@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/pre-aggregate
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;
         }


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

Posted by ja...@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/pre-aggregate
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()
   }


[08/24] carbondata git commit: [CARBONDATA-1617] Merging carbonindex files within segment

Posted by ja...@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
 }