[CARBONDATA-3609][CARBONDATA-3610] Remove preaggregate and timeseries datamap

Remove preaggregate and timeseries datamap as the same features are
handled in MV.

This closes #3522
diff --git a/README.md b/README.md
index da5b547..a34784d 100644
--- a/README.md
+++ b/README.md
@@ -58,8 +58,6 @@
 * [CarbonData DataMap Management](https://github.com/apache/carbondata/blob/master/docs/datamap/datamap-management.md) 
  * [CarbonData BloomFilter DataMap](https://github.com/apache/carbondata/blob/master/docs/datamap/bloomfilter-datamap-guide.md) 
  * [CarbonData Lucene DataMap](https://github.com/apache/carbondata/blob/master/docs/datamap/lucene-datamap-guide.md) 
- * [CarbonData Pre-aggregate DataMap](https://github.com/apache/carbondata/blob/master/docs/datamap/preaggregate-datamap-guide.md) 
- * [CarbonData Timeseries DataMap](https://github.com/apache/carbondata/blob/master/docs/datamap/timeseries-datamap-guide.md) 
  * [CarbonData MV DataMap](https://github.com/apache/carbondata/blob/master/docs/datamap/mv-datamap-guide.md)
 * [SDK Guide](https://github.com/apache/carbondata/blob/master/docs/sdk-guide.md) 
 * [C++ SDK Guide](https://github.com/apache/carbondata/blob/master/docs/csdk-guide.md)
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 2e3e7d3..1424e1a 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
@@ -1513,15 +1513,8 @@
   public static final String CARBON_DATAMAP_VISIBLE = "carbon.datamap.visible.";
 
   /**
-   * Fetch and validate the segments.
-   * Used for aggregate table load as segment validation is not required.
-   */
-  @CarbonProperty(dynamicConfigurable = true)
-  public static final String VALIDATE_CARBON_INPUT_SEGMENTS = "validate.carbon.input.segments.";
-
-  /**
    * Whether load/insert command is fired internally or by the user.
-   * Used to block load/insert on pre-aggregate if fired by user
+   * Used to block load/insert on MV if fired by user
    */
   @CarbonProperty
   public static final String IS_INTERNAL_LOAD_CALL = "is.internal.load.call";
diff --git a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstantsInternal.java b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstantsInternal.java
index 5111d25..3ea6494 100644
--- a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstantsInternal.java
+++ b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstantsInternal.java
@@ -23,8 +23,6 @@
  */
 public interface CarbonCommonConstantsInternal {
 
-  String QUERY_ON_PRE_AGG_STREAMING = "carbon.query.on.preagg.streaming.";
-
   String ROW_COUNT = "rowCount";
 
 }
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapProvider.java b/core/src/main/java/org/apache/carbondata/core/datamap/DataMapProvider.java
index e58b61b..107e63f 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapProvider.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/DataMapProvider.java
@@ -64,8 +64,6 @@
  *
  * <br>Currently CarbonData supports following provider:
  * <ol>
- *   <li> preaggregate: pre-aggregate table of single table </li>
- *   <li> timeseries: pre-aggregate table based on time dimension of the table </li>
  *   <li> lucene: index backed by Apache Lucene </li>
  *   <li> bloomfilter: index backed by Bloom Filter </li>
  * </ol>
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java b/core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java
index 78c3290..c01a1ad 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java
@@ -52,7 +52,6 @@
 import org.apache.carbondata.core.util.ThreadLocalSessionInfo;
 
 import static org.apache.carbondata.core.metadata.schema.datamap.DataMapClassProvider.MV;
-import static org.apache.carbondata.core.metadata.schema.datamap.DataMapClassProvider.PREAGGREGATE;
 
 import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.lang.StringUtils;
@@ -200,10 +199,6 @@
       String dataBaseName =  relationIdentifier.getDatabaseName();
       String tableId = relationIdentifier.getTableId();
       String providerName = dataMapSchema.getProviderName();
-      // if the preaggregate datamap,not be modified the schema
-      if (providerName.equalsIgnoreCase(PREAGGREGATE.toString())) {
-        continue;
-      }
       // if the mv datamap,not be modified the relationIdentifier
       if (!providerName.equalsIgnoreCase(MV.toString())) {
         RelationIdentifier newRelationIdentifier = new RelationIdentifier(dataBaseName,
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapUtil.java b/core/src/main/java/org/apache/carbondata/core/datamap/DataMapUtil.java
index 1a1200c..ca56962 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/DataMapUtil.java
@@ -261,7 +261,7 @@
       CarbonTable carbonTable, Configuration configuration) throws IOException {
     SegmentStatusManager ssm =
         new SegmentStatusManager(carbonTable.getAbsoluteTableIdentifier(), configuration);
-    return ssm.getValidAndInvalidSegments(carbonTable.isChildTable());
+    return ssm.getValidAndInvalidSegments(carbonTable.isChildTableForMV());
   }
 
   /**
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java b/core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java
index caf1328..87b68c0 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java
@@ -915,7 +915,7 @@
     if (toBeDeleteSegments.size() > 0 || toBeUpdatedSegments.size() > 0) {
       Set<Segment> segmentSet = new HashSet<>(
           new SegmentStatusManager(carbonTable.getAbsoluteTableIdentifier())
-              .getValidAndInvalidSegments(carbonTable.isChildTable()).getValidSegments());
+              .getValidAndInvalidSegments(carbonTable.isChildTableForMV()).getValidSegments());
       CarbonUpdateUtil.updateTableMetadataStatus(segmentSet, carbonTable, uniqueId, true,
           Segment.toSegmentList(toBeDeleteSegments, null),
           Segment.toSegmentList(toBeUpdatedSegments, null), uuid);
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/datamap/DataMapClassProvider.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/datamap/DataMapClassProvider.java
index 33cf25a..4095857 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/datamap/DataMapClassProvider.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/datamap/DataMapClassProvider.java
@@ -27,8 +27,6 @@
  */
 
 public enum DataMapClassProvider {
-  PREAGGREGATE("org.apache.carbondata.core.datamap.AggregateDataMap", "preaggregate"),
-  TIMESERIES("org.apache.carbondata.core.datamap.TimeSeriesDataMap", "timeseries"),
   LUCENE("org.apache.carbondata.datamap.lucene.LuceneFineGrainDataMapFactory", "lucene"),
   BLOOMFILTER("org.apache.carbondata.datamap.bloom.BloomCoarseGrainDataMapFactory", "bloomfilter"),
   MV("org.apache.carbondata.core.datamap.MVDataMap", "mv");
@@ -63,11 +61,7 @@
   }
 
   public static DataMapClassProvider getDataMapProviderOnName(String dataMapShortname) {
-    if (TIMESERIES.isEqual(dataMapShortname)) {
-      return TIMESERIES;
-    } else if (PREAGGREGATE.isEqual(dataMapShortname)) {
-      return PREAGGREGATE;
-    } else if (LUCENE.isEqual(dataMapShortname)) {
+    if (LUCENE.isEqual(dataMapShortname)) {
       return LUCENE;
     } else if (BLOOMFILTER.isEqual(dataMapShortname)) {
       return BLOOMFILTER;
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/AggregationDataMapSchema.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/AggregationDataMapSchema.java
deleted file mode 100644
index 20174c5..0000000
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/AggregationDataMapSchema.java
+++ /dev/null
@@ -1,385 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.metadata.schema.table;
-
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.carbondata.core.metadata.schema.datamap.DataMapClassProvider;
-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.preagg.TimeSeriesFunctionEnum;
-
-/**
- * data map schema class for pre aggregation
- */
-public class AggregationDataMapSchema extends DataMapSchema {
-
-  private static final long serialVersionUID = 5900935117929888412L;
-  /**
-   * map of parent column name to set of child column column without
-   * aggregation function
-   */
-  private Map<String, Set<ColumnSchema>> parentToNonAggChildMapping;
-
-  /**
-   * map of parent column name to set of child columns column with
-   * aggregation function
-   */
-  private Map<String, Set<ColumnSchema>> parentToAggChildMapping;
-
-  /**
-   * map of parent column name to set of aggregation function applied in
-   * in parent column
-   */
-  private Map<String, Set<String>> parentColumnToAggregationsMapping;
-
-  /**
-   * whether its a timeseries data map
-   */
-  private boolean isTimeseriesDataMap;
-
-  /**
-   * below ordinal will be used during sorting the data map
-   * to support rollup for loading
-   */
-  private int ordinal = Integer.MAX_VALUE;
-
-  // Dont remove transient otherwise serialization for carbonTable will fail using
-  // JavaSerialization in spark.
-  private transient Set aggExpToColumnMapping;
-
-  AggregationDataMapSchema(String dataMapName, String className) {
-    super(dataMapName, className);
-  }
-
-  public void setChildSchema(TableSchema childSchema) {
-    super.setChildSchema(childSchema);
-    List<ColumnSchema> listOfColumns = getChildSchema().getListOfColumns();
-    fillNonAggFunctionColumns(listOfColumns);
-    fillAggFunctionColumns(listOfColumns);
-    fillParentNameToAggregationMapping(listOfColumns);
-  }
-
-  /**
-   * Below method will be used to get the columns on which aggregate function
-   * and time series function is not applied
-   * @param columnName
-   *                parent column name
-   * @return child column schema
-   */
-  public ColumnSchema getNonAggNonTimeseriesChildColBasedByParent(String columnName) {
-    Set<ColumnSchema> columnSchemas = parentToNonAggChildMapping.get(columnName);
-    if (null != columnSchemas) {
-      Iterator<ColumnSchema> iterator = columnSchemas.iterator();
-      while (iterator.hasNext()) {
-        ColumnSchema next = iterator.next();
-        if ((null == next.getAggFunction() || next.getAggFunction().isEmpty()) && null == next
-            .getTimeSeriesFunction() || next.getTimeSeriesFunction().isEmpty()) {
-          return next;
-        }
-      }
-    }
-    return null;
-  }
-
-  /**
-   * Below method will be used to get the columns on which aggregate function is not applied
-   * @param columnName
-   *                parent column name
-   * @return child column schema
-   */
-  public ColumnSchema getNonAggChildColBasedByParent(String columnName) {
-    Set<ColumnSchema> columnSchemas = parentToNonAggChildMapping.get(columnName);
-    if (null != columnSchemas) {
-      Iterator<ColumnSchema> iterator = columnSchemas.iterator();
-      while (iterator.hasNext()) {
-        ColumnSchema next = iterator.next();
-        if ((null == next.getAggFunction() || next.getAggFunction().isEmpty())) {
-          return next;
-        }
-      }
-    }
-    return null;
-  }
-
-  /**
-   * Below method will be used to get the columns on which aggregate function is not applied
-   *
-   * @param columnName parent column name
-   * @return child column schema
-   */
-  public ColumnSchema getTimeseriesChildColBasedByParent(String columnName,
-      String timeseriesFunction) {
-    Set<ColumnSchema> columnSchemas = parentToNonAggChildMapping.get(columnName);
-    if (null != columnSchemas) {
-      Iterator<ColumnSchema> iterator = columnSchemas.iterator();
-      while (iterator.hasNext()) {
-        ColumnSchema next = iterator.next();
-        if (timeseriesFunction.equals(next.getTimeSeriesFunction())) {
-          return next;
-        }
-      }
-    }
-    return null;
-  }
-
-  /**
-   * Below method will be used to get the column schema based on parent column name
-   * @param columName
-   *                parent column name
-   * @return child column schema
-   */
-  public ColumnSchema getChildColByParentColName(String columName) {
-    List<ColumnSchema> listOfColumns = childSchema.getListOfColumns();
-    for (ColumnSchema columnSchema : listOfColumns) {
-      List<ParentColumnTableRelation> parentColumnTableRelations =
-          columnSchema.getParentColumnTableRelations();
-      if (null != parentColumnTableRelations && parentColumnTableRelations.size() == 1
-          && parentColumnTableRelations.get(0).getColumnName().equalsIgnoreCase(columName) &&
-          columnSchema.getColumnName().endsWith(columName)) {
-        return columnSchema;
-      }
-    }
-    return null;
-  }
-
-  /**
-   * Below method will be used to get the child column schema based on parent name and aggregate
-   * function applied on column
-   * @param columnName
-   *                  parent column name
-   * @param aggFunction
-   *                  aggregate function applied
-   * @return child column schema
-   */
-  public ColumnSchema getAggChildColByParent(String columnName,
-      String aggFunction) {
-    Set<ColumnSchema> columnSchemas = parentToAggChildMapping.get(columnName);
-    if (null != columnSchemas) {
-      Iterator<ColumnSchema> iterator = columnSchemas.iterator();
-      while (iterator.hasNext()) {
-        ColumnSchema next = iterator.next();
-        if (null != next.getAggFunction() && next.getAggFunction().equalsIgnoreCase(aggFunction)) {
-          return next;
-        }
-      }
-    }
-    return null;
-  }
-
-  /**
-   * Below method will be used to get the column schema based on parent column name
-   * @param columName
-   *                parent column name
-   * @param timeseriesFunction
-   *                timeseries function applied on column
-   * @return child column schema
-   */
-  public ColumnSchema getTimeseriesChildColByParent(String columName, String timeseriesFunction) {
-    List<ColumnSchema> listOfColumns = childSchema.getListOfColumns();
-    for (ColumnSchema columnSchema : listOfColumns) {
-      List<ParentColumnTableRelation> parentColumnTableRelations =
-          columnSchema.getParentColumnTableRelations();
-      if (null != parentColumnTableRelations && parentColumnTableRelations.size() == 1
-          && parentColumnTableRelations.get(0).getColumnName().equalsIgnoreCase(columName)
-          && timeseriesFunction.equalsIgnoreCase(columnSchema.getTimeSeriesFunction())) {
-        return columnSchema;
-      }
-    }
-    return null;
-  }
-
-  /**
-   * Method to prepare mapping of parent to list of aggregation function applied on that column
-   * @param listOfColumns
-   *        child column schema list
-   */
-  private void fillParentNameToAggregationMapping(List<ColumnSchema> listOfColumns) {
-    parentColumnToAggregationsMapping = new HashMap<>();
-    for (ColumnSchema column : listOfColumns) {
-      if (null != column.getAggFunction() && !column.getAggFunction().isEmpty()) {
-        List<ParentColumnTableRelation> parentColumnTableRelations =
-            column.getParentColumnTableRelations();
-        if (null != parentColumnTableRelations && parentColumnTableRelations.size() == 1) {
-          String columnName = column.getParentColumnTableRelations().get(0).getColumnName();
-          Set<String> aggFunctions = parentColumnToAggregationsMapping.get(columnName);
-          if (null == aggFunctions) {
-            aggFunctions = new HashSet<>();
-            parentColumnToAggregationsMapping.put(columnName, aggFunctions);
-          }
-          aggFunctions.add(column.getAggFunction());
-        }
-      }
-    }
-  }
-
-  /**
-   * Below method will be used prepare mapping between parent column to non aggregation function
-   * columns
-   * @param listOfColumns
-   *                    list of child columns
-   */
-  private void fillNonAggFunctionColumns(List<ColumnSchema> listOfColumns) {
-    parentToNonAggChildMapping = new HashMap<>();
-    for (ColumnSchema column : listOfColumns) {
-      if (!isTimeseriesDataMap) {
-        isTimeseriesDataMap =
-            null != column.getTimeSeriesFunction() && !column.getTimeSeriesFunction().isEmpty();
-        if (isTimeseriesDataMap) {
-          this.ordinal =
-              TimeSeriesFunctionEnum.valueOf(column.getTimeSeriesFunction().toUpperCase())
-                  .getOrdinal();
-        }
-      }
-      if (null == column.getAggFunction() || column.getAggFunction().isEmpty()) {
-        fillMappingDetails(column, parentToNonAggChildMapping);
-      }
-    }
-  }
-
-  private void fillMappingDetails(ColumnSchema column,
-      Map<String, Set<ColumnSchema>> map) {
-    List<ParentColumnTableRelation> parentColumnTableRelations =
-        column.getParentColumnTableRelations();
-    if (null != parentColumnTableRelations && parentColumnTableRelations.size() == 1) {
-      String columnName = column.getParentColumnTableRelations().get(0).getColumnName();
-      Set<ColumnSchema> columnSchemas = map.get(columnName);
-      if (null == columnSchemas) {
-        columnSchemas = new HashSet<>();
-        map.put(columnName, columnSchemas);
-      }
-      columnSchemas.add(column);
-    }
-  }
-
-  /**
-   * Below method will be used to fill parent to list of aggregation column mapping
-   * @param listOfColumns
-   *        list of child columns
-   */
-  private void fillAggFunctionColumns(List<ColumnSchema> listOfColumns) {
-    parentToAggChildMapping = new HashMap<>();
-    for (ColumnSchema column : listOfColumns) {
-      if (null != column.getAggFunction() && !column.getAggFunction().isEmpty()) {
-        fillMappingDetails(column, parentToAggChildMapping);
-      }
-    }
-  }
-
-  public boolean isTimeseriesDataMap() {
-    return isTimeseriesDataMap;
-  }
-
-  /**
-   * Below method is to support rollup during loading the data in pre aggregate table
-   * In case of timeseries year level table data loading can be done using month level table or any
-   * time series level below year level for example day,hour minute, second.
-   * @TODO need to handle for pre aggregate table without timeseries
-   *
-   * @param aggregationDataMapSchema
-   * @return whether aggregation data map can be selected or not
-   */
-  public boolean canSelectForRollup(AggregationDataMapSchema aggregationDataMapSchema) {
-    List<ColumnSchema> listOfColumns = childSchema.getListOfColumns();
-    for (ColumnSchema column : listOfColumns) {
-      List<ParentColumnTableRelation> parentColumnTableRelations =
-          column.getParentColumnTableRelations();
-      //@TODO handle scenario when aggregate datamap columns is derive from multiple column
-      // which is not supported currently
-      if (null != parentColumnTableRelations && parentColumnTableRelations.size() == 1) {
-        if (null != column.getAggFunction() && !column.getAggFunction().isEmpty()) {
-          if (null == aggregationDataMapSchema
-              .getAggChildColByParent(parentColumnTableRelations.get(0).getColumnName(),
-                  column.getAggFunction())) {
-            return false;
-          }
-        } else {
-          if (null == aggregationDataMapSchema.getNonAggChildColBasedByParent(
-              parentColumnTableRelations.get(0).getColumnName())) {
-            return false;
-          }
-        }
-      } else {
-        // in case of any expression one column can be derived from multiple column
-        // in that case we cannot do rollup so hit the maintable
-        return false;
-      }
-    }
-    return true;
-  }
-
-  public int getOrdinal() {
-    return ordinal;
-  }
-
-  /**
-   * Below method will be used to get the aggregation column based on index
-   * It will return the first aggregation column found based on index
-   * @param searchStartIndex
-   *  start index
-   * @param sortedColumnSchema
-   * list of sorted table columns
-   * @return found column list
-   *
-   */
-  public ColumnSchema getAggColumnBasedOnIndex(int searchStartIndex,
-      List<ColumnSchema> sortedColumnSchema) {
-    ColumnSchema columnSchema = null;
-    for (int i = searchStartIndex; i < sortedColumnSchema.size(); i++) {
-      if (!sortedColumnSchema.get(i).getAggFunction().isEmpty()) {
-        columnSchema = sortedColumnSchema.get(i);
-        break;
-      }
-    }
-    return columnSchema;
-  }
-
-  public synchronized Set getAggExpToColumnMapping() {
-    return aggExpToColumnMapping;
-  }
-
-  public synchronized void setAggExpToColumnMapping(Set aggExpToColumnMapping) {
-    if (null == this.aggExpToColumnMapping) {
-      this.aggExpToColumnMapping = aggExpToColumnMapping;
-    }
-  }
-
-  public DataMapClassProvider getProvider() {
-    return isTimeseriesDataMap ?
-        DataMapClassProvider.TIMESERIES : DataMapClassProvider.PREAGGREGATE;
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) return true;
-    if (o == null || getClass() != o.getClass()) return false;
-    if (!super.equals(o)) return false;
-    AggregationDataMapSchema that = (AggregationDataMapSchema) o;
-    return that == this;
-  }
-
-  @Override
-  public int hashCode() {
-    return super.hashCode();
-  }
-}
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 9b9b756..86ce9ba 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
@@ -861,21 +861,6 @@
     return streaming != null && streaming.equalsIgnoreCase("source");
   }
 
-  /**
-   * whether this table has aggregation DataMap or not
-   */
-  public boolean hasAggregationDataMap() {
-    List<DataMapSchema> dataMapSchemaList = tableInfo.getDataMapSchemaList();
-    if (dataMapSchemaList != null && !dataMapSchemaList.isEmpty()) {
-      for (DataMapSchema dataMapSchema : dataMapSchemaList) {
-        if (dataMapSchema instanceof AggregationDataMapSchema) {
-          return true;
-        }
-      }
-    }
-    return false;
-  }
-
   public int getDimensionOrdinalMax() {
     return dimensionOrdinalMax;
   }
@@ -894,12 +879,7 @@
     return null;
   }
 
-  public boolean isChildDataMap() {
-    return null != tableInfo.getParentRelationIdentifiers() && !tableInfo
-        .getParentRelationIdentifiers().isEmpty();
-  }
-
-  public boolean isChildTable() {
+  public boolean isChildTableForMV() {
     return null != tableInfo.getFactTable().getTableProperties()
         .get(CarbonCommonConstants.PARENT_TABLES) && !tableInfo.getFactTable().getTableProperties()
         .get(CarbonCommonConstants.PARENT_TABLES).isEmpty();
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchema.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchema.java
index 69ff837..ac5ce19 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchema.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchema.java
@@ -171,9 +171,7 @@
    * @return
    */
   public boolean isIndexDataMap() {
-    if (providerName.equalsIgnoreCase(DataMapClassProvider.PREAGGREGATE.getShortName()) ||
-        providerName.equalsIgnoreCase(DataMapClassProvider.TIMESERIES.getShortName()) ||
-        providerName.equalsIgnoreCase(DataMapClassProvider.MV.getShortName()) ||
+    if (providerName.equalsIgnoreCase(DataMapClassProvider.MV.getShortName()) ||
         ctasQuery != null) {
       return false;
     } else {
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchemaFactory.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchemaFactory.java
index bd79e08..d7abc4b 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchemaFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchemaFactory.java
@@ -17,8 +17,6 @@
 
 package org.apache.carbondata.core.metadata.schema.table;
 
-import org.apache.carbondata.core.metadata.schema.datamap.DataMapClassProvider;
-
 public class DataMapSchemaFactory {
   public static final DataMapSchemaFactory INSTANCE = new DataMapSchemaFactory();
 
@@ -29,14 +27,6 @@
    * @return data map schema
    */
   public DataMapSchema getDataMapSchema(String dataMapName, String providerName) {
-    if (providerName.equalsIgnoreCase(DataMapClassProvider.PREAGGREGATE.toString()) || providerName
-        .equalsIgnoreCase(DataMapClassProvider.PREAGGREGATE.getClassName())) {
-      return new AggregationDataMapSchema(dataMapName, providerName);
-    } else if (providerName.equalsIgnoreCase(DataMapClassProvider.TIMESERIES.toString())
-        || providerName.equalsIgnoreCase(DataMapClassProvider.TIMESERIES.getClassName())) {
-      return new AggregationDataMapSchema(dataMapName, providerName);
-    } else {
-      return new DataMapSchema(dataMapName, providerName);
-    }
+    return new DataMapSchema(dataMapName, providerName);
   }
 }
diff --git a/core/src/main/java/org/apache/carbondata/core/preagg/AggregateQueryPlan.java b/core/src/main/java/org/apache/carbondata/core/preagg/AggregateQueryPlan.java
deleted file mode 100644
index 4ff89f8..0000000
--- a/core/src/main/java/org/apache/carbondata/core/preagg/AggregateQueryPlan.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.preagg;
-
-import java.util.List;
-
-/**
- * class to maintain the query plan to select the data map tables
- */
-public class AggregateQueryPlan {
-
-  /**
-   * List of projection columns
-   */
-  private List<QueryColumn> projectionColumn;
-
-  /**
-   * list of filter columns
-   */
-  private List<QueryColumn> filterColumns;
-
-  public AggregateQueryPlan(List<QueryColumn> projectionColumn, List<QueryColumn> filterColumns) {
-    this.projectionColumn = projectionColumn;
-    this.filterColumns = filterColumns;
-  }
-
-  public List<QueryColumn> getProjectionColumn() {
-    return projectionColumn;
-  }
-
-  public List<QueryColumn> getFilterColumns() {
-    return filterColumns;
-  }
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/preagg/AggregateTableSelector.java b/core/src/main/java/org/apache/carbondata/core/preagg/AggregateTableSelector.java
deleted file mode 100644
index 5cdbad0..0000000
--- a/core/src/main/java/org/apache/carbondata/core/preagg/AggregateTableSelector.java
+++ /dev/null
@@ -1,140 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.preagg;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.carbondata.core.metadata.schema.table.AggregationDataMapSchema;
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
-import org.apache.carbondata.core.metadata.schema.table.DataMapSchema;
-import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
-
-/**
- * Below class will be used to select the aggregate table based
- * query plan. Rules for selecting the aggregate table is below:
- * 1. Select all aggregate table based on projection
- * 2. select aggregate table based on filter exp,
- * 2. select if aggregate tables based on aggregate columns
- */
-public class AggregateTableSelector {
-
-  /**
-   * current query plan
-   */
-  private AggregateQueryPlan aggregateQueryPlan;
-
-  /**
-   * parent table
-   */
-  private CarbonTable parentTable;
-
-  public AggregateTableSelector(AggregateQueryPlan aggregateQueryPlan, CarbonTable parentTable) {
-    this.aggregateQueryPlan = aggregateQueryPlan;
-    this.parentTable = parentTable;
-  }
-
-  /**
-   * Below method will be used to select pre aggregate tables based on query plan
-   * Rules for selecting the aggregate table is below:
-   * 1. Select all aggregate table based on projection
-   * 2. select aggregate table based on filter exp,
-   * 2. select if aggregate tables based on aggregate columns
-   *
-   * @return selected pre aggregate table schema
-   */
-  public List<DataMapSchema> selectPreAggDataMapSchema() {
-    List<QueryColumn> projectionColumn = aggregateQueryPlan.getProjectionColumn();
-    List<QueryColumn> filterColumns = aggregateQueryPlan.getFilterColumns();
-    List<DataMapSchema> dataMapSchemaList = parentTable.getTableInfo().getDataMapSchemaList();
-    List<DataMapSchema> selectedDataMapSchema = new ArrayList<>();
-    boolean isMatch;
-    // match projection columns
-    if (null != projectionColumn && !projectionColumn.isEmpty()) {
-      for (DataMapSchema dmSchema : dataMapSchemaList) {
-        AggregationDataMapSchema aggregationDataMapSchema = (AggregationDataMapSchema) dmSchema;
-        isMatch = true;
-        for (QueryColumn queryColumn : projectionColumn) {
-          ColumnSchema columnSchemaByParentName =
-              getColumnSchema(queryColumn, aggregationDataMapSchema);
-          if (null == columnSchemaByParentName) {
-            isMatch = false;
-            break;
-          }
-        }
-        if (isMatch) {
-          selectedDataMapSchema.add(dmSchema);
-        }
-      }
-      // if projection column is present but selected table list size is zero then
-      if (selectedDataMapSchema.size() == 0) {
-        return selectedDataMapSchema;
-      }
-    }
-
-    // match filter columns
-    if (null != filterColumns && !filterColumns.isEmpty()) {
-      List<DataMapSchema> dmSchemaToIterate =
-          selectedDataMapSchema.isEmpty() ? dataMapSchemaList : selectedDataMapSchema;
-      selectedDataMapSchema = new ArrayList<>();
-      for (DataMapSchema dmSchema : dmSchemaToIterate) {
-        isMatch = true;
-        for (QueryColumn queryColumn : filterColumns) {
-          AggregationDataMapSchema aggregationDataMapSchema = (AggregationDataMapSchema) dmSchema;
-          ColumnSchema columnSchemaByParentName =
-              getColumnSchema(queryColumn, aggregationDataMapSchema);
-          if (null == columnSchemaByParentName) {
-            isMatch = false;
-            break;
-          }
-        }
-        if (isMatch) {
-          selectedDataMapSchema.add(dmSchema);
-        }
-      }
-      // if filter column is present and selection size is zero then return
-      if (selectedDataMapSchema.size() == 0) {
-        return selectedDataMapSchema;
-      }
-    }
-    return selectedDataMapSchema;
-  }
-
-  /**
-   * Below method will be used to get column schema for projection and
-   * filter query column
-   *
-   * @param queryColumn              query column
-   * @param aggregationDataMapSchema selected data map schema
-   * @return column schema
-   */
-  private ColumnSchema getColumnSchema(QueryColumn queryColumn,
-      AggregationDataMapSchema aggregationDataMapSchema) {
-    ColumnSchema columnSchemaByParentName = null;
-    if (!queryColumn.getTimeseriesFunction().isEmpty()) {
-      columnSchemaByParentName = aggregationDataMapSchema
-          .getTimeseriesChildColBasedByParent(queryColumn.getColumnSchema().getColumnName(),
-              queryColumn.getTimeseriesFunction());
-    } else {
-      columnSchemaByParentName = aggregationDataMapSchema
-          .getNonAggNonTimeseriesChildColBasedByParent(
-              queryColumn.getColumnSchema().getColumnName());
-    }
-    return columnSchemaByParentName;
-  }
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/preagg/QueryColumn.java b/core/src/main/java/org/apache/carbondata/core/preagg/QueryColumn.java
deleted file mode 100644
index 64b4967..0000000
--- a/core/src/main/java/org/apache/carbondata/core/preagg/QueryColumn.java
+++ /dev/null
@@ -1,87 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.preagg;
-
-import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
-
-/**
- * column present in query
- */
-public class QueryColumn {
-
-  /**
-   * parent column schema
-   */
-  private ColumnSchema columnSchema;
-
-  /**
-   * is filter column
-   */
-  private boolean isFilterColumn;
-
-  /**
-   * timeseries udf applied on column
-   */
-  private String timeseriesFunction;
-
-  public QueryColumn(ColumnSchema columnSchema, boolean isFilterColumn, String timeseriesFunction) {
-    this.columnSchema = columnSchema;
-    this.isFilterColumn = isFilterColumn;
-    this.timeseriesFunction = timeseriesFunction;
-  }
-
-  public ColumnSchema getColumnSchema() {
-    return columnSchema;
-  }
-
-  public boolean isFilterColumn() {
-    return isFilterColumn;
-  }
-
-  public String getTimeseriesFunction() {
-    return timeseriesFunction;
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) {
-      return true;
-    }
-    if (o == null || getClass() != o.getClass()) {
-      return false;
-    }
-    QueryColumn that = (QueryColumn) o;
-    if (isFilterColumn != that.isFilterColumn) {
-      return false;
-    }
-    if (!columnSchema.equals(that.columnSchema)) {
-      return false;
-    }
-    return timeseriesFunction != null ?
-        timeseriesFunction.equals(that.timeseriesFunction) :
-        that.timeseriesFunction == null;
-  }
-
-  @Override
-  public int hashCode() {
-    int result = columnSchema.hashCode();
-    result = 31 * result + (timeseriesFunction != null ? timeseriesFunction.hashCode() : 0);
-    result = 31 * result + (isFilterColumn ? 1 : 0);
-    return result;
-  }
-}
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 df5a49e..d3fa33b 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
@@ -88,9 +88,7 @@
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.SchemaEvolution;
 import org.apache.carbondata.core.metadata.schema.SchemaEvolutionEntry;
-import org.apache.carbondata.core.metadata.schema.table.AggregationDataMapSchema;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
-import org.apache.carbondata.core.metadata.schema.table.DataMapSchema;
 import org.apache.carbondata.core.metadata.schema.table.RelationIdentifier;
 import org.apache.carbondata.core.metadata.schema.table.TableInfo;
 import org.apache.carbondata.core.metadata.schema.table.TableSchema;
@@ -2711,38 +2709,6 @@
   }
 
   /**
-   * Utility function to check whether table has timseries datamap or not
-   * @param carbonTable
-   * @return timeseries data map present
-   */
-  public static boolean hasTimeSeriesDataMap(CarbonTable carbonTable) {
-    List<DataMapSchema> dataMapSchemaList = carbonTable.getTableInfo().getDataMapSchemaList();
-    for (DataMapSchema dataMapSchema : dataMapSchemaList) {
-      if (dataMapSchema instanceof AggregationDataMapSchema) {
-        if (((AggregationDataMapSchema) dataMapSchema).isTimeseriesDataMap()) {
-          return true;
-        }
-      }
-    }
-    return false;
-  }
-
-  /**
-   * Utility function to check whether table has aggregation datamap or not
-   * @param carbonTable
-   * @return timeseries data map present
-   */
-  public static boolean hasAggregationDataMap(CarbonTable carbonTable) {
-    List<DataMapSchema> dataMapSchemaList = carbonTable.getTableInfo().getDataMapSchemaList();
-    for (DataMapSchema dataMapSchema : dataMapSchemaList) {
-      if (dataMapSchema instanceof AggregationDataMapSchema) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  /**
    * Convert the bytes to base64 encode string
    * @param bytes
    * @return
@@ -3199,7 +3165,7 @@
       SegmentStatusManager segmentStatusManager =
           new SegmentStatusManager(carbonTable.getAbsoluteTableIdentifier());
       SegmentStatusManager.ValidAndInvalidSegmentsInfo validAndInvalidSegmentsInfo =
-          segmentStatusManager.getValidAndInvalidSegments(carbonTable.isChildTable());
+          segmentStatusManager.getValidAndInvalidSegments(carbonTable.isChildTableForMV());
       List<Segment> validSegments = validAndInvalidSegmentsInfo.getValidSegments();
       if (validSegments.isEmpty()) {
         return carbonProperties.getFormatVersion();
@@ -3350,22 +3316,6 @@
     return UUID.randomUUID().toString();
   }
 
-  /**
-   * Below method will be used to get the datamap schema name from datamap table name
-   * it will split name based on character '_' and get the last name
-   * This is only for pre aggregate and timeseries tables
-   *
-   * @param tableName
-   * @return datamapschema name
-   */
-  public static String getDatamapNameFromTableName(String tableName) {
-    int i = tableName.lastIndexOf('_');
-    if (i != -1) {
-      return tableName.substring(i + 1, tableName.length());
-    }
-    return null;
-  }
-
   public static String getIndexServerTempPath(String tablePath, String queryId) {
     String tempFolderPath = CarbonProperties.getInstance()
         .getProperty(CarbonCommonConstants.CARBON_INDEX_SERVER_TEMP_PATH);
diff --git a/core/src/main/java/org/apache/carbondata/core/util/SessionParams.java b/core/src/main/java/org/apache/carbondata/core/util/SessionParams.java
index 151eef0..1769e16 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/SessionParams.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/SessionParams.java
@@ -26,7 +26,6 @@
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.cache.CacheProvider;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.constants.CarbonCommonConstantsInternal;
 import org.apache.carbondata.core.constants.CarbonLoadOptionConstants;
 import org.apache.carbondata.core.exception.InvalidConfigurationException;
 
@@ -219,12 +218,8 @@
           if (!isValid) {
             throw new InvalidConfigurationException("Invalid CARBON_INPUT_SEGMENT_IDs");
           }
-        } else if (key.startsWith(CarbonCommonConstants.VALIDATE_CARBON_INPUT_SEGMENTS)) {
-          isValid = true;
         } else if (key.equalsIgnoreCase(CarbonCommonConstants.SUPPORT_DIRECT_QUERY_ON_DATAMAP)) {
           isValid = true;
-        } else if (key.startsWith(CarbonCommonConstantsInternal.QUERY_ON_PRE_AGG_STREAMING)) {
-          isValid = true;
         } else if (key.startsWith(CarbonCommonConstants.CARBON_DATAMAP_VISIBLE)) {
           isValid = true;
         } else if (key.startsWith(CarbonCommonConstants.CARBON_LOAD_DATAMAPS_PARALLEL)) {
diff --git a/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMap.java b/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMap.java
index dfec812..882ee45 100644
--- a/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMap.java
+++ b/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMap.java
@@ -41,11 +41,9 @@
 import org.apache.carbondata.core.indexstore.Blocklet;
 import org.apache.carbondata.core.indexstore.PartitionSpec;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.metadata.CarbonMetadata;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
-import org.apache.carbondata.core.metadata.schema.table.RelationIdentifier;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
 import org.apache.carbondata.core.scan.expression.ColumnExpression;
 import org.apache.carbondata.core.scan.expression.Expression;
@@ -115,7 +113,6 @@
     for (CarbonColumn col : indexedColumn) {
       this.name2Col.put(col.getColName(), col);
     }
-    String parentTablePath = getAncestorTablePath(carbonTable);
 
     try {
       this.name2Converters = new HashMap<>(indexedColumn.size());
@@ -137,7 +134,7 @@
         dataField.setTimestampFormat(tsFormat);
         FieldConverter fieldConverter = FieldEncoderFactory.getInstance()
             .createFieldEncoder(dataField, absoluteTableIdentifier, i, nullFormat, null, false,
-                localCaches[i], false, parentTablePath, false);
+                localCaches[i], false, carbonTable.getTablePath(), false);
         this.name2Converters.put(indexedColumn.get(i).getColName(), fieldConverter);
       }
     } catch (IOException e) {
@@ -148,22 +145,6 @@
     this.badRecordLogHolder.setLogged(false);
   }
 
-  /**
-   * recursively find the ancestor's table path. This is used for dictionary scenario
-   * where preagg will use the dictionary of the parent table.
-   */
-  private String getAncestorTablePath(CarbonTable currentTable) {
-    if (!currentTable.isChildDataMap()) {
-      return currentTable.getTablePath();
-    }
-
-    RelationIdentifier parentIdentifier =
-        currentTable.getTableInfo().getParentRelationIdentifiers().get(0);
-    CarbonTable parentTable = CarbonMetadata.getInstance().getCarbonTable(
-        parentIdentifier.getDatabaseName(), parentIdentifier.getTableName());
-    return getAncestorTablePath(parentTable);
-  }
-
   @Override
   public List<Blocklet> prune(FilterResolverIntf filterExp, SegmentProperties segmentProperties,
       List<PartitionSpec> partitions) throws IOException {
diff --git a/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMapFactory.java b/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMapFactory.java
index e3902b3..4cb5385 100644
--- a/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMapFactory.java
+++ b/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMapFactory.java
@@ -374,7 +374,7 @@
         new SegmentStatusManager(getCarbonTable().getAbsoluteTableIdentifier());
     try {
       List<Segment> validSegments =
-          ssm.getValidAndInvalidSegments(getCarbonTable().isChildTable()).getValidSegments();
+          ssm.getValidAndInvalidSegments(getCarbonTable().isChildTableForMV()).getValidSegments();
       for (Segment segment : validSegments) {
         deleteDatamapData(segment);
       }
diff --git a/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneDataMapFactoryBase.java b/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneDataMapFactoryBase.java
index 52dfa09..247e856 100644
--- a/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneDataMapFactoryBase.java
+++ b/datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneDataMapFactoryBase.java
@@ -181,7 +181,7 @@
     SegmentStatusManager ssm = new SegmentStatusManager(tableIdentifier);
     try {
       List<Segment> validSegments =
-          ssm.getValidAndInvalidSegments(getCarbonTable().isChildTable()).getValidSegments();
+          ssm.getValidAndInvalidSegments(getCarbonTable().isChildTableForMV()).getValidSegments();
       for (Segment segment : validSegments) {
         deleteDatamapData(segment);
       }
diff --git a/datamap/mv/core/src/main/scala/org/apache/carbondata/mv/datamap/MVAnalyzerRule.scala b/datamap/mv/core/src/main/scala/org/apache/carbondata/mv/datamap/MVAnalyzerRule.scala
index 0ba6bfc..da38bae 100644
--- a/datamap/mv/core/src/main/scala/org/apache/carbondata/mv/datamap/MVAnalyzerRule.scala
+++ b/datamap/mv/core/src/main/scala/org/apache/carbondata/mv/datamap/MVAnalyzerRule.scala
@@ -18,7 +18,7 @@
 
 import scala.collection.JavaConverters._
 
-import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.{CarbonEnv, SparkSession}
 import org.apache.spark.sql.catalyst.analysis.{UnresolvedAlias, UnresolvedAttribute}
 import org.apache.spark.sql.catalyst.catalog.CatalogTable
 import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, ScalaUDF}
@@ -53,10 +53,10 @@
   override def apply(plan: LogicalPlan): LogicalPlan = {
     var needAnalysis = true
     plan.transformAllExpressions {
-      // first check if any preAgg scala function is applied it is present is in plan
-      // then call is from create preaggregate table class so no need to transform the query plan
+      // first check if any mv UDF is applied it is present is in plan
+      // then call is from create MV so no need to transform the query plan
       // TODO Add different UDF name
-      case al@Alias(udf: ScalaUDF, name) if name.equalsIgnoreCase("preAgg") =>
+      case al@Alias(udf: ScalaUDF, name) if name.equalsIgnoreCase(CarbonEnv.MV_SKIP_RULE_UDF) =>
         needAnalysis = false
         al
       // in case of query if any unresolve alias is present then wait for plan to be resolved
@@ -75,7 +75,7 @@
           if (p.isInstanceOf[UnresolvedAlias]) {
             false
           } else {
-            p.name.equals("preAggLoad") || p.name.equals("preAgg")
+            p.name.equals(CarbonEnv.MV_SKIP_RULE_UDF)
           }
         }
         if (isPreAggLoad) {
diff --git a/datamap/mv/core/src/main/scala/org/apache/carbondata/mv/datamap/MVDataMapProvider.scala b/datamap/mv/core/src/main/scala/org/apache/carbondata/mv/datamap/MVDataMapProvider.scala
index 6499a8f..2c2ad1e 100644
--- a/datamap/mv/core/src/main/scala/org/apache/carbondata/mv/datamap/MVDataMapProvider.scala
+++ b/datamap/mv/core/src/main/scala/org/apache/carbondata/mv/datamap/MVDataMapProvider.scala
@@ -20,7 +20,7 @@
 
 import scala.collection.JavaConverters._
 
-import org.apache.spark.sql.{CarbonSession, SparkSession}
+import org.apache.spark.sql.{CarbonEnv, CarbonSession, SparkSession}
 import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.catalyst.plans.logical.SubqueryAlias
 import org.apache.spark.sql.execution.command.management.CarbonLoadDataCommand
@@ -121,18 +121,10 @@
     val ctasQuery = dataMapSchema.getCtasQuery
     if (ctasQuery != null) {
       val identifier = dataMapSchema.getRelationIdentifier
-      val logicalPlan =
-        new FindDataSourceTable(sparkSession).apply(
-          SparkSQLUtil.sessionState(sparkSession).catalog.lookupRelation(
-          TableIdentifier(identifier.getTableName,
-            Some(identifier.getDatabaseName)))) match {
-          case s: SubqueryAlias => s.child
-          case other => other
-        }
-      val updatedQuery = new CarbonSpark2SqlParser().addPreAggFunction(ctasQuery)
+      val updatedQuery = new CarbonSpark2SqlParser().addMVSkipFunction(ctasQuery)
       val queryPlan = SparkSQLUtil.execute(
         sparkSession.sql(updatedQuery).queryExecution.analyzed,
-        sparkSession).drop("preAgg")
+        sparkSession).drop("mv")
       var isOverwriteTable = false
       val isFullRefresh =
         if (null != dataMapSchema.getProperties.get("full_refresh")) {
diff --git a/datamap/mv/core/src/main/scala/org/apache/carbondata/mv/datamap/MVHelper.scala b/datamap/mv/core/src/main/scala/org/apache/carbondata/mv/datamap/MVHelper.scala
index c28a7a5..b32367b 100644
--- a/datamap/mv/core/src/main/scala/org/apache/carbondata/mv/datamap/MVHelper.scala
+++ b/datamap/mv/core/src/main/scala/org/apache/carbondata/mv/datamap/MVHelper.scala
@@ -68,7 +68,7 @@
     }
     val mvUtil = new MVUtil
     mvUtil.validateDMProperty(dmProperties)
-    val updatedQuery = new CarbonSpark2SqlParser().addPreAggFunction(queryString)
+    val updatedQuery = new CarbonSpark2SqlParser().addMVSkipFunction(queryString)
     val query = sparkSession.sql(updatedQuery)
     val logicalPlan = MVHelper.dropDummFuc(query.queryExecution.analyzed)
     // if there is limit in MV ctas query string, throw exception, as its not a valid usecase
@@ -136,7 +136,7 @@
       if (!mainCarbonTable.get.getTableInfo.isTransactionalTable) {
         throw new MalformedCarbonCommandException("Unsupported operation on NonTransactional table")
       }
-      if (mainCarbonTable.get.isChildTable || mainCarbonTable.get.isChildDataMap) {
+      if (mainCarbonTable.get.isChildTableForMV) {
         throw new MalformedCarbonCommandException(
           "Cannot create Datamap on child table " + mainCarbonTable.get.getTableUniqueName)
       }
@@ -242,7 +242,6 @@
       tableProperties,
       None,
       isAlterFlow = false,
-      isPreAggFlow = false,
       None)
 
     val tablePath = if (dmProperties.contains("path")) {
@@ -403,8 +402,10 @@
 
   private def dropDummyExp(exps: Seq[NamedExpression]) = {
     exps.map {
-      case al@Alias(udf: ScalaUDF, name) if name.equalsIgnoreCase("preAgg") => None
-      case attr: AttributeReference if attr.name.equalsIgnoreCase("preAgg") => None
+      case al@Alias(udf: ScalaUDF, name) if name.equalsIgnoreCase(CarbonEnv.MV_SKIP_RULE_UDF) =>
+        None
+      case attr: AttributeReference if attr.name.equalsIgnoreCase(CarbonEnv.MV_SKIP_RULE_UDF) =>
+        None
       case other => Some(other)
     }.filter(_.isDefined).map(_.get)
   }
diff --git a/datamap/mv/core/src/main/scala/org/apache/carbondata/mv/rewrite/SummaryDatasetCatalog.scala b/datamap/mv/core/src/main/scala/org/apache/carbondata/mv/rewrite/SummaryDatasetCatalog.scala
index ea0bfea..645b304 100644
--- a/datamap/mv/core/src/main/scala/org/apache/carbondata/mv/rewrite/SummaryDatasetCatalog.scala
+++ b/datamap/mv/core/src/main/scala/org/apache/carbondata/mv/rewrite/SummaryDatasetCatalog.scala
@@ -107,8 +107,7 @@
    */
   private[mv] def registerSchema(dataMapSchema: DataMapSchema): Unit = {
     writeLock {
-      // TODO Add mvfunction here, don't use preagg function
-      val updatedQuery = parser.addPreAggFunction(dataMapSchema.getCtasQuery)
+      val updatedQuery = parser.addMVSkipFunction(dataMapSchema.getCtasQuery)
       val query = sparkSession.sql(updatedQuery)
       val planToRegister = MVHelper.dropDummFuc(query.queryExecution.analyzed)
       val modularPlan =
diff --git a/datamap/mv/core/src/test/scala/org/apache/carbondata/mv/rewrite/MVCreateTestCase.scala b/datamap/mv/core/src/test/scala/org/apache/carbondata/mv/rewrite/MVCreateTestCase.scala
index 16ca086..8972068 100644
--- a/datamap/mv/core/src/test/scala/org/apache/carbondata/mv/rewrite/MVCreateTestCase.scala
+++ b/datamap/mv/core/src/test/scala/org/apache/carbondata/mv/rewrite/MVCreateTestCase.scala
@@ -23,6 +23,7 @@
 import org.apache.spark.sql.test.util.QueryTest
 import org.scalatest.BeforeAndAfterAll
 
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.util.CarbonProperties
@@ -813,7 +814,6 @@
 
   test("jira carbondata-2560") {
 
-    sql("drop datamap if exists MV_exp1")
     sql("drop datamap if exists MV_exp2")
     sql("create datamap MV_exp1 using 'mv' as select empname, sum(utilization) from fact_table1 group by empname")
     intercept[UnsupportedOperationException] {
@@ -884,7 +884,6 @@
   test("test create datamap with streaming table")  {
     sql("drop datamap if exists dm_stream_test1")
     sql("drop datamap if exists dm_stream_bloom")
-    sql("drop datamap if exists dm_stream_PreAggMax")
     sql("drop table if exists fact_streaming_table1")
     sql(
       """
@@ -902,9 +901,6 @@
          | DMProperties('INDEX_COLUMNS'='empname,deptname', 'BLOOM_SIZE'='640000')
       """.stripMargin)
 
-    sql("create datamap dm_stream_PreAggMax on table fact_streaming_table1 using 'preaggregate' " +
-        "as select empname,max(salary) as max from fact_streaming_table1 group by empname")
-    
     val exception_tb_mv: Exception = intercept[Exception] {
       sql("create datamap dm_stream_test1 using 'mv' as select empname, sum(utilization) from " +
           "fact_streaming_table1 group by empname")
diff --git a/datamap/mv/core/src/test/scala/org/apache/carbondata/mv/rewrite/TestAllOperationsOnMV.scala b/datamap/mv/core/src/test/scala/org/apache/carbondata/mv/rewrite/TestAllOperationsOnMV.scala
index f140e5d..d0f9ce2 100644
--- a/datamap/mv/core/src/test/scala/org/apache/carbondata/mv/rewrite/TestAllOperationsOnMV.scala
+++ b/datamap/mv/core/src/test/scala/org/apache/carbondata/mv/rewrite/TestAllOperationsOnMV.scala
@@ -203,11 +203,10 @@
     sql("rebuild datamap dm1")
     intercept[UnsupportedOperationException] {
       sql("insert into dm1_table select 2")
-    }.getMessage.contains("Cannot insert/load data directly into pre-aggregate/child table")
+    }.getMessage.contains("Cannot insert data directly into MV table")
     sql("drop table IF EXISTS maintable")
   }
 
-
   test("test drop datamap with tablename") {
     sql("drop table IF EXISTS maintable")
     sql("create table maintable(name string, c_code int, price int) stored by 'carbondata'")
@@ -317,9 +316,6 @@
     sql("drop datamap if exists dm1 ")
     sql("create datamap dm1 using 'mv' as select name, price from maintable")
     intercept[Exception] {
-      sql("create datamap dm_agg on table dm1_table using 'preaggregate' as select maintable_name, sum(maintable_price) from dm1_table group by maintable_name")
-    }.getMessage.contains("Cannot create DataMap on child table default.dm1_table")
-    intercept[Exception] {
       sql("create datamap dm_agg using 'mv' as select maintable_name, sum(maintable_price) from dm1_table group by maintable_name")
     }.getMessage.contains("Cannot create DataMap on child table default.dm1_table")
   }
@@ -420,17 +416,6 @@
     sql("drop table IF EXISTS maintable")
   }
 
-  test("test global dictionary inherited from parent table - Preaggregate") {
-    sql("drop table IF EXISTS maintable")
-    sql("create table maintable(name string, c_code int, price int) stored by 'carbondata' tblproperties('dictionary_include'='name')")
-    sql("insert into table maintable select 'abc',21,2000")
-    sql("drop datamap if exists dm ")
-    sql("create datamap dm on table maintable using 'preaggregate' as select name, sum(price) from maintable group by name")
-    checkExistence(sql("describe formatted maintable_dm"), true, "Global Dictionary maintable_name")
-    checkAnswer(sql("select name, sum(price) from maintable group by name"), Seq(Row("abc", 2000)))
-    sql("drop table IF EXISTS maintable")
-  }
-
   test("test preagg and mv") {
     sql("drop table IF EXISTS maintable")
     sql("create table maintable(name string, c_code int, price int) stored by 'carbondata'")
@@ -438,10 +423,8 @@
     sql("drop datamap if exists dm_mv ")
     sql("create datamap dm_mv using 'mv' as select name, sum(price) from maintable group by name")
     sql("drop datamap if exists dm_pre ")
-    sql("create datamap dm_pre on table maintable using 'preaggregate' as select name, sum(price) from maintable group by name")
     sql("insert into table maintable select 'abcd',21,20002")
     checkAnswer(sql("select count(*) from dm_mv_table"), Seq(Row(2)))
-    checkAnswer(sql("select count(*) from maintable_dm_pre"), Seq(Row(2)))
     sql("drop table IF EXISTS maintable")
   }
 
@@ -456,28 +439,13 @@
     sql("drop table IF EXISTS maintable")
   }
 
-  test("test inverted index & no-inverted index inherited from parent table - Preaggregate") {
-    sql("drop table IF EXISTS maintable")
-    sql("create table maintable(name string, c_code int, price int) stored by 'carbondata' tblproperties('sort_columns'='name', 'inverted_index'='name','sort_scope'='local_sort')")
-    sql("insert into table maintable select 'abc',21,2000")
-    sql("drop datamap if exists dm ")
-    sql("create datamap dm on table maintable using 'preaggregate' as select name, sum(price) from maintable group by name")
-    checkExistence(sql("describe formatted maintable_dm"), true, "Inverted Index Columns maintable_name")
-    checkAnswer(sql("select name, sum(price) from maintable group by name"), Seq(Row("abc", 2000)))
-    sql("drop table IF EXISTS maintable")
-  }
-
   test("test column compressor on preagg and mv") {
     sql("drop table IF EXISTS maintable")
     sql("create table maintable(name string, c_code int, price int) stored by 'carbondata' tblproperties('carbon.column.compressor'='zstd')")
     sql("insert into table maintable select 'abc',21,2000")
-    sql("drop datamap if exists dm_pre ")
-    sql("create datamap dm_pre on table maintable using 'preaggregate' as select name, sum(price) from maintable group by name")
-    var dataMapTable = CarbonMetadata.getInstance().getCarbonTable(CarbonCommonConstants.DATABASE_DEFAULT_NAME, "maintable_dm_pre")
-    assert(dataMapTable.getTableInfo.getFactTable.getTableProperties.get(CarbonCommonConstants.COMPRESSOR).equalsIgnoreCase("zstd"))
     sql("drop datamap if exists dm_mv ")
     sql("create datamap dm_mv on table maintable using 'mv' as select name, sum(price) from maintable group by name")
-    dataMapTable = CarbonMetadata.getInstance().getCarbonTable(CarbonCommonConstants.DATABASE_DEFAULT_NAME, "dm_mv_table")
+    val dataMapTable = CarbonMetadata.getInstance().getCarbonTable(CarbonCommonConstants.DATABASE_DEFAULT_NAME, "dm_mv_table")
     assert(dataMapTable.getTableInfo.getFactTable.getTableProperties.get(CarbonCommonConstants.COMPRESSOR).equalsIgnoreCase("zstd"))
     sql("drop table IF EXISTS maintable")
   }
@@ -486,9 +454,6 @@
     sql("drop table IF EXISTS maintable")
     sql("create table maintable(name string, c_code int, price int) stored by 'carbondata' tblproperties('sort_columns'='name')")
     sql("insert into table maintable select 'abc',21,2000")
-    sql("drop datamap if exists dm_pre ")
-    sql("create datamap dm_pre on table maintable using 'preaggregate' as select name, sum(price) from maintable group by name")
-    checkExistence(sql("describe formatted maintable_dm_pre"), true, "Sort Scope LOCAL_SORT")
     sql("create datamap dm_mv on table maintable using 'mv' as select name, sum(price) from maintable group by name")
     checkExistence(sql("describe formatted dm_mv_table"), true, "Sort Scope LOCAL_SORT")
     sql("drop table IF EXISTS maintable")
@@ -499,9 +464,6 @@
     sql("create table maintable(name string, c_code int, price int) stored by 'carbondata' tblproperties('sort_scope'='no_sort','sort_columns'='name', 'inverted_index'='name')")
     sql("insert into table maintable select 'abc',21,2000")
     checkExistence(sql("describe formatted maintable"), true, "Inverted Index Columns name")
-    sql("drop datamap if exists dm_pre ")
-    sql("create datamap dm_pre on table maintable using 'preaggregate' as select name, sum(price) from maintable group by name")
-    checkExistence(sql("describe formatted maintable_dm_pre"), true, "Inverted Index Columns maintable_name")
     sql("create datamap dm_mv on table maintable using 'mv' as select name, sum(price) from maintable group by name")
     checkExistence(sql("describe formatted dm_mv_table"), true, "Inverted Index Columns maintable_name")
     sql("drop table IF EXISTS maintable")
diff --git a/docs/datamap-developer-guide.md b/docs/datamap-developer-guide.md
index e1fa355..d1748c1 100644
--- a/docs/datamap-developer-guide.md
+++ b/docs/datamap-developer-guide.md
@@ -26,9 +26,7 @@
 ### DataMap Provider
 When user issues `CREATE DATAMAP dm ON TABLE main USING 'provider'`, the corresponding DataMapProvider implementation will be created and initialized. 
 Currently, the provider string can be:
-1. preaggregate: A type of MVDataMap that do pre-aggregate of single table
-2. timeseries: A type of MVDataMap that do pre-aggregate based on time dimension of the table
-3. class name IndexDataMapFactory implementation: Developer can implement new type of IndexDataMap by extending IndexDataMapFactory
+1. class name IndexDataMapFactory implementation: Developer can implement new type of IndexDataMap by extending IndexDataMapFactory
 
 When user issues `DROP DATAMAP dm ON TABLE main`, the corresponding DataMapProvider interface will be called.
 
diff --git a/docs/datamap/datamap-management.md b/docs/datamap/datamap-management.md
index f910559..768ad06 100644
--- a/docs/datamap/datamap-management.md
+++ b/docs/datamap/datamap-management.md
@@ -25,7 +25,6 @@
 - [DataMap Related Commands](#datamap-related-commands)
   - [Explain](#explain)
   - [Show DataMap](#show-datamap)
-  - [Compaction on DataMap](#compaction-on-datamap)
 
 
 
@@ -47,8 +46,6 @@
 
 | DataMap Provider | Description                              | DMPROPERTIES                             | Management       |
 | ---------------- | ---------------------------------------- | ---------------------------------------- | ---------------- |
-| preaggregate     | single table pre-aggregate table         | No DMPROPERTY is required                | Automatic        |
-| timeseries       | time dimension rollup table              | event_time, xx_granularity, please refer to [Timeseries DataMap](./timeseries-datamap-guide.md) | Automatic        |
 | mv               | multi-table pre-aggregate table          | No DMPROPERTY is required                | Manual/Automatic           |
 | lucene           | lucene indexing for text column          | index_columns to specifying the index columns | Automatic |
 | bloomfilter      | bloom filter for high cardinality column, geospatial column | index_columns to specifying the index columns | Automatic |
@@ -136,17 +133,9 @@
 There is a SHOW DATAMAPS command, when this is issued, system will read all datamap from *system* folder and print all information on screen. The current information includes:
 
 - DataMapName
-- DataMapProviderName like mv, preaggreagte, timeseries, etc
+- DataMapProviderName like mv
 - Associated Table
 - DataMap Properties
 - DataMap status (ENABLED/DISABLED)
 - Sync Status - which displays Last segment Id of main table synced with datamap table and its load
   end time (Applicable only for mv datamap)
-
-### Compaction on DataMap
-
-This feature applies for preaggregate datamap only
-
-Running Compaction command (`ALTER TABLE COMPACT`) on main table will **not automatically** compact the pre-aggregate tables created on the main table. User need to run Compaction command separately on each pre-aggregate table to compact them.
-
-Compaction is an optional operation for pre-aggregate table. If compaction is performed on main table but not performed on pre-aggregate table, all queries still can benefit from pre-aggregate tables. To further improve the query performance, compaction on pre-aggregate tables can be triggered to merge the segments and files in the pre-aggregate tables.
diff --git a/docs/datamap/preaggregate-datamap-guide.md b/docs/datamap/preaggregate-datamap-guide.md
deleted file mode 100644
index 5369bb7..0000000
--- a/docs/datamap/preaggregate-datamap-guide.md
+++ /dev/null
@@ -1,275 +0,0 @@
-<!--
-    Licensed to the Apache Software Foundation (ASF) under one or more 
-    contributor license agreements.  See the NOTICE file distributed with
-    this work for additional information regarding copyright ownership. 
-    The ASF licenses this file to you under the Apache License, Version 2.0
-    (the "License"); you may not use this file except in compliance with 
-    the License.  You may obtain a copy of the License at
-
-      http://www.apache.org/licenses/LICENSE-2.0
-
-    Unless required by applicable law or agreed to in writing, software 
-    distributed under the License is distributed on an "AS IS" BASIS, 
-    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-    See the License for the specific language governing permissions and 
-    limitations under the License.
--->
-
-# CarbonData Pre-aggregate DataMap
-  
-* [Quick Example](#quick-example)
-* [DataMap Management](#datamap-management)
-* [Pre-aggregate Table](#preaggregate-datamap-introduction)
-* [Loading Data](#loading-data)
-* [Querying Data](#querying-data)
-* [Compaction](#compacting-pre-aggregate-tables)
-* [Data Management](#data-management-with-pre-aggregate-tables)
-
-## Quick example
-Download and unzip spark-2.2.0-bin-hadoop2.7.tgz, and export $SPARK_HOME
-
-Package carbon jar, and copy assembly/target/scala-2.11/carbondata_2.11-x.x.x-SNAPSHOT-shade-hadoop2.7.2.jar to $SPARK_HOME/jars
-```shell
-mvn clean package -DskipTests -Pspark-2.2
-```
-
-Start spark-shell in new terminal, type :paste, then copy and run the following code.
-```scala
- import java.io.File
- import org.apache.spark.sql.{CarbonEnv, SparkSession}
- import org.apache.spark.sql.CarbonSession._
- import org.apache.spark.sql.streaming.{ProcessingTime, StreamingQuery}
- import org.apache.carbondata.core.util.path.CarbonStorePath
- 
- val warehouse = new File("./warehouse").getCanonicalPath
- val metastore = new File("./metastore").getCanonicalPath
- 
- val spark = SparkSession
-   .builder()
-   .master("local")
-   .appName("preAggregateExample")
-   .config("spark.sql.warehouse.dir", warehouse)
-   .getOrCreateCarbonSession(warehouse, metastore)
-
- spark.sparkContext.setLogLevel("ERROR")
-
- // drop table if exists previously
- spark.sql(s"DROP TABLE IF EXISTS sales")
- 
- // Create main table
- spark.sql(
-   s"""
-      | CREATE TABLE sales (
-      | user_id string,
-      | country string,
-      | quantity int,
-      | price bigint)
-      | STORED AS carbondata
-    """.stripMargin)
- 
- // Create pre-aggregate table on the main table
- // If main table already have data, following command 
- // will trigger one immediate load to the pre-aggregate table
- spark.sql(
-   s"""
-      | CREATE DATAMAP agg_sales
-      | ON TABLE sales
-      | USING "preaggregate"
-      | AS
-      | SELECT country, sum(quantity), avg(price)
-      | FROM sales
-      | GROUP BY country
-    """.stripMargin)
-      
-  import spark.implicits._
-  import org.apache.spark.sql.SaveMode
-  import scala.util.Random
- 
-  // Load data to the main table, it will also
-  // trigger immediate load to pre-aggregate table.
-  // These two loading operation is carried out in a
-  // transactional manner, meaning that the whole 
-  // operation will fail if one of the loading fails
-  val r = new Random()
-  spark.sparkContext.parallelize(1 to 10)
-   .map(x => ("ID." + r.nextInt(100000), "country" + x % 8, x % 50, x % 60))
-   .toDF("user_id", "country", "quantity", "price")
-   .write
-   .format("carbondata")
-   .option("tableName", "sales")
-   .option("compress", "true")
-   .mode(SaveMode.Append)
-   .save()
-      
-  spark.sql(
-    s"""
-       |SELECT country, sum(quantity), avg(price)
-       | from sales GROUP BY country
-     """.stripMargin).show
-
-  spark.stop
-```
-
-#### DataMap Management
-DataMap can be created using following DDL
-  ```
-  CREATE DATAMAP [IF NOT EXISTS] datamap_name
-  ON TABLE main_table
-  USING "datamap_provider"
-  DMPROPERTIES ('key'='value', ...)
-  AS
-    SELECT statement
-  ```
-The string followed by USING is called DataMap Provider, in this version CarbonData supports two 
-kinds of DataMap: 
-1. preaggregate, for pre-aggregate table. Pre-Aggregate table supports two values for DMPROPERTIES.
-   a. 'path' is used to specify the store location of the datamap.('path'='/location/').
-   b. 'partitioning' when set to false enables user to disable partitioning of the datamap.
-       Default value is true for this property.
-2. timeseries, for timeseries roll-up table. Please refer to [Timeseries DataMap](./timeseries-datamap-guide.md)
-
-DataMap can be dropped using following DDL
-  ```
-  DROP DATAMAP [IF EXISTS] datamap_name
-  ON TABLE main_table
-  ```
-To show all DataMaps created, use:
-  ```
-  SHOW DATAMAP 
-  ON TABLE main_table
-  ```
-It will show all DataMaps created on main table.
-
-
-## Preaggregate DataMap Introduction
-  Pre-aggregate tables are created as DataMaps and managed as tables internally by CarbonData. 
-  User can create as many pre-aggregate datamaps required to improve query performance, 
-  provided the storage requirements and loading speeds are acceptable.
-  
-  Once pre-aggregate datamaps are created, CarbonData's SparkSQL optimizer extension supports to 
-  select the most efficient pre-aggregate datamap and rewrite the SQL to query against the selected 
-  datamap instead of the main table. Since the data size of pre-aggregate datamap is smaller, 
-  user queries are much faster. In our previous experience, we have seen 5X to 100X times faster 
-  in production SQLs.
-    
-  For instance, main table called **sales** which is defined as 
-  
-  ```
-  CREATE TABLE sales (
-    order_time timestamp,
-    user_id string,
-    sex string,
-    country string,
-    quantity int,
-    price bigint)
-  STORED AS carbondata
-  ```
-  
-  User can create pre-aggregate tables using the Create DataMap DDL
-  
-  ```
-  CREATE DATAMAP agg_sales
-  ON TABLE sales
-  USING "preaggregate"
-  AS
-    SELECT country, sex, sum(quantity), avg(price)
-    FROM sales
-    GROUP BY country, sex
-  ```
-  **NOTE**:
-   * Preaggregate datamap is deprecated and it is replaced by MV datamap.
-     Please refer [CarbonData MV DataMap](./mv-datamap-guide.md) for more info.
-  
-#### Functions supported in pre-aggregate table
-
-| Function | Rollup supported |
-|----------|:----------------:|
-| SUM      |Yes               |
-| AVG      |Yes               |
-| MAX      |Yes               |
-| MIN      |Yes               |
-| COUNT    |Yes               |
-
-
-#### How pre-aggregate tables are selected
-When a user query is submitted, during query planning phase, CarbonData will collect all matched 
-pre-aggregate tables as candidates according to Relational Algebra transformation rules. Then, the 
-best pre-aggregate table for this query will be selected among the candidates based on cost. 
-For simplicity, current cost estimation is based on the data size of the pre-aggregate table. (We 
-assume that query will be faster on smaller table)
-
-For the main table **sales** and pre-aggregate table **agg_sales** created above, following queries 
-```
-SELECT country, sex, sum(quantity), avg(price) from sales GROUP BY country, sex
-
-SELECT sex, sum(quantity) from sales GROUP BY sex
-
-SELECT avg(price), country from sales GROUP BY country
-``` 
-
-will be transformed by CarbonData's query planner to query against pre-aggregate table 
-**agg_sales** instead of the main table **sales**
-
-However, for following queries
-```
-SELECT user_id, country, sex, sum(quantity), avg(price) from sales GROUP BY user_id, country, sex
-
-SELECT sex, avg(quantity) from sales GROUP BY sex
-
-SELECT country, max(price) from sales GROUP BY country
-```
-
-will query against main table **sales** only, because it does not satisfy pre-aggregate table 
-selection logic. 
-
-## Loading data
-For existing table with loaded data, data load to pre-aggregate table will be triggered by the 
-CREATE DATAMAP statement when user creates the pre-aggregate table. For incremental loads after 
-aggregates tables are created, loading data to main table triggers the load to pre-aggregate tables 
-once main table loading is complete. 
-
-These loads are transactional 
-meaning that data on main table and pre-aggregate tables are only visible to the user after all 
-tables are loaded successfully, if one of these loads fails, new data are not visible in all tables 
-as if the load operation is not happened.   
-
-## Querying data
-As a technique for query acceleration, Pre-aggregate tables cannot be queried directly. 
-Queries are to be made on main table. While doing query planning, internally CarbonData will check 
-associated pre-aggregate tables with the main table, and do query plan transformation accordingly. 
-
-User can verify whether a query can leverage pre-aggregate table or not by executing `EXPLAIN`
-command, which will show the transformed logical plan, and thus user can check whether pre-aggregate
-table is selected.
-
-
-## Compacting pre-aggregate tables
-Running Compaction command (`ALTER TABLE COMPACT`) on main table will **not automatically** 
-compact the pre-aggregate tables created on the main table. User need to run Compaction command 
-separately on each pre-aggregate table to compact them.
-
-Compaction is an optional operation for pre-aggregate table. If compaction is performed on
-main table but not performed on pre-aggregate table, all queries still can benefit from 
-pre-aggregate tables. To further improve the query performance, compaction on pre-aggregate tables 
-can be triggered to merge the segments and files in the pre-aggregate tables. 
-
-## Data Management with pre-aggregate tables
-In current implementation, data consistency needs to be maintained for both main table and pre-aggregate
-tables. Once there is pre-aggregate table created on the main table, following command on the main 
-table
-is not supported:
-1. Data management command: `UPDATE/DELETE/DELETE SEGMENT`. 
-2. Schema management command: `ALTER TABLE DROP COLUMN`, `ALTER TABLE CHANGE DATATYPE`, 
-`ALTER TABLE RENAME`. Note that adding a new column is supported, and for dropping columns and 
-change datatype command, CarbonData will check whether it will impact the pre-aggregate table, if 
- not, the operation is allowed, otherwise operation will be rejected by throwing exception.   
-3. Partition management command: `ALTER TABLE ADD/DROP PARTITION`
-4. Complex Datatypes for preaggregate is not supported.
-
-However, there is still way to support these operations on main table, in current CarbonData 
-release, user can do as following:
-1. Remove the pre-aggregate table by `DROP DATAMAP` command
-2. Carry out the data management operation on main table
-3. Create the pre-aggregate table again by `CREATE DATAMAP` command
-Basically, user can manually trigger the operation by re-building the datamap.
-
diff --git a/docs/datamap/timeseries-datamap-guide.md b/docs/datamap/timeseries-datamap-guide.md
deleted file mode 100644
index d3357fa..0000000
--- a/docs/datamap/timeseries-datamap-guide.md
+++ /dev/null
@@ -1,154 +0,0 @@
-<!--
-    Licensed to the Apache Software Foundation (ASF) under one or more 
-    contributor license agreements.  See the NOTICE file distributed with
-    this work for additional information regarding copyright ownership. 
-    The ASF licenses this file to you under the Apache License, Version 2.0
-    (the "License"); you may not use this file except in compliance with 
-    the License.  You may obtain a copy of the License at
-
-      http://www.apache.org/licenses/LICENSE-2.0
-
-    Unless required by applicable law or agreed to in writing, software 
-    distributed under the License is distributed on an "AS IS" BASIS, 
-    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-    See the License for the specific language governing permissions and 
-    limitations under the License.
--->
-
-# CarbonData Timeseries DataMap
-
-* [Timeseries DataMap Introduction](#timeseries-datamap-introduction-alpha-feature)
-* [Compaction](#compacting-timeseries-datamp)
-* [Data Management](#data-management-on-timeseries-datamap)
-
-## Timeseries DataMap Introduction (Alpha Feature)
-Timeseries DataMap is a pre-aggregate table implementation based on 'pre-aggregate' DataMap.
-Difference is that Timeseries DataMap has built-in understanding of time hierarchy and
-levels: year, month, day, hour, minute, so that it supports automatic roll-up in time dimension 
-for query.
-
-**CAUTION:** Current version of CarbonData does not support roll-up.It will be implemented in future versions.
-
-The data loading, querying, compaction command and its behavior is the same as preaggregate DataMap.
-Please refer to [Pre-aggregate DataMap](./preaggregate-datamap-guide.md)
-for more information.
-  
-To use this datamap, user can create multiple timeseries datamap on the main table which has 
-a *event_time* column, one datamap for one time granularity.
-
-For example, below statement effectively create multiple pre-aggregate tables  on main table called 
-**timeseries**
-
-```
-CREATE DATAMAP agg_year
-ON TABLE sales
-USING "timeseries"
-DMPROPERTIES (
-  'event_time'='order_time',
-  'year_granularity'='1',
-) AS
-SELECT order_time, country, sex, sum(quantity), max(quantity), count(user_id), sum(price),
- avg(price) FROM sales GROUP BY order_time, country, sex
-  
-CREATE DATAMAP agg_month
-ON TABLE sales
-USING "timeseries"
-DMPROPERTIES (
-  'event_time'='order_time',
-  'month_granularity'='1',
-) AS
-SELECT order_time, country, sex, sum(quantity), max(quantity), count(user_id), sum(price),
- avg(price) FROM sales GROUP BY order_time, country, sex
-  
-CREATE DATAMAP agg_day
-ON TABLE sales
-USING "timeseries"
-DMPROPERTIES (
-  'event_time'='order_time',
-  'day_granularity'='1',
-) AS
-SELECT order_time, country, sex, sum(quantity), max(quantity), count(user_id), sum(price),
- avg(price) FROM sales GROUP BY order_time, country, sex
-      
-CREATE DATAMAP agg_sales_hour
-ON TABLE sales
-USING "timeseries"
-DMPROPERTIES (
-  'event_time'='order_time',
-  'hour_granularity'='1',
-) AS
-SELECT order_time, country, sex, sum(quantity), max(quantity), count(user_id), sum(price),
- avg(price) FROM sales GROUP BY order_time, country, sex
-
-CREATE DATAMAP agg_minute
-ON TABLE sales
-USING "timeseries"
-DMPROPERTIES (
-  'event_time'='order_time',
-  'minute_granularity'='1',
-) AS
-SELECT order_time, country, sex, sum(quantity), max(quantity), count(user_id), sum(price),
- avg(price) FROM sales GROUP BY order_time, country, sex
-```
-  
-For querying timeseries data, Carbondata has builtin support for following time related UDF 
-
-```
-timeseries(timeseries column name, 'aggregation level')
-```
-```
-SELECT timeseries(order_time, 'hour'), sum(quantity) FROM sales GROUP BY timeseries(order_time,'hour')
-```
-  
-It is **not necessary** to create pre-aggregate tables for each granularity unless required for 
-query.
- 
-For Example: For main table **sales** , if following timeseries datamaps were created for day 
-level and hour level pre-aggregate
-  
-```
-CREATE DATAMAP agg_day
-ON TABLE sales
-USING "timeseries"
-DMPROPERTIES (
-  'event_time'='order_time',
-  'day_granularity'='1',
-) AS
-SELECT order_time, country, sex, sum(quantity), max(quantity), count(user_id), sum(price),
- avg(price) FROM sales GROUP BY order_time, country, sex
-      
-CREATE DATAMAP agg_sales_hour
-ON TABLE sales
-USING "timeseries"
-DMPROPERTIES (
-  'event_time'='order_time',
-  'hour_granularity'='1',
-) AS
-SELECT order_time, country, sex, sum(quantity), max(quantity), count(user_id), sum(price),
- avg(price) FROM sales GROUP BY order_time, country, sex
-```
-
-Queries like below will not be rolled-up and hit the main table
-```
-Select timeseries(order_time, 'month'), sum(quantity) from sales group by timeseries(order_time,
-  'month')
-  
-Select timeseries(order_time, 'year'), sum(quantity) from sales group by timeseries(order_time,
-  'year')
-```
-
-NOTE (**RESTRICTION**):
-* Only value of 1 is supported for hierarchy levels. Other hierarchy levels will be supported in
-the future CarbonData release. 
-* timeseries datamap for the desired levels needs to be created one after the other
-* timeseries datamaps created for each level needs to be dropped separately 
-      
-
-## Compacting timeseries datamp
-Refer to Compaction section in [preaggregation datamap](./preaggregate-datamap-guide.md). 
-Same applies to timeseries datamap.
-
-## Data Management on timeseries datamap
-Refer to Data Management section in [preaggregation datamap](./preaggregate-datamap-guide.md).
-Same applies to timeseries datamap.
-
diff --git a/docs/introduction.md b/docs/introduction.md
index 037a296..7c11718 100644
--- a/docs/introduction.md
+++ b/docs/introduction.md
@@ -71,14 +71,6 @@
 
 ### DataMaps
 
-- ##### Pre-Aggregate
-
-  CarbonData has concept of datamaps to assist in pruning of data while querying so that performance is faster.Pre Aggregate tables are kind of datamaps which can improve the query performance by order of magnitude.CarbonData will automatically pre-aggregate the incremental data and re-write the query to automatically fetch from the most appropriate pre-aggregate table to serve the query faster.
-
-- ##### Time Series
-
-  CarbonData has built in understanding of time order(Year, month,day,hour, minute,second). Time series is a pre-aggregate table which can automatically roll-up the data to the desired level during incremental load and serve the query from the most appropriate pre-aggregate table.
-
 - ##### Bloom filter
 
   CarbonData supports bloom filter as a datamap in order to quickly and efficiently prune the data for scanning and acheive faster query performance.
@@ -89,7 +81,7 @@
 
 - ##### MV (Materialized Views)
 
-  MVs are kind of pre-aggregate tables which can support efficent query re-write and processing.CarbonData provides MV which can rewrite query to fetch from any table(including non-carbondata tables). Typical usecase is to store the aggregated data of a non-carbondata fact table into carbondata and use mv to rewrite the query to fetch from carbondata.
+  MVs are kind of pre-aggregate and pre-join tables which can support efficient query re-write and processing.CarbonData provides MV which can rewrite query to fetch from any table(including non-carbondata tables). Typical usecase is to store the aggregated data of a non-carbondata fact table into carbondata and use mv to rewrite the query to fetch from carbondata.
 
 ### Streaming
 
diff --git a/docs/language-manual.md b/docs/language-manual.md
index 79aad00..d85c358 100644
--- a/docs/language-manual.md
+++ b/docs/language-manual.md
@@ -27,8 +27,6 @@
   - [DataMaps](./datamap/datamap-management.md)
     - [Bloom](./datamap/bloomfilter-datamap-guide.md)
     - [Lucene](./datamap/lucene-datamap-guide.md)
-    - [Pre-Aggregate](./datamap/preaggregate-datamap-guide.md)
-    - [Time Series](./datamap/timeseries-datamap-guide.md)
   - Materialized Views (MV)
   - [Streaming](./streaming-guide.md)
 - Data Manipulation Statements
diff --git a/docs/presto-guide.md b/docs/presto-guide.md
index c5bcd76..483585f 100644
--- a/docs/presto-guide.md
+++ b/docs/presto-guide.md
@@ -294,5 +294,5 @@
 ### Supported features of presto carbon
 Presto carbon only supports reading the carbon table which is written by spark carbon or carbon SDK. 
 During reading, it supports the non-distributed datamaps like block datamap and bloom datamap.
-It doesn't support MV datamap and Pre-aggregate datamap as it needs query plan to be changed and presto does not allow it.
+It doesn't support Materialized View as it needs query plan to be changed and presto does not allow it.
 Also Presto carbon supports streaming segment read from streaming table created by spark.
diff --git a/docs/streaming-guide.md b/docs/streaming-guide.md
index b993f82..b66fce4 100644
--- a/docs/streaming-guide.md
+++ b/docs/streaming-guide.md
@@ -273,8 +273,8 @@
 ## Constraint
 1. reject set streaming property from true to false.
 2. reject UPDATE/DELETE command on the streaming table.
-3. reject create pre-aggregation DataMap on the streaming table.
-4. reject add the streaming property on the table with pre-aggregation DataMap.
+3. reject create MV on the streaming table.
+4. reject add the streaming property on the table with MV DataMap.
 5. if the table has dictionary columns, it will not support concurrent data loading.
 6. block delete "streaming" segment while the streaming ingestion is running.
 7. block drop the streaming table while the streaming ingestion is running.
diff --git a/docs/usecases.md b/docs/usecases.md
index 8ff4975..c270eef 100644
--- a/docs/usecases.md
+++ b/docs/usecases.md
@@ -189,8 +189,7 @@
 
 - Use table block size as 128MB so that pruning is more effective
 - Use global sort mode so that the data to be fetched are grouped together
-- Create pre-aggregate tables for non timestamp based group by queries
-- For queries containing group by date, create timeseries based Datamap(pre-aggregate) tables so that the data is rolled up during creation and fetch is faster
+- Create Materialized View for aggregation queries
 - Reduce the Spark shuffle partitions.(In our configuration on 14 node cluster, it was reduced to 35 from default of 200)
 - Enable global dictionary for columns which have less cardinalities. Aggregation can be done on encoded data, there by improving the performance
 - For columns whose cardinality is high,enable the local dictionary so that store size is less and can take dictionary benefit for scan
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/MVDataMapExample.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/MVDataMapExample.scala
index fbff90b..2600ab9 100644
--- a/examples/spark2/src/main/scala/org/apache/carbondata/examples/MVDataMapExample.scala
+++ b/examples/spark2/src/main/scala/org/apache/carbondata/examples/MVDataMapExample.scala
@@ -25,7 +25,7 @@
 import org.apache.carbondata.examples.util.ExampleUtils
 
 /**
- * This example is for pre-aggregate tables.
+ * This example is for Materialized View.
  */
 
 object MVDataMapExample {
@@ -42,7 +42,6 @@
                             + "../../../..").getCanonicalPath
     val testData = s"$rootPath/integration/spark-common-test/src/test/resources/sample.csv"
 
-    // 1. simple usage for Pre-aggregate tables creation and query
     spark.sql("DROP TABLE IF EXISTS mainTable")
     spark.sql("DROP TABLE IF EXISTS dimtable")
     spark.sql(
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/PreAggregateDataMapExample.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/PreAggregateDataMapExample.scala
deleted file mode 100644
index b008bd4..0000000
--- a/examples/spark2/src/main/scala/org/apache/carbondata/examples/PreAggregateDataMapExample.scala
+++ /dev/null
@@ -1,239 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.examples
-
-import java.io.File
-
-import org.apache.spark.sql.{SaveMode, SparkSession}
-
-import org.apache.carbondata.examples.util.ExampleUtils
-
-/**
- * This example is for pre-aggregate tables.
- */
-
-object PreAggregateDataMapExample {
-
-  def main(args: Array[String]) {
-    val spark = ExampleUtils.createCarbonSession("PreAggregateTableExample")
-    exampleBody(spark)
-    spark.close()
-  }
-
-  def exampleBody(spark : SparkSession): Unit = {
-    val rootPath = new File(this.getClass.getResource("/").getPath
-                            + "../../../..").getCanonicalPath
-    val testData = s"$rootPath/integration/spark-common-test/src/test/resources/sample.csv"
-
-    // 1. simple usage for Pre-aggregate tables creation and query
-    spark.sql("DROP TABLE IF EXISTS mainTable")
-    spark.sql("DROP TABLE IF EXISTS mainTable_other")
-    spark.sql("""
-                | CREATE TABLE mainTable
-                | (id Int,
-                | name String,
-                | city String,
-                | age Int)
-                | STORED BY 'org.apache.carbondata.format'
-              """.stripMargin)
-
-    spark.sql("""
-                | CREATE TABLE mainTable_other
-                | (id Int,
-                | name String,
-                | city String,
-                | age Int)
-                | STORED BY 'org.apache.carbondata.format'
-              """.stripMargin)
-
-    spark.sql(s"""
-       LOAD DATA LOCAL INPATH '$testData' into table mainTable
-       """)
-
-    spark.sql(s"""
-       LOAD DATA LOCAL INPATH '$testData' into table mainTable_other
-       """)
-
-    // 1. create pre-aggregate table by datamap
-
-    // sum() be hit
-    spark.sql(
-      s"""create datamap preagg_sum on table mainTable using 'preaggregate' as
-         | select id,sum(age) from mainTable group by id"""
-        .stripMargin)
-
-    // avg() and sum() both be hit, because avg = sum()/count()
-    spark.sql(
-      s"""create datamap preagg_avg on table mainTable using 'preaggregate' as
-         | select id,avg(age) from mainTable group by id"""
-        .stripMargin)
-
-    // count() be hit
-    spark.sql(
-      s"""create datamap preagg_count_age on table mainTable using 'preaggregate' as
-         | select id,count(age) from mainTable group by id"""
-        .stripMargin)
-
-    // min() be hit
-    spark.sql(
-      s"""create datamap preagg_min on table mainTable using 'preaggregate' as
-         | select id,min(age) from mainTable group by id"""
-        .stripMargin)
-
-    // max() be hit
-    spark.sql(
-      s"""create datamap preagg_max on table mainTable using 'preaggregate' as
-         | select id,max(age) from mainTable group by id"""
-        .stripMargin)
-
-    // show datamap
-    spark.sql("show datamap on table mainTable").show(false)
-
-    // drop datamap
-    spark.sql("drop datamap preagg_count_age on table mainTable").show()
-    spark.sql("show datamap on table mainTable").show(false)
-
-    spark.sql(
-      s"""
-         | create datamap preagg_case on table mainTable using 'preaggregate' as
-         | select name,sum(case when age=35 then id else 0 end) from mainTable group by name
-         | """.stripMargin)
-
-    spark.sql(
-      s"""create datamap preagg_count on table maintable using 'preaggregate' as
-         | select name, count(*) from maintable group by name""".stripMargin)
-
-    spark.sql("show datamap on table maintable").show(false)
-
-    spark.sql(
-      s"""
-         | SELECT id,max(age)
-         | FROM mainTable group by id
-      """.stripMargin).show()
-
-    spark.sql(
-      s"""
-         | select name, count(*) from
-         | mainTable group by name
-      """.stripMargin).show()
-
-    spark.sql(
-      s"""
-         | select name as NewName,
-         | sum(case when age=35 then id else 0 end) as sum
-         | from mainTable group by name order by name
-      """.stripMargin).show()
-
-    spark.sql(
-      s"""
-         | select t1.name,t1.city from mainTable_other t1 join
-         | (select name as newnewname,sum(age) as sum
-         | from mainTable group by name) t2 on t1.name=t2.newnewname
-      """.stripMargin).show()
-
-    // 2.compare the performance : with pre-aggregate VS main table
-
-    // build the test data, please increase the data for more obvious comparison.
-    // if set the data is larger than 100M, it will take 10+ mins.
-
-    import spark.implicits._
-
-    import scala.util.Random
-    val r = new Random()
-    val df = spark.sparkContext.parallelize(1 to 10 * 10 * 1000)
-      .map(x => ("No." + r.nextInt(100000), "name" + x % 8, "city" + x % 50, x % 60))
-      .toDF("ID", "name", "city", "age")
-
-    // Create table with pre-aggregate
-    spark.sql("DROP TABLE IF EXISTS personTable")
-    spark.sql("DROP TABLE IF EXISTS personTableWithoutAgg")
-    df.write.format("carbondata")
-      .option("tableName", "personTable")
-      .option("compress", "true")
-      .mode(SaveMode.Overwrite).save()
-
-    // Create table without pre-aggregate
-    df.write.format("carbondata")
-      .option("tableName", "personTableWithoutAgg")
-      .option("compress", "true")
-      .mode(SaveMode.Overwrite).save()
-
-    // create pre-aggregate table by datamap
-    spark.sql("""
-       CREATE datamap preagg_avg on table personTable using 'preaggregate' as
-       | select id,avg(age) from personTable group by id
-              """.stripMargin)
-
-    // define time function
-    def time(code: => Unit): Double = {
-      val start = System.currentTimeMillis()
-      code
-      // return time in second
-      (System.currentTimeMillis() - start).toDouble / 1000
-    }
-
-    val time_without_aggTable = time {
-      spark.sql(
-        s"""
-           | SELECT id, avg(age)
-           | FROM personTableWithoutAgg group by id
-      """.stripMargin).count()
-    }
-
-    val time_with_aggTable = time {
-      spark.sql(
-        s"""
-           | SELECT id, avg(age)
-           | FROM personTable group by id
-      """.stripMargin).count()
-    }
-    // scalastyle:off
-    println("time for query on table with pre-aggregate table:" + time_with_aggTable.toString)
-    println("time for query on table without pre-aggregate table:" + time_without_aggTable.toString)
-    // scalastyle:on
-
-    // 3. if avg function is defined for a column, sum also can be used on that;but not other way
-    // round
-    val time_without_aggTable_sum = time {
-      spark.sql(
-        s"""
-           | SELECT id, sum(age)
-           | FROM personTableWithoutAgg group by id
-      """.stripMargin).count()
-    }
-
-    val time_with_aggTable_sum = time {
-      spark.sql(
-        s"""
-           | SELECT id, sum(age)
-           | FROM personTable group by id
-      """.stripMargin).count()
-    }
-    // scalastyle:off
-    println("time for query with function sum on table with pre-aggregate table:" +
-            time_with_aggTable_sum.toString)
-    println("time for query with function sum on table without pre-aggregate table:" +
-            time_without_aggTable_sum.toString)
-    // scalastyle:on
-
-    spark.sql("DROP TABLE IF EXISTS mainTable")
-    spark.sql("DROP TABLE IF EXISTS mainTable_other")
-    spark.sql("DROP TABLE IF EXISTS personTable")
-    spark.sql("DROP TABLE IF EXISTS personTableWithoutAgg")
-  }
-}
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/TimeSeriesPreAggregateTableExample.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/TimeSeriesPreAggregateTableExample.scala
deleted file mode 100644
index 5a79e0b..0000000
--- a/examples/spark2/src/main/scala/org/apache/carbondata/examples/TimeSeriesPreAggregateTableExample.scala
+++ /dev/null
@@ -1,108 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.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
-import org.apache.carbondata.examples.util.ExampleUtils
-
-/**
- * This example is for time series pre-aggregate tables.
- */
-
-object TimeSeriesPreAggregateTableExample {
-
-  def main(args: Array[String]) {
-    val spark = ExampleUtils.createCarbonSession("TimeSeriesPreAggregateTableExample")
-    exampleBody(spark)
-    spark.close()
-  }
-
-  def exampleBody(spark : SparkSession): Unit = {
-    val rootPath = new File(this.getClass.getResource("/").getPath + "../../../..").getCanonicalPath
-    val testData = s"$rootPath/integration/spark-common-test/src/test/resources/timeseriestest.csv"
-    import spark.implicits._
-
-    import scala.util.Random
-    val r = new Random()
-    val df = spark.sparkContext.parallelize(1 to 10 * 1000 )
-      .map(x => ("" + 20 + "%02d".format(r.nextInt(20)) + "-" + "%02d".format(r.nextInt(11) + 1) +
-                 "-" + "%02d".format(r.nextInt(27) + 1) + " " + "%02d".format(r.nextInt(12)) + ":" +
-                 "%02d".format(r.nextInt(59)) + ":" + "%02d".format(r.nextInt(59)), "name" + x % 8,
-        r.nextInt(60))).toDF("mytime", "name", "age")
-
-    // 1. usage for time series Pre-aggregate tables creation and query
-    spark.sql("drop table if exists timeSeriesTable")
-    spark.sql("CREATE TABLE timeSeriesTable(mytime timestamp," +
-              " name string, age int) STORED BY 'org.apache.carbondata.format'")
-    spark.sql(
-      s"""
-         | CREATE DATAMAP agg0_hour ON TABLE timeSeriesTable
-         | USING 'timeSeries'
-         | DMPROPERTIES (
-         | 'EVENT_TIME'='mytime',
-         | 'HOUR_GRANULARITY'='1')
-         | AS SELECT mytime, SUM(age) FROM timeSeriesTable
-         | GROUP BY mytime
-       """.stripMargin)
-    spark.sql(
-      s"""
-         | CREATE DATAMAP agg0_day ON TABLE timeSeriesTable
-         | USING 'timeSeries'
-         | DMPROPERTIES (
-         | 'EVENT_TIME'='mytime',
-         | 'DAY_GRANULARITY'='1')
-         | AS SELECT mytime, SUM(age) FROM timeSeriesTable
-         | GROUP BY mytime
-       """.stripMargin)
-
-
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy-MM-dd HH:mm:ss")
-
-    df.write.format("carbondata")
-      .option("tableName", "timeSeriesTable")
-      .option("compress", "true")
-      .mode(SaveMode.Append).save()
-
-    spark.sql(
-      s"""
-         select sum(age), timeseries(mytime,'hour') from timeSeriesTable group by timeseries(mytime,
-         'hour')
-      """.stripMargin).show()
-
-    spark.sql(
-      s"""
-         select avg(age),timeseries(mytime,'year') from timeSeriesTable group by timeseries(mytime,
-         'year')
-      """.stripMargin).show()
-
-    CarbonProperties.getInstance().addProperty(
-      CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
-      CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT)
-    CarbonProperties.getInstance().addProperty(
-      CarbonCommonConstants.CARBON_DATE_FORMAT,
-      CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT)
-
-    spark.sql("DROP TABLE IF EXISTS timeSeriesTable")
-  }
-}
diff --git a/examples/spark2/src/test/scala/org/apache/carbondata/examplesCI/RunExamples.scala b/examples/spark2/src/test/scala/org/apache/carbondata/examplesCI/RunExamples.scala
index cef2b97..b42fd13 100644
--- a/examples/spark2/src/test/scala/org/apache/carbondata/examplesCI/RunExamples.scala
+++ b/examples/spark2/src/test/scala/org/apache/carbondata/examplesCI/RunExamples.scala
@@ -90,10 +90,6 @@
     DataUpdateDeleteExample.exampleBody(spark)
   }
 
-  test("PreAggregateDataMapExample") {
-    PreAggregateDataMapExample.exampleBody(spark)
-  }
-
   test("QuerySegmentExample") {
     QuerySegmentExample.exampleBody(spark)
   }
@@ -106,10 +102,6 @@
     TableLevelCompactionOptionExample.exampleBody(spark)
   }
 
-  test("TimeSeriesPreAggregateTableExample") {
-    TimeSeriesPreAggregateTableExample.exampleBody(spark)
-  }
-
   test("LuceneDataMapExample") {
     LuceneDataMapExample.exampleBody(spark)
   }
diff --git a/format/src/main/thrift/schema.thrift b/format/src/main/thrift/schema.thrift
index 1180991..ca4bbad 100644
--- a/format/src/main/thrift/schema.thrift
+++ b/format/src/main/thrift/schema.thrift
@@ -105,6 +105,7 @@
 	/** 
 	 * Used when this column is part of an aggregate table.
 	 */
+	 /** Deprecated */
 	11: optional string aggregate_function;
 
 	12: optional binary default_value;
@@ -129,6 +130,7 @@
   *  to maintain the column relation with parent table.
   *  will be usefull in case of pre-aggregate
   **/
+  /** Deprecated */
 	17: optional list<ParentColumnTableRelation> parentColumnTableRelations;
 }
 
@@ -204,10 +206,11 @@
     // stores properties of select query, query type like groupby, join in
     // case of preaggregate/timeseries
     3: optional map<string, string> properties;
-    // relation identifier of a table which stores data of datamaps like preaggregate/timeseries.
+    // relation identifier of a table which stores data of Materialize View.
     4: optional RelationIdentifier childTableIdentifier;
     // in case of preaggregate/timeseries datamap it will be used to maintain the child schema
     // which will be usefull in case of query and data load
+    // Deprecated
     5: optional TableSchema childTableSchema;
 }
 
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonFileInputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonFileInputFormat.java
index 52796fb..6ab91cb 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonFileInputFormat.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonFileInputFormat.java
@@ -110,99 +110,96 @@
     }
     AbsoluteTableIdentifier identifier = carbonTable.getAbsoluteTableIdentifier();
 
-    if (getValidateSegmentsToAccess(job.getConfiguration())) {
-      // get all valid segments and set them into the configuration
-      // check for externalTable segment (Segment_null)
-      // process and resolve the expression
+    // get all valid segments and set them into the configuration
+    // check for externalTable segment (Segment_null)
+    // process and resolve the expression
 
-      ReadCommittedScope readCommittedScope = null;
-      if (carbonTable.isTransactionalTable()) {
-        readCommittedScope = new LatestFilesReadCommittedScope(
-            identifier.getTablePath() + "/Fact/Part0/Segment_null/", job.getConfiguration());
+    ReadCommittedScope readCommittedScope = null;
+    if (carbonTable.isTransactionalTable()) {
+      readCommittedScope = new LatestFilesReadCommittedScope(
+          identifier.getTablePath() + "/Fact/Part0/Segment_null/", job.getConfiguration());
+    } else {
+      readCommittedScope = getReadCommittedScope(job.getConfiguration());
+      if (readCommittedScope == null) {
+        readCommittedScope = new LatestFilesReadCommittedScope(identifier.getTablePath(), job
+            .getConfiguration());
       } else {
-        readCommittedScope = getReadCommittedScope(job.getConfiguration());
-        if (readCommittedScope == null) {
-          readCommittedScope = new LatestFilesReadCommittedScope(identifier.getTablePath(), job
-              .getConfiguration());
-        } else {
-          readCommittedScope.setConfiguration(job.getConfiguration());
-        }
+        readCommittedScope.setConfiguration(job.getConfiguration());
       }
-      // this will be null in case of corrupt schema file.
-      DataMapFilter filter = getFilterPredicates(job.getConfiguration());
-
-      // if external table Segments are found, add it to the List
-      List<Segment> externalTableSegments = new ArrayList<Segment>();
-      Segment seg;
-      if (carbonTable.isTransactionalTable()) {
-        // SDK some cases write into the Segment Path instead of Table Path i.e. inside
-        // the "Fact/Part0/Segment_null". The segment in this case is named as "null".
-        // The table is denoted by default as a transactional table and goes through
-        // the path of CarbonFileInputFormat. The above scenario is handled in the below code.
-        seg = new Segment("null", null, readCommittedScope);
-        externalTableSegments.add(seg);
-      } else {
-        LoadMetadataDetails[] loadMetadataDetails = readCommittedScope.getSegmentList();
-        for (LoadMetadataDetails load : loadMetadataDetails) {
-          seg = new Segment(load.getLoadName(), null, readCommittedScope);
-          if (fileLists != null) {
-            for (int i = 0; i < fileLists.size(); i++) {
-              if (fileLists.get(i).toString().endsWith(seg.getSegmentNo()
-                  + CarbonTablePath.CARBON_DATA_EXT)) {
-                externalTableSegments.add(seg);
-                break;
-              }
-            }
-          } else {
-            externalTableSegments.add(seg);
-          }
-        }
-      }
-      List<InputSplit> splits = new ArrayList<>();
-      boolean useBlockDataMap = job.getConfiguration().getBoolean("filter_blocks", true);
-      // useBlockDataMap would be false in case of SDK when user has not provided any filter, In
-      // this case we don't want to load block/blocklet datamap. It would be true in all other
-      // scenarios
-      if (filter != null) {
-        filter.resolve(false);
-      }
-      if (useBlockDataMap) {
-        // do block filtering and get split
-        splits = getSplits(job, filter, externalTableSegments);
-      } else {
-        List<CarbonFile> carbonFiles = null;
-        if (null != this.fileLists) {
-          carbonFiles = getAllCarbonDataFiles(this.fileLists);
-        } else {
-          carbonFiles = getAllCarbonDataFiles(carbonTable.getTablePath());
-        }
-
-        for (CarbonFile carbonFile : carbonFiles) {
-          // Segment id is set to null because SDK does not write carbondata files with respect
-          // to segments. So no specific name is present for this load.
-          CarbonInputSplit split =
-              new CarbonInputSplit("null", carbonFile.getAbsolutePath(), 0,
-                  carbonFile.getLength(), carbonFile.getLocations(), FileFormat.COLUMNAR_V3);
-          split.setVersion(ColumnarFormatVersion.V3);
-          BlockletDetailInfo info = new BlockletDetailInfo();
-          split.setDetailInfo(info);
-          info.setBlockSize(carbonFile.getLength());
-          info.setVersionNumber(split.getVersion().number());
-          info.setUseMinMaxForPruning(false);
-          splits.add(split);
-        }
-        Collections.sort(splits, new Comparator<InputSplit>() {
-          @Override
-          public int compare(InputSplit o1, InputSplit o2) {
-            return ((CarbonInputSplit) o1).getFilePath()
-                .compareTo(((CarbonInputSplit) o2).getFilePath());
-          }
-        });
-      }
-      setAllColumnProjectionIfNotConfigured(job, carbonTable);
-      return splits;
     }
-    return null;
+    // this will be null in case of corrupt schema file.
+    DataMapFilter filter = getFilterPredicates(job.getConfiguration());
+
+    // if external table Segments are found, add it to the List
+    List<Segment> externalTableSegments = new ArrayList<Segment>();
+    Segment seg;
+    if (carbonTable.isTransactionalTable()) {
+      // SDK some cases write into the Segment Path instead of Table Path i.e. inside
+      // the "Fact/Part0/Segment_null". The segment in this case is named as "null".
+      // The table is denoted by default as a transactional table and goes through
+      // the path of CarbonFileInputFormat. The above scenario is handled in the below code.
+      seg = new Segment("null", null, readCommittedScope);
+      externalTableSegments.add(seg);
+    } else {
+      LoadMetadataDetails[] loadMetadataDetails = readCommittedScope.getSegmentList();
+      for (LoadMetadataDetails load : loadMetadataDetails) {
+        seg = new Segment(load.getLoadName(), null, readCommittedScope);
+        if (fileLists != null) {
+          for (int i = 0; i < fileLists.size(); i++) {
+            if (fileLists.get(i).toString().endsWith(seg.getSegmentNo()
+                + CarbonTablePath.CARBON_DATA_EXT)) {
+              externalTableSegments.add(seg);
+              break;
+            }
+          }
+        } else {
+          externalTableSegments.add(seg);
+        }
+      }
+    }
+    List<InputSplit> splits = new ArrayList<>();
+    boolean useBlockDataMap = job.getConfiguration().getBoolean("filter_blocks", true);
+    // useBlockDataMap would be false in case of SDK when user has not provided any filter, In
+    // this case we don't want to load block/blocklet datamap. It would be true in all other
+    // scenarios
+    if (filter != null) {
+      filter.resolve(false);
+    }
+    if (useBlockDataMap) {
+      // do block filtering and get split
+      splits = getSplits(job, filter, externalTableSegments);
+    } else {
+      List<CarbonFile> carbonFiles = null;
+      if (null != this.fileLists) {
+        carbonFiles = getAllCarbonDataFiles(this.fileLists);
+      } else {
+        carbonFiles = getAllCarbonDataFiles(carbonTable.getTablePath());
+      }
+
+      for (CarbonFile carbonFile : carbonFiles) {
+        // Segment id is set to null because SDK does not write carbondata files with respect
+        // to segments. So no specific name is present for this load.
+        CarbonInputSplit split =
+            new CarbonInputSplit("null", carbonFile.getAbsolutePath(), 0,
+                carbonFile.getLength(), carbonFile.getLocations(), FileFormat.COLUMNAR_V3);
+        split.setVersion(ColumnarFormatVersion.V3);
+        BlockletDetailInfo info = new BlockletDetailInfo();
+        split.setDetailInfo(info);
+        info.setBlockSize(carbonFile.getLength());
+        info.setVersionNumber(split.getVersion().number());
+        info.setUseMinMaxForPruning(false);
+        splits.add(split);
+      }
+      Collections.sort(splits, new Comparator<InputSplit>() {
+        @Override
+        public int compare(InputSplit o1, InputSplit o2) {
+          return ((CarbonInputSplit) o1).getFilePath()
+              .compareTo(((CarbonInputSplit) o2).getFilePath());
+        }
+      });
+    }
+    setAllColumnProjectionIfNotConfigured(job, carbonTable);
+    return splits;
   }
 
   public void setAllColumnProjectionIfNotConfigured(JobContext job, CarbonTable carbonTable) {
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonInputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonInputFormat.java
index 5f7814a..10aabf2 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonInputFormat.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonInputFormat.java
@@ -31,7 +31,6 @@
 
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.constants.CarbonCommonConstantsInternal;
 import org.apache.carbondata.core.datamap.DataMapChooser;
 import org.apache.carbondata.core.datamap.DataMapFilter;
 import org.apache.carbondata.core.datamap.DataMapJob;
@@ -102,8 +101,6 @@
   // comma separated list of input segment numbers
   public static final String INPUT_SEGMENT_NUMBERS =
       "mapreduce.input.carboninputformat.segmentnumbers";
-  private static final String VALIDATE_INPUT_SEGMENT_IDs =
-      "mapreduce.input.carboninputformat.validsegments";
   // comma separated list of input files
   private static final String ALTER_PARTITION_ID = "mapreduce.input.carboninputformat.partitionid";
   private static final Logger LOG =
@@ -321,21 +318,6 @@
   }
 
   /**
-   * set list of segment to access
-   */
-  public static void setValidateSegmentsToAccess(Configuration configuration, Boolean validate) {
-    configuration.set(CarbonInputFormat.VALIDATE_INPUT_SEGMENT_IDs, validate.toString());
-  }
-
-  /**
-   * get list of segment to access
-   */
-  public static boolean getValidateSegmentsToAccess(Configuration configuration) {
-    return configuration.get(CarbonInputFormat.VALIDATE_INPUT_SEGMENT_IDs, "true")
-        .equalsIgnoreCase("true");
-  }
-
-  /**
    * set list of partitions to prune
    */
   public static void setPartitionsToPrune(Configuration configuration,
@@ -844,25 +826,6 @@
     return tableName;
   }
 
-  public static void setAccessStreamingSegments(Configuration configuration, Boolean validate)
-      throws InvalidConfigurationException {
-    configuration.set(
-        CarbonCommonConstantsInternal.QUERY_ON_PRE_AGG_STREAMING + "." + getDatabaseName(
-            configuration) + "." + getTableName(configuration), validate.toString());
-  }
-
-  public static boolean getAccessStreamingSegments(Configuration configuration) {
-    try {
-      return configuration.get(
-          CarbonCommonConstantsInternal.QUERY_ON_PRE_AGG_STREAMING + "." + getDatabaseName(
-              configuration) + "." + getTableName(
-                  configuration), "false").equalsIgnoreCase("true");
-
-    } catch (InvalidConfigurationException e) {
-      return false;
-    }
-  }
-
   /**
    * Project all Columns for carbon reader
    *
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonOutputCommitter.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonOutputCommitter.java
index 3eda548..3c14ee3 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonOutputCommitter.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonOutputCommitter.java
@@ -140,11 +140,6 @@
     OperationContext operationContext = (OperationContext) getOperationContext();
     CarbonTable carbonTable = loadModel.getCarbonDataLoadSchema().getCarbonTable();
     String uuid = "";
-    if (loadModel.getCarbonDataLoadSchema().getCarbonTable().isChildDataMap() &&
-        operationContext != null) {
-      uuid = operationContext.getProperty("uuid").toString();
-    }
-
     SegmentFileStore.updateTableStatusFile(carbonTable, loadModel.getSegmentId(),
         segmentFileName + CarbonTablePath.SEGMENT_EXT,
         carbonTable.getCarbonTableIdentifier().getTableId(),
@@ -207,8 +202,10 @@
           context.getConfiguration().get(CarbonTableOutputFormat.SEGMENTS_TO_BE_DELETED, "");
       List<Segment> segmentDeleteList = Segment.toSegmentList(segmentsToBeDeleted.split(","), null);
       Set<Segment> segmentSet = new HashSet<>(
-          new SegmentStatusManager(carbonTable.getAbsoluteTableIdentifier(),
-              context.getConfiguration()).getValidAndInvalidSegments(carbonTable.isChildTable())
+          new SegmentStatusManager(
+              carbonTable.getAbsoluteTableIdentifier(),
+              context.getConfiguration()
+          ).getValidAndInvalidSegments(carbonTable.isChildTableForMV())
               .getValidSegments());
       if (updateTime != null) {
         CarbonUpdateUtil.updateTableMetadataStatus(segmentSet, carbonTable, updateTime, true,
@@ -244,7 +241,7 @@
     if (partitionSpecs != null && partitionSpecs.size() > 0) {
       List<Segment> validSegments =
           new SegmentStatusManager(table.getAbsoluteTableIdentifier())
-              .getValidAndInvalidSegments(table.isChildTable()).getValidSegments();
+              .getValidAndInvalidSegments(table.isChildTableForMV()).getValidSegments();
       String uniqueId = String.valueOf(System.currentTimeMillis());
       List<String> tobeUpdatedSegs = new ArrayList<>();
       List<String> tobeDeletedSegs = new ArrayList<>();
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 6a0f946..5468c24 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
@@ -116,54 +116,35 @@
     SegmentStatusManager segmentStatusManager = new SegmentStatusManager(identifier,
         readCommittedScope.getConfiguration());
     SegmentStatusManager.ValidAndInvalidSegmentsInfo segments = segmentStatusManager
-        .getValidAndInvalidSegments(carbonTable.isChildTable(), loadMetadataDetails,
+        .getValidAndInvalidSegments(carbonTable.isChildTableForMV(), loadMetadataDetails,
             this.readCommittedScope);
 
-    // to check whether only streaming segments access is enabled or not,
-    // if access streaming segment is true then data will be read from streaming segments
-    boolean accessStreamingSegments = getAccessStreamingSegments(job.getConfiguration());
-    if (getValidateSegmentsToAccess(job.getConfiguration())) {
-      if (!accessStreamingSegments) {
-        List<Segment> validSegments = segments.getValidSegments();
-        streamSegments = segments.getStreamSegments();
-        streamSegments = getFilteredSegment(job, streamSegments, readCommittedScope);
-        if (validSegments.size() == 0) {
-          return getSplitsOfStreaming(job, streamSegments, carbonTable);
-        }
-        List<Segment> filteredSegmentToAccess =
-            getFilteredSegment(job, segments.getValidSegments(), readCommittedScope);
-        if (filteredSegmentToAccess.size() == 0) {
-          return getSplitsOfStreaming(job, streamSegments, carbonTable);
-        } else {
-          setSegmentsToAccess(job.getConfiguration(), filteredSegmentToAccess);
-        }
-      } else {
-        List<Segment> filteredNormalSegments =
-            getFilteredNormalSegments(segments.getValidSegments(),
-                getSegmentsToAccess(job, readCommittedScope));
-        streamSegments = segments.getStreamSegments();
-        if (filteredNormalSegments.size() == 0) {
-          return getSplitsOfStreaming(job, streamSegments, carbonTable);
-        }
-        setSegmentsToAccess(job.getConfiguration(), filteredNormalSegments);
-      }
-      // remove entry in the segment index if there are invalid segments
-      for (Segment segment : segments.getInvalidSegments()) {
-        invalidSegmentIds.add(segment.getSegmentNo());
-      }
-      if (invalidSegmentIds.size() > 0) {
-        DataMapStoreManager.getInstance()
-            .clearInvalidSegments(getOrCreateCarbonTable(job.getConfiguration()),
-                invalidSegmentIds);
-      }
+    List<Segment> validSegments = segments.getValidSegments();
+    streamSegments = segments.getStreamSegments();
+    streamSegments = getFilteredSegment(job, streamSegments, readCommittedScope);
+    if (validSegments.size() == 0) {
+      return getSplitsOfStreaming(job, streamSegments, carbonTable);
     }
-    List<Segment> validAndInProgressSegments = new ArrayList<>(segments.getValidSegments());
-    // Add in progress segments also to filter it as in case of aggregate table load it loads
-    // data from in progress table.
-    validAndInProgressSegments.addAll(segments.getListOfInProgressSegments());
-    // get updated filtered list
     List<Segment> filteredSegmentToAccess =
-        getFilteredSegment(job, new ArrayList<>(validAndInProgressSegments), readCommittedScope);
+        getFilteredSegment(job, segments.getValidSegments(), readCommittedScope);
+    if (filteredSegmentToAccess.size() == 0) {
+      return getSplitsOfStreaming(job, streamSegments, carbonTable);
+    } else {
+      setSegmentsToAccess(job.getConfiguration(), filteredSegmentToAccess);
+    }
+
+    // remove entry in the segment index if there are invalid segments
+    for (Segment segment : segments.getInvalidSegments()) {
+      invalidSegmentIds.add(segment.getSegmentNo());
+    }
+    if (invalidSegmentIds.size() > 0) {
+      DataMapStoreManager.getInstance()
+          .clearInvalidSegments(getOrCreateCarbonTable(job.getConfiguration()),
+              invalidSegmentIds);
+    }
+
+    List<Segment> segmentToAccess =
+        getFilteredSegment(job, segments.getValidSegments(), readCommittedScope);
 
     // process and resolve the expression
     DataMapFilter dataMapFilter = getFilterPredicates(job.getConfiguration());
@@ -174,7 +155,7 @@
 
     // do block filtering and get split
     List<InputSplit> splits = getSplits(
-        job, dataMapFilter, filteredSegmentToAccess,
+        job, dataMapFilter, segmentToAccess,
         updateStatusManager, segments.getInvalidSegments());
 
     // add all splits of streaming
@@ -196,28 +177,6 @@
    */
 
   /**
-   * Below method will be used to get the filter segments when query is fired on pre Aggregate
-   * and main table in case of streaming.
-   * For Pre Aggregate rules it will set all the valid segments for both streaming and
-   * and normal for fact table, so if any handoff happened in between it will
-   * select only new hand off segments segments for fact.
-   * @param validSegments
-   * @param segmentsToAccess
-   * @return
-   */
-  private List<Segment> getFilteredNormalSegments(List<Segment> validSegments,
-      Segment[] segmentsToAccess) {
-    List<Segment> segmentToAccessSet = Arrays.asList(segmentsToAccess);
-    List<Segment> filteredSegment = new ArrayList<>();
-    for (Segment seg : validSegments) {
-      if (!segmentToAccessSet.contains(seg)) {
-        filteredSegment.add(seg);
-      }
-    }
-    return filteredSegment;
-  }
-
-  /**
    * Return segment list after filtering out valid segments and segments set by user by
    * `INPUT_SEGMENT_NUMBERS` in job configuration
    */
@@ -420,7 +379,7 @@
         table, loadMetadataDetails);
     SegmentStatusManager.ValidAndInvalidSegmentsInfo allSegments =
         new SegmentStatusManager(identifier, readCommittedScope.getConfiguration())
-            .getValidAndInvalidSegments(table.isChildTable(), loadMetadataDetails,
+            .getValidAndInvalidSegments(table.isChildTableForMV(), loadMetadataDetails,
                 readCommittedScope);
     Map<String, Long> blockRowCountMapping = new HashMap<>();
     Map<String, Long> segmentAndBlockCountMapping = new HashMap<>();
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java
index 872ebb5..d78e44b 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java
@@ -24,7 +24,6 @@
 
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.constants.CarbonCommonConstantsInternal;
 import org.apache.carbondata.core.datamap.DataMapFilter;
 import org.apache.carbondata.core.datamap.DataMapJob;
 import org.apache.carbondata.core.datamap.DataMapUtil;
@@ -111,7 +110,7 @@
       AbsoluteTableIdentifier identifier,
       DataMapFilter dataMapFilter,
       CarbonProjection columnProjection,
-      DataMapJob dataMapJob) throws InvalidConfigurationException, IOException {
+      DataMapJob dataMapJob) throws IOException {
     CarbonTableInputFormat<V> format = new CarbonTableInputFormat<>();
     CarbonInputFormat.setTablePath(
         conf,
@@ -128,31 +127,12 @@
     CarbonSessionInfo carbonSessionInfo = ThreadLocalSessionInfo.getCarbonSessionInfo();
     if (carbonSessionInfo != null) {
       String tableUniqueKey = identifier.getDatabaseName() + "." + identifier.getTableName();
-      String validateInputSegmentsKey = CarbonCommonConstants.VALIDATE_CARBON_INPUT_SEGMENTS +
-          tableUniqueKey;
-      CarbonInputFormat.setValidateSegmentsToAccess(
-          conf,
-          Boolean.valueOf(carbonSessionInfo.getThreadParams().getProperty(
-              validateInputSegmentsKey, "true")));
-      String queryOnPreAggStreamingKey = CarbonCommonConstantsInternal.QUERY_ON_PRE_AGG_STREAMING +
-          tableUniqueKey;
-      boolean queryOnPreAggStreaming = Boolean.valueOf(carbonSessionInfo.getThreadParams()
-          .getProperty(queryOnPreAggStreamingKey, "false"));
       String inputSegmentsKey = CarbonCommonConstants.CARBON_INPUT_SEGMENTS + tableUniqueKey;
-      CarbonInputFormat.setValidateSegmentsToAccess(conf,
-          Boolean.valueOf(carbonSessionInfo.getThreadParams()
-              .getProperty(validateInputSegmentsKey, "true")));
       CarbonInputFormat.setQuerySegment(
           conf,
           carbonSessionInfo.getThreadParams().getProperty(
               inputSegmentsKey,
               CarbonProperties.getInstance().getProperty(inputSegmentsKey, "*")));
-      if (queryOnPreAggStreaming) {
-        CarbonInputFormat.setAccessStreamingSegments(conf, true);
-        carbonSessionInfo.getThreadParams().removeProperty(queryOnPreAggStreamingKey);
-        carbonSessionInfo.getThreadParams().removeProperty(inputSegmentsKey);
-        carbonSessionInfo.getThreadParams().removeProperty(validateInputSegmentsKey);
-      }
     }
     return format;
   }
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/PartitionWithPreAggregateTestCase.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/PartitionWithPreAggregateTestCase.scala
deleted file mode 100644
index 23d5924..0000000
--- a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/PartitionWithPreAggregateTestCase.scala
+++ /dev/null
@@ -1,255 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.cluster.sdv.generated
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.spark.sql.Row
-import org.apache.spark.sql.common.util._
-import org.scalatest.BeforeAndAfterAll
-/**
-  * Test Class for Support of Partition with PreAggregate table
-  */
-class PartitionWithPreAggregateTestCase extends QueryTest with BeforeAndAfterAll {
-  override def beforeAll = {
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/MM/dd HH:mm:ss")
-      .addProperty(CarbonCommonConstants.CARBON_DATE_FORMAT, "yyyy/MM/dd")
-  }
-
-  //Loading data into partitioned table with SORT_SCOPE=LOCAL_SORT
-  test("Partition-With-PreAggregate_TC001", Include) {
-    sql("drop table if exists partition_table")
-    sql(
-      s"""CREATE TABLE partition_table(shortField SHORT, intField INT, bigintField LONG,
-         |doubleField DOUBLE, timestamp TIMESTAMP, decimalField DECIMAL(18,2),dateField DATE,
-         |charField CHAR(5), floatField FLOAT ) PARTITIONED BY (stringField STRING) STORED BY
-         |'carbondata' TBLPROPERTIES('SORT_SCOPE'='LOCAL_SORT')""".stripMargin)
-    sql(
-      s"""load data inpath '$resourcesPath/Data/partition/list_partition_table.csv' into table
-         |partition_table""".stripMargin)
-    sql(
-      "create datamap ag1 on table partition_table using 'preaggregate' as select shortField, sum" +
-      "(intField) from partition_table group by shortField")
-    checkAnswer(sql(
-      s"""select decimalfield from partition_table where charfield='e' and
-         |floatfield=307.301 group by decimalfield limit 1""".stripMargin),
-      Seq(Row(159.10)))
-  }
-
-  //Loading data into partitioned table with SORT_SCOPE=GLOBAL_SORT
-  test("Partition-With-PreAggregate_TC002", Include) {
-    sql("drop table if exists partition_table")
-    sql(
-      s"""CREATE TABLE partition_table(shortField SHORT, intField INT, bigintField LONG,
-         |doubleField DOUBLE, timestamp TIMESTAMP, decimalField DECIMAL(18,2),dateField DATE,
-         |charField CHAR(5), floatField FLOAT ) PARTITIONED BY (stringField STRING) STORED BY
-         |'carbondata' TBLPROPERTIES('SORT_SCOPE'='GLOBAL_SORT')""".stripMargin)
-    sql(
-      s"""load data inpath '$resourcesPath/Data/partition/list_partition_table.csv' into table
-         |partition_table""".stripMargin)
-    sql(
-      "create datamap ag1 on table partition_table using 'preaggregate' as select shortField, sum" +
-      "(intField) from partition_table group by shortField")
-    checkAnswer(sql(
-      s"""select decimalfield from partition_table where charfield='e' and
-         |floatfield=307.301 group by decimalfield limit 1""".stripMargin),
-      Seq(Row(159.10)))
-  }
-
-  //Loading data into partitioned table with SORT_SCOPE=NO_SORT
-  test("Partition-With-PreAggregate_TC004", Include) {
-    sql("drop table if exists partition_table")
-    sql(
-      s"""CREATE TABLE partition_table(shortField SHORT, intField INT, bigintField LONG,
-         |doubleField DOUBLE, timestamp TIMESTAMP, decimalField DECIMAL(18,2),dateField DATE,
-         |charField CHAR(5), floatField FLOAT ) PARTITIONED BY (stringField STRING) STORED BY
-         |'carbondata' TBLPROPERTIES('SORT_SCOPE'='NO_SORT')""".stripMargin)
-    sql(
-      s"""load data inpath '$resourcesPath/Data/partition/list_partition_table.csv' into table
-         |partition_table""".stripMargin)
-    sql(
-      "create datamap ag1 on table partition_table using 'preaggregate' as select shortField, sum" +
-      "(intField) from partition_table group by shortField")
-    checkAnswer(sql(
-      s"""select decimalfield from partition_table where charfield='e' and
-         |floatfield=307.301 group by decimalfield limit 1""".stripMargin),
-      Seq(Row(159.10)))
-  }
-
-  //Verify Aggregation query on Loaded Partition table with Pre-Aggregate table
-  test("Partition-With-PreAggregate_TC005", Include) {
-    sql("drop table if exists partition_table")
-    sql(
-      s"""CREATE TABLE partition_table(shortField SHORT, intField INT, bigintField LONG,
-         |doubleField DOUBLE, timestamp TIMESTAMP, decimalField DECIMAL(18,2),dateField DATE,
-         |charField CHAR(5), floatField FLOAT ) PARTITIONED BY (stringField STRING) STORED BY
-         |'carbondata'""".stripMargin)
-    sql(
-      "create datamap ag1 on table partition_table using 'preaggregate' as select shortField, sum" +
-      "(intField) from partition_table group by shortField")
-    sql(
-      s"""load data inpath '$resourcesPath/Data/partition/list_partition_table.csv' into table
-         |partition_table""".stripMargin)
-    assert(sql("explain select shortField, sum(intField) from partition_table group by shortField")
-      .collect().head.get(0).toString.contains("partition_table_ag1"))
-  }
-
-  //Verify load with Pre_Aggregate table on Partition table after compaction
-  test("Partition-With-PreAggregate_TC006", Include) {
-    sql("drop table if exists uniqdata")
-    sql(
-      s"""CREATE TABLE uniqdata (CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp,
-         |BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10),
-         |DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,
-         |INTEGER_COLUMN1 int, DOJ timestamp) PARTITIONED BY (CUST_ID int) STORED BY
-         |'carbondata'""".stripMargin)
-    sql(
-      s"""LOAD DATA INPATH  '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table
-         | uniqdata partition(CUST_ID='1') OPTIONS('DELIMITER'=',','BAD_RECORDS_ACTION'='FORCE',
-         | 'QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""".stripMargin)
-    sql(
-      s"""LOAD DATA INPATH  '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table
-         | uniqdata partition(CUST_ID='1') OPTIONS('DELIMITER'=',','BAD_RECORDS_ACTION'='FORCE',
-         | 'QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""".stripMargin)
-    sql(
-      s"""LOAD DATA INPATH  '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table
-         | uniqdata partition(CUST_ID='1') OPTIONS('DELIMITER'=',','BAD_RECORDS_ACTION'='FORCE',
-         | 'QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""".stripMargin)
-    sql(
-      s"""LOAD DATA INPATH  '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table
-         | uniqdata partition(CUST_ID='1') OPTIONS('DELIMITER'=',','BAD_RECORDS_ACTION'='FORCE',
-         | 'QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""".stripMargin)
-    sql("create datamap ag1 on table uniqdata using 'preaggregate' as select cust_id, " +
-        "sum(Double_COLUMN1) from uniqdata group by cust_id")
-    sql(s"""alter table uniqdata compact 'minor'""")
-    assert(sql("show segments for table uniqdata").collect().tail(2).get(0).toString.equals("0.1"))
-  }
-
-  //Verify load with Pre_Aggregate table on Partition table after two-level compaction
-  test("Partition-With-PreAggregate_TC007", Include) {
-    sql("drop table if exists uniqdata")
-    sql(
-      s"""CREATE TABLE uniqdata (CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp,
-         |BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10),
-         |DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,
-         |INTEGER_COLUMN1 int, DOJ timestamp) PARTITIONED BY (CUST_ID int) STORED BY
-         |'carbondata'""".stripMargin)
-    sql(
-      s"""LOAD DATA INPATH  '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table
-         | uniqdata partition(CUST_ID='1') OPTIONS('DELIMITER'=',','BAD_RECORDS_ACTION'='FORCE',
-         | 'QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""".stripMargin)
-    sql(
-      s"""LOAD DATA INPATH  '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table
-         | uniqdata partition(CUST_ID='1') OPTIONS('DELIMITER'=',','BAD_RECORDS_ACTION'='FORCE',
-         | 'QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""".stripMargin)
-    sql(
-      s"""LOAD DATA INPATH  '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table
-         | uniqdata partition(CUST_ID='1') OPTIONS('DELIMITER'=',','BAD_RECORDS_ACTION'='FORCE',
-         | 'QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""".stripMargin)
-    sql(
-      s"""LOAD DATA INPATH  '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table
-         | uniqdata partition(CUST_ID='1') OPTIONS('DELIMITER'=',','BAD_RECORDS_ACTION'='FORCE',
-         | 'QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""".stripMargin)
-    sql(
-      s"""LOAD DATA INPATH  '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table
-         | uniqdata partition(CUST_ID='1') OPTIONS('DELIMITER'=',','BAD_RECORDS_ACTION'='FORCE',
-         | 'QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""".stripMargin)
-    sql(
-      s"""LOAD DATA INPATH  '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table
-         | uniqdata partition(CUST_ID='1') OPTIONS('DELIMITER'=',','BAD_RECORDS_ACTION'='FORCE',
-         | 'QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""".stripMargin)
-    sql(
-      s"""LOAD DATA INPATH  '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table
-         | uniqdata partition(CUST_ID='1') OPTIONS('DELIMITER'=',','BAD_RECORDS_ACTION'='FORCE',
-         | 'QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""".stripMargin)
-    sql(
-      s"""LOAD DATA INPATH  '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table
-         | uniqdata partition(CUST_ID='1') OPTIONS('DELIMITER'=',','BAD_RECORDS_ACTION'='FORCE',
-         | 'QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""".stripMargin)
-    sql("create datamap ag1 on table uniqdata using 'preaggregate' as select cust_id, " +
-        "sum(Double_COLUMN1) from uniqdata group by cust_id")
-    sql(s"""alter table uniqdata compact 'minor'""")
-    sql(s"""alter table uniqdata compact 'major'""")
-    assert(sql("show segments for table uniqdata").collect().tail(7).get(0).toString.equals("0.2"))
-    }
-
-  //Verify join operation on Partitonwith Pre-Aggregate table
-    test("Partition-With-PreAggregate_TC008", Include) {
-      sql("drop table if exists uniqdata")
-      sql(s"""drop table if exists uniqdata1""")
-    sql(
-      s"""CREATE TABLE uniqdata (CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp,
-         |BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10),
-         |DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,
-         |INTEGER_COLUMN1 int, DOJ timestamp) PARTITIONED BY (CUST_ID int) STORED BY 'carbondata'"""
-        .stripMargin)
-    sql(
-      s"""CREATE TABLE uniqdata1 (CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp,
-         |BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10),
-         |DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,
-         |INTEGER_COLUMN1 int, DOJ timestamp) PARTITIONED BY (CUST_ID int) STORED BY 'carbondata'"""
-        .stripMargin)
-    sql(
-      s"""LOAD DATA INPATH  '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table
-         | uniqdata partition(CUST_ID='1') OPTIONS('DELIMITER'=',','BAD_RECORDS_ACTION'='FORCE',
-         | 'QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""".stripMargin)
-    sql(
-      s"""LOAD DATA INPATH  '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table
-         | uniqdata1 partition(CUST_ID='1') OPTIONS('DELIMITER'=',','BAD_RECORDS_ACTION'='FORCE',
-         | 'QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""".stripMargin)
-    sql("create datamap ag1 on table uniqdata using 'preaggregate' as select cust_id, " +
-        "sum(Double_COLUMN1) from uniqdata group by cust_id")
-    sql("create datamap ag1 on table uniqdata1 using 'preaggregate' as select cust_id, " +
-        "sum(Double_COLUMN1) from uniqdata1 group by cust_id")
-    checkAnswer(sql(
-      s""" select a.cust_id, b.cust_id from uniqdata a, uniqdata1 b where
-         |a.cust_id >= b.cust_id limit 1""".stripMargin),Seq(Row(1,1)))
-    sql(s"""drop table if exists uniqdata1""")
-  }
-
-  //Verify date with > filter condition on Partitonwith Pre-Aggregate table
-  test("Partition-With-PreAggregate_TC009", Include) {
-    sql("drop table if exists uniqdata")
-    sql(
-      s"""CREATE TABLE uniqdata (CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp,
-         |BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10),
-         |DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,
-         |INTEGER_COLUMN1 int, DOJ timestamp) PARTITIONED BY (CUST_ID int) STORED BY 'carbondata'"""
-        .stripMargin)
-    sql("create datamap ag1 on table uniqdata using 'preaggregate' as select cust_id, " +
-        "sum(Double_COLUMN1) from uniqdata group by cust_id")
-    sql(
-      s"""LOAD DATA INPATH  '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table
-         | uniqdata partition(CUST_ID='4') OPTIONS('DELIMITER'=',' , 'BAD_RECORDS_ACTION'='FORCE',
-         | 'QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""".stripMargin)
-    checkAnswer(sql(
-      s"""select cust_name from uniqdata where active_emui_version =
-         |'ACTIVE_EMUI_VERSION_00000'""".stripMargin),Seq(Row("CUST_NAME_00000")))
-  }
-
-  override def afterAll {
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
-        CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT)
-      .addProperty(CarbonCommonConstants.CARBON_DATE_FORMAT,
-        CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT)
-    sql("drop table if exists uniqdata")
-    sql("drop table if exists partition_table")
-    sql(s"""drop table if exists uniqdata1""")
-
-  }
-}
\ No newline at end of file
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/PreAggregateTestCase.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/PreAggregateTestCase.scala
deleted file mode 100644
index c949be6..0000000
--- a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/PreAggregateTestCase.scala
+++ /dev/null
@@ -1,230 +0,0 @@
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.cluster.sdv.generated
-
-import org.apache.spark.sql.Row
-import org.apache.spark.sql.common.util.{Include, QueryTest}
-import org.junit.Assert
-import org.scalatest.BeforeAndAfterEach
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.util.CarbonProperties
-
-
-class PreAggregateTestCase extends QueryTest with BeforeAndAfterEach {
-  val csvPath = s"$resourcesPath/source.csv"
-
-  override def beforeEach: Unit = {
-    sql("drop table if exists PreAggMain")
-    sql("drop table if exists AggMain")
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.CARBON_DATE_FORMAT, "yyyy/MM/dd")
-    sql("CREATE TABLE PreAggMain (id Int, date date, country string, phonetype string, " +
-        "serialname String,salary int ) STORED BY 'org.apache.carbondata.format' " +
-        "tblproperties('dictionary_include'='country')")
-    sql("CREATE TABLE AggMain (id Int, date date, country string, phonetype string, " +
-        "serialname String,salary int ) STORED BY 'org.apache.carbondata.format'" +
-        "tblproperties('dictionary_include'='country')")
-    sql("create datamap PreAggSum on table PreAggMain using 'preaggregate' as " +
-        "select country,sum(salary) as sum from PreAggMain group by country")
-    sql("create datamap PreAggAvg on table PreAggMain using 'preaggregate' as " +
-        "select country,avg(salary) as avg from PreAggMain group by country")
-    sql("create datamap PreAggCount on table PreAggMain using 'preaggregate' as " +
-        "select country,count(salary) as count from PreAggMain group by country")
-    sql("create datamap PreAggMin on table PreAggMain using 'preaggregate' as " +
-        "select country,min(salary) as min from PreAggMain group by country")
-    sql("create datamap PreAggMax on table PreAggMain using 'preaggregate' as " +
-        "select country,max(salary) as max from PreAggMain group by country")
-  }
-
-  //test to check existence of datamap
-  test("PreAggregateTestCase_TC001", Include) {
-    Assert.assertEquals(sql("show datamap on table PreAggMain").count(), 5)
-    checkExistence(sql("Describe formatted PreAggMain_PreAggSum"), true, "Dictionary")
-  }
-
-  //check for load data should reflects in all preaggregate tables
-  test("PreAggregateTestCase_TC002", Include) {
-    sql(s"LOAD DATA INPATH '$csvPath' into table PreAggMain ").collect
-    sql(s"LOAD DATA INPATH '$csvPath' into table AggMain ").collect
-
-    val expectedSum = sql("select country,sum(salary) as sum from AggMain group by country")
-    checkAnswer(sql("select * from PreAggMain_PreAggSum"), expectedSum)
-
-    val expectedAvg = sql("select country,sum(salary),count(salary) from AggMain group by country")
-    checkAnswer(sql("select * from PreAggMain_PreAggAvg"), expectedAvg)
-
-    val expectedCount = sql("select country,count(salary) as count from AggMain group by country")
-    checkAnswer(sql("select * from PreAggMain_PreAggCount"), expectedCount)
-
-    val expectedMin = sql("select country,min(salary) as min from AggMain group by country")
-    checkAnswer(sql("select * from PreAggMain_PreAggMin"), expectedMin)
-
-    val expectedMax = sql("select country,max(salary) as max from AggMain group by country")
-    checkAnswer(sql("select * from PreAggMain_PreAggMax"), expectedMax)
-  }
-
-
-  //test for incremental load
-  test("PreAggregateTestCase_TC003", Include) {
-    sql(s"LOAD DATA INPATH '$csvPath' into table PreAggMain").collect
-    sql(s"LOAD DATA INPATH '$csvPath' into table PreAggMain").collect
-    sql(s"LOAD DATA INPATH '$csvPath' into table AggMain").collect
-
-    val expectedSum = sql("select country,sum(salary) as sum from " +
-                          "AggMain group by country")
-    checkAnswer(sql("select * from PreAggMain_PreAggSum"), expectedSum.union(expectedSum))
-
-    val expectedAvg = sql("select country,sum(salary),count(salary) from AggMain group by country")
-    checkAnswer(sql("select * from PreAggMain_PreAggAvg"), expectedAvg.union(expectedAvg))
-
-    val expectedCount = sql("select country,count(salary) as count from AggMain group by country")
-    checkAnswer(sql("select * from PreAggMain_PreAggCount"), expectedCount.union(expectedCount))
-
-    val expectedMin = sql("select country,min(salary) as min from AggMain group by country")
-    checkAnswer(sql("select * from PreAggMain_PreAggMin"), expectedMin.union(expectedMin))
-
-    val expectedMax = sql("select country,max(salary) as max from AggMain group by country")
-    checkAnswer(sql("select * from PreAggMain_PreAggMax"), expectedMax.union(expectedMax))
-  }
-
-  //test for creating datamap having data from all segment after incremental load
-  test("PreAggregateTestCase_TC004", Include) {
-    sql("insert into PreAggMain values(1,'2015/7/23','country1','phone197','ASD69643',15000)")
-    sql("insert into PreAggMain values(2,'2015/8/23','country2','phone197','ASD69643',10000)")
-    sql("insert into PreAggMain values(3,'2005/7/28','country1','phone197','ASD69643',5000)")
-    sql("create datamap testDataMap on table PreAggMain using 'preaggregate' as " +
-        "select country,sum(salary) as sum from PreAggMain group by country")
-    checkAnswer(sql("select * from PreAggMain_testDataMap"),
-      Seq(Row("country1", 20000), Row("country2", 10000)))
-  }
-
-  //test for insert overwrite in main table
-  test("PreAggregateTestCase_TC005", Include) {
-    sql("insert into PreAggMain values(1,'2015/7/23','country1','phone197','ASD696',15000)")
-    sql("insert into PreAggMain values(2,'2003/8/13','country2','phone197','AD6943',10000)")
-    sql("insert overwrite table PreAggMain values(3,'2005/7/28','country3','phone197','ASD69643',5000)")
-    sql("create datamap testDataMap on table PreAggMain using 'preaggregate' as " +
-        "select country,sum(salary) as sum from PreAggMain group by country")
-    checkAnswer(sql("select * from PreAggMain_testDataMap"), Seq(Row("country3", 5000)))
-  }
-
-  // test output for join query with preaggregate and without preaggregate
-  test("PreAggregateTestCase_TC006", Include) {
-    sql(s"LOAD DATA INPATH '$csvPath' into table PreAggMain").collect
-    sql(s"LOAD DATA INPATH '$csvPath' into table AggMain").collect
-    val actual = sql("select t1.country,sum(id) from PreAggMain t1 join " +
-                     "(select country as newcountry,sum(salary) as sum from PreAggMain group by " +
-                     "country )t2 on t1.country=t2.newcountry group by country")
-
-    val expected = sql("select t1.country,sum(id) from AggMain t1 join " +
-                       "(select country as newcountry,sum(salary) as sum from AggMain group by " +
-                       "country )t2 on t1.country=t2.newcountry group by country")
-    checkAnswer(actual, expected)
-  }
-
-  test("PreAggregateTestCase_TC007", Include) {
-    sql(s"LOAD DATA INPATH '$csvPath' into table PreAggMain").collect
-    sql(s"LOAD DATA INPATH '$csvPath' into table AggMain").collect
-    val actual = sql("select t1.country,count(t1.country) from PreAggMain t1 join " +
-                     " (select country,count(salary) as count from PreAggMain group by country )" +
-                     "t2 on t1.country=t2.country group by t1.country")
-
-    val expected = sql("select t1.country,count(t1.country) from AggMain t1 join " +
-                       "(select country,count(salary) as count from AggMain group by country )t2 " +
-                       "on t1.country=t2.country group by t1.country")
-    checkAnswer(actual, expected)
-  }
-
-  //test to verify correct data in preaggregate table
-  test("PreAggregateTestCase_TC008", Include) {
-    sql(s"LOAD DATA INPATH '$csvPath' into table PreAggMain").collect
-    sql(s"LOAD DATA INPATH '$csvPath' into table AggMain").collect
-    sql("create datamap testDatamap on table PreAggMain using 'preaggregate' as " +
-        "select sum(CASE WHEN country='china' THEN id ELSE 0 END) as sum,country from " +
-        "PreAggMain group by country")
-    val actual = sql("select * from PreAggMain_testDatamap")
-    val expected = sql(
-      "select sum(CASE WHEN country='china' THEN id ELSE 0 END) as sum,country " +
-      "from AggMain group by country")
-    checkAnswer(actual, expected)
-  }
-
-  //test for select using in clause in preaggregate table
-  test("PreAggregateTestCase_TC009", Include) {
-    sql(s"LOAD DATA INPATH '$csvPath' into table PreAggMain").collect
-    sql(s"LOAD DATA INPATH '$csvPath' into table AggMain").collect
-    sql("create datamap testDatamap on table PreAggMain using 'preaggregate' as " +
-        "select sum(CASE WHEN id in (10,11,12) THEN salary ELSE 0 END) as sum from PreAggMain " +
-        "group by country")
-    val actual = sql("select * from PreAggMain_testDatamap")
-    val expected = sql(
-      "select sum(CASE WHEN id in (10,11,12) THEN salary ELSE 0 END) as sum,country from AggMain " +
-      "group by country")
-    checkAnswer(actual, expected)
-  }
-
-  //test to check data using preaggregate and without preaggregate
-  test("PreAggregateTestCase_TC010", Include) {
-    sql(s"LOAD DATA INPATH '$csvPath' into table PreAggMain").collect
-    sql(s"LOAD DATA INPATH '$csvPath' into table AggMain").collect
-
-    val actual = sql(
-      "select count(CASE WHEN country='usa' THEN id ELSE 0 END) as count,country from PreAggMain " +
-      "group by country")
-    val expected = sql(
-      "select count(CASE WHEN country='usa' THEN id ELSE 0 END) as count,country from AggMain group" +
-      " by country")
-    checkAnswer(actual, expected)
-  }
-
-  //test to check data using preaggregate and without preaggregate with in clause
-  test("PreAggregateTestCase_TC011", Include) {
-    sql(s"LOAD DATA INPATH '$csvPath' into table PreAggMain").collect
-    sql(s"LOAD DATA INPATH '$csvPath' into table AggMain").collect
-    sql("create datamap testDatamap on table PreAggMain using 'preaggregate' as " +
-        "select sum(CASE WHEN id in (12,13,14) THEN salary ELSE 0 END) as sum from PreAggMain " +
-        "group by country")
-    val actual = sql(
-      "select sum(CASE WHEN id in (12,13,14) THEN salary ELSE 0 END) as sum,country from " +
-      "PreAggMain group by country")
-    val expected = sql(
-      "select sum(CASE WHEN id in (12,13,14) THEN salary ELSE 0 END) as sum,country from AggMain " +
-      "group by country")
-    checkAnswer(actual, expected)
-  }
-
-  test("Test CleanUp of Pre_aggregate tables") {
-    sql("drop table if exists maintable")
-    sql("create table maintable(name string, c_code int, price int) stored by 'carbondata'")
-    sql("insert into table maintable select 'abc',21,2000")
-    sql("create datamap ag1 on table maintable using 'preaggregate' as select name,sum(price) from maintable group by name")
-    sql("insert into table maintable select 'abcd',22,3000")
-    sql("insert into table maintable select 'abcd',22,3000")
-    sql("insert into table maintable select 'abcd',22,3000")
-    sql("alter table maintable compact 'minor'")
-    sql("clean files for table maintable")
-    assert(sql("show segments for table maintable").collect().head.get(0).toString.contains("0.1"))
-  }
-
-  override def afterEach: Unit = {
-    sql("drop table if exists mainTable")
-  }
-
-}
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/TimeSeriesPreAggregateTestCase.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/TimeSeriesPreAggregateTestCase.scala
deleted file mode 100644
index ab05a84..0000000
--- a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/TimeSeriesPreAggregateTestCase.scala
+++ /dev/null
@@ -1,192 +0,0 @@
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.cluster.sdv.generated
-
-import java.util.TimeZone
-
-import org.apache.spark.sql.test.util.QueryTest
-import org.scalatest.BeforeAndAfterAll
-import org.scalatest.Matchers._
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.metadata.schema.datamap.DataMapClassProvider.TIMESERIES
-import org.apache.carbondata.core.util.CarbonProperties
-
-class TimeSeriesPreAggregateTestCase extends QueryTest with BeforeAndAfterAll {
-
-  val timeSeries = TIMESERIES.toString
-  val timeZonePre = TimeZone.getDefault
-  val csvPath = s"$resourcesPath/Data/timeseriestest.csv"
-  override def beforeAll: Unit = {
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
-        CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT)
-    TimeZone.setDefault(TimeZone.getTimeZone(System.getProperty("user.timezone")))
-    sql("drop table if exists mainTable")
-    sql(
-      "CREATE TABLE mainTable(mytime timestamp, name string, age int) STORED BY 'org.apache" +
-      ".carbondata.format'")
-    sql(
-      s"""
-         | CREATE DATAMAP agg0_second ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         | 'EVENT_TIME'='mytime',
-         | 'SECOND_GRANULARITY'='1')
-         | AS SELECT mytime, SUM(age) FROM mainTable
-         | GROUP BY mytime
-       """.stripMargin)
-    sql(
-      s"""
-         | CREATE DATAMAP agg0_minute ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         | 'EVENT_TIME'='mytime',
-         | 'minute_granularity'='1')
-         | AS SELECT mytime, SUM(age) FROM mainTable
-         | GROUP BY mytime
-       """.stripMargin)
-    sql(
-      s"""
-         | CREATE DATAMAP agg0_hour ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         | 'EVENT_TIME'='mytime',
-         | 'HOUR_GRANULARITY'='1')
-         | AS SELECT mytime, SUM(age) FROM mainTable
-         | GROUP BY mytime
-       """.stripMargin)
-    sql(
-      s"""
-         | CREATE DATAMAP agg0_day ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         | 'EVENT_TIME'='mytime',
-         | 'DAY_GRANULARITY'='1')
-         | AS SELECT mytime, SUM(age) FROM mainTable
-         | GROUP BY mytime
-       """.stripMargin)
-    sql(
-      s"""
-         | CREATE DATAMAP agg0_month ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         | 'EVENT_TIME'='mytime',
-         | 'MONTH_GRANULARITY'='1')
-         | AS SELECT mytime, SUM(age) FROM mainTable
-         | GROUP BY mytime
-       """.stripMargin)
-    sql(
-      s"""
-         | CREATE DATAMAP agg0_year ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         | 'EVENT_TIME'='mytime',
-         | 'year_granularity'='1')
-         | AS SELECT mytime, SUM(age) FROM mainTable
-         | GROUP BY mytime
-       """.stripMargin)
-
-    sql(s"LOAD DATA LOCAL INPATH '$csvPath' into table mainTable")
-  }
-
-  test("TimeSeriesPreAggregateTestCase_001") {
-    val expected = sql("select cast(date_format(mytime, 'YYYY') as timestamp) as mytime,sum(age) " +
-                       "from mainTable group by date_format(mytime , 'YYYY')")
-    val actual = sql("select * from maintable_agg0_year")
-    checkAnswer(actual, expected)
-  }
-
-  test("TimeSeriesPreAggregateTestCase_002") {
-    val expected = sql(
-      "select cast(date_format(mytime, 'YYYY-MM') as timestamp) as mytime,sum(age) " +
-      "from mainTable group by date_format(mytime , 'YYYY-MM')")
-    val actual = sql("select * from maintable_agg0_month")
-    checkAnswer(actual, expected)
-  }
-
-  test("TimeSeriesPreAggregateTestCase_003") {
-    val expected = sql(
-      "select cast(date_format(mytime, 'YYYY-MM-dd') as timestamp) as mytime,sum(age) " +
-      "from mainTable group by date_format(mytime , 'YYYY-MM-dd')")
-    val actual = sql("select * from maintable_agg0_day")
-    checkAnswer(actual, expected)
-  }
-
-  test("TimeSeriesPreAggregateTestCase_004") {
-    val expected = sql(
-      "select cast(date_format(mytime, 'YYYY-MM-dd HH') as timestamp) as mytime,sum(age) " +
-      "from mainTable group by date_format(mytime , 'YYYY-MM-dd HH')")
-    val actual = sql("select * from maintable_agg0_hour")
-    checkAnswer(actual, expected)
-  }
-
-  test("TimeSeriesPreAggregateTestCase_005") {
-    val expected = sql(
-      "select cast(date_format(mytime, 'YYYY-MM-dd HH:mm') as timestamp) as mytime,sum(age) " +
-      "from mainTable group by date_format(mytime , 'YYYY-MM-dd HH:mm')")
-    val actual = sql("select * from maintable_agg0_minute")
-    checkAnswer(actual, expected)
-  }
-
-  test("TimeSeriesPreAggregateTestCase_006") {
-    val expected = sql(
-      "select cast(date_format(mytime, 'YYYY-MM-dd HH:mm:ss') as timestamp) as mytime,sum(age) " +
-      "from mainTable group by date_format(mytime , 'YYYY-MM-dd HH:mm:ss')")
-    val actual = sql("select * from maintable_agg0_second")
-    checkAnswer(actual, expected)
-  }
-
-  //test case for compaction
-  test("TimeSeriesPreAggregateTestCase_007") {
-    sql(s"LOAD DATA LOCAL INPATH '$csvPath' into table mainTable")
-    sql(s"LOAD DATA LOCAL INPATH '$csvPath' into table mainTable")
-    sql(s"LOAD DATA LOCAL INPATH '$csvPath' into table mainTable")
-    sql("alter table maintable compact 'minor'")
-    val segmentNamesSecond = sql("show segments for table maintable_agg0_second").collect()
-      .map(_.get(0).toString)
-    segmentNamesSecond should equal(Array("3", "2", "1", "0.1", "0"))
-
-    val segmentNamesMinute = sql("show segments for table maintable_agg0_minute").collect()
-      .map(_.get(0).toString)
-    segmentNamesMinute should equal(Array("3", "2", "1", "0.1", "0"))
-
-    val segmentNamesHour = sql("show segments for table maintable_agg0_hour").collect()
-      .map(_.get(0).toString)
-    segmentNamesHour should equal(Array("3", "2", "1", "0.1", "0"))
-
-    val segmentNamesday = sql("show segments for table maintable_agg0_day").collect()
-      .map(_.get(0).toString)
-    segmentNamesday should equal(Array("3", "2", "1", "0.1", "0"))
-
-    val segmentNamesmonth = sql("show segments for table maintable_agg0_month").collect()
-      .map(_.get(0).toString)
-    segmentNamesmonth should equal(Array("3", "2", "1", "0.1", "0"))
-
-    val segmentNamesyear = sql("show segments for table maintable_agg0_year").collect()
-      .map(_.get(0).toString)
-    segmentNamesyear should equal(Array("3", "2", "1", "0.1", "0"))
-  }
-
-  override def afterAll: Unit = {
-    TimeZone.setDefault(timeZonePre)
-    sql("drop table if exists mainTable")
-
-  }
-}
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/register/TestRegisterCarbonTable.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/register/TestRegisterCarbonTable.scala
index caae8e1..7d93f15 100644
--- a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/register/TestRegisterCarbonTable.scala
+++ b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/register/TestRegisterCarbonTable.scala
@@ -109,73 +109,6 @@
     checkAnswer(sql("select c1 from carbontable"), Seq(Row("a")))
   }
 
-   test("register pre aggregate tables test") {
-    sql("drop database if exists carbon cascade")
-    sql(s"create database carbon location '$dbLocationCustom'")
-    sql("use carbon")
-    sql("""create table carbon.carbontable (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
-    sql("insert into carbontable select 'a',1,'aa','aaa'")
-    sql("insert into carbontable select 'b',1,'aa','aaa'")
-    sql("insert into carbontable select 'a',10,'aa','aaa'")
-    sql("create datamap preagg1 on table carbontable using 'preaggregate' as select c1,sum(c2) from carbontable group by c1")
-    if (!CarbonEnv.getInstance(sqlContext.sparkSession).carbonMetaStore.isReadFromHiveMetaStore) {
-      backUpData(dbLocationCustom, "carbontable")
-      backUpData(dbLocationCustom, "carbontable_preagg1")
-      sql("drop table carbontable")
-      restoreData(dbLocationCustom, "carbontable")
-      restoreData(dbLocationCustom, "carbontable_preagg1")
-      sql("refresh table carbontable")
-    }
-     checkAnswer(sql("select count(*) from carbontable"), Row(3))
-     checkAnswer(sql("select c1 from carbontable"), Seq(Row("a"), Row("b"), Row("a")))
-     checkAnswer(sql("select count(*) from carbontable_preagg1"), Row(2))
-     checkAnswer(sql("select carbontable_c1 from carbontable_preagg1"), Seq(Row("a"), Row("b")))
-   }
-
-  test("register pre aggregate table test") {
-    sql("drop database if exists carbon cascade")
-    sql(s"create database carbon location '$dbLocationCustom'")
-    sql("use carbon")
-    sql("""create table carbon.carbontable (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
-    sql("insert into carbontable select 'a',1,'aa','aaa'")
-    sql("insert into carbontable select 'b',1,'aa','aaa'")
-    sql("insert into carbontable select 'a',10,'aa','aaa'")
-    sql("create datamap preagg1 on table carbontable using 'preaggregate' as select c1,sum(c2) from carbontable group by c1")
-    if (!CarbonEnv.getInstance(sqlContext.sparkSession).carbonMetaStore.isReadFromHiveMetaStore) {
-      backUpData(dbLocationCustom, "carbontable")
-      backUpData(dbLocationCustom, "carbontable_preagg1")
-      sql("drop table carbontable")
-      restoreData(dbLocationCustom, "carbontable")
-      restoreData(dbLocationCustom, "carbontable_preagg1")
-      sql("refresh table carbontable")
-    }
-    checkAnswer(sql("select count(*) from carbontable"), Row(3))
-    checkAnswer(sql("select c1 from carbontable"), Seq(Row("a"), Row("b"), Row("a")))
-    checkAnswer(sql("select count(*) from carbontable_preagg1"), Row(2))
-    checkAnswer(sql("select carbontable_c1 from carbontable_preagg1"), Seq(Row("a"), Row("b")))
-  }
-
-  test("register pre aggregate table should fail if the aggregate table not copied") {
-    sql("drop database if exists carbon cascade")
-    sql(s"create database carbon location '$dbLocationCustom'")
-    sql("use carbon")
-    sql("""create table carbon.carbontable (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
-    sql("insert into carbontable select 'a',1,'aa','aaa'")
-    sql("insert into carbontable select 'b',1,'aa','aaa'")
-    sql("insert into carbontable select 'a',10,'aa','aaa'")
-    sql("create datamap preagg1 on table carbontable using 'preaggregate' as select c1,sum(c2) from carbontable group by c1")
-    if (!CarbonEnv.getInstance(sqlContext.sparkSession).carbonMetaStore.isReadFromHiveMetaStore) {
-      backUpData(dbLocationCustom, "carbontable")
-      backUpData(dbLocationCustom, "carbontable_preagg1")
-      sql("drop table carbontable")
-      restoreData(dbLocationCustom, "carbontable")
-      intercept[ProcessMetaDataException] {
-        sql("refresh table carbontable")
-      }
-      restoreData(dbLocationCustom, "carbontable_preagg1")
-    }
-  }
-
   test("Update operation on carbon table should pass after registration or refresh") {
     sql("drop database if exists carbon cascade")
     sql(s"create database carbon location '$dbLocationCustom'")
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 dd32356..1d14d06 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
@@ -57,10 +57,7 @@
                new StandardPartitionTestCase ::
                new Vector1TestCase ::
                new Vector2TestCase ::
-               new PreAggregateTestCase ::
-               new TimeSeriesPreAggregateTestCase ::
                new TestPartitionWithGlobalSort ::
-               new PartitionWithPreAggregateTestCase ::
                new CreateTableWithLocalDictionaryTestCase ::
                new LoadTableWithLocalDictionaryTestCase :: Nil
 
@@ -157,13 +154,10 @@
                new TestRegisterCarbonTable ::
                new TableCommentAlterTableTestCase ::
                new StandardPartitionTestCase ::
-               new PreAggregateTestCase ::
                new LuceneTestCase ::
-               new TimeSeriesPreAggregateTestCase ::
                new TestPartitionWithGlobalSort ::
                new SDKwriterTestCase ::
                new SetParameterTestCase ::
-               new PartitionWithPreAggregateTestCase ::
                new CreateTableWithLocalDictionaryTestCase ::
                new LoadTableWithLocalDictionaryTestCase :: Nil
 
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMapSuite.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMapSuite.scala
index 9c0ab89..09a93c3 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMapSuite.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMapSuite.scala
@@ -471,24 +471,6 @@
     sql("drop datamap if exists dm on table datamap_test_table")
   }
 
-  test("test lucene fine grain data map for show datamaps with Preaggregate and Lucene") {
-    sql("DROP TABLE IF EXISTS datamap_main")
-    sql("create table datamap_main (a string, b string, c string) stored by 'carbondata'")
-    sql(
-      s"""
-         | CREATE DATAMAP dm_lucene ON TABLE datamap_main
-         | USING 'lucene'
-         | DMProperties('INDEX_COLUMNS'='c')
-      """.stripMargin)
-    sql(
-      "create datamap dm_pre on table datamap_main USING 'preaggregate' as select a,sum(b) " +
-      "from datamap_main group by a")
-    checkExistence(sql("show datamap on table datamap_main"), true, "dm_pre")
-    checkExistence(sql("show datamap on table datamap_main"), true, "dm_lucene")
-    sql("drop datamap if exists dm_pre on table datamap_main")
-    sql("drop datamap if exists dm_lucene on table datamap_main")
-  }
-
   test("test lucene fine grain data map with CTAS") {
     sql("DROP TABLE IF EXISTS source_table")
     sql("DROP TABLE IF EXISTS target_table")
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/complexType/TestComplexDataType.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/complexType/TestComplexDataType.scala
index 4fe0c46..bfc1935 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/complexType/TestComplexDataType.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/complexType/TestComplexDataType.scala
@@ -827,23 +827,6 @@
     assertResult("Cannot use struct<b:int> for partition column;")(structException.getMessage)
   }
 
-  test("test block preaggregate") {
-    sql("DROP TABLE IF EXISTS test")
-    sql("create table test(id int,a struct<b:int>) stored by 'carbondata'")
-    sql("insert into test values(1, named_struct('b', 2))")
-    sql("insert into test values(1, named_struct('b', 2))")
-    sql("insert into test values(1, named_struct('b', 2))")
-    val structException = intercept[UnsupportedOperationException](
-      sql("create datamap preagg_sum on table test using 'preaggregate' as select id,sum(a.b) from test group by id"))
-    assertResult("Preaggregate is unsupported for ComplexData type column: a.b")(structException.getMessage)
-    sql("DROP TABLE IF EXISTS test")
-    sql("create table test(id int,a array<int>) stored by 'carbondata'")
-    sql("insert into test values(1, array(2))")
-    val arrayException = intercept[UnsupportedOperationException](
-      sql("create datamap preagg_sum on table test using 'preaggregate' as select id,sum(a[0]) from test group by id"))
-    assertResult("Preaggregate is unsupported for ComplexData type column: a[0]")(arrayException.getMessage)
-  }
-
   test("test block dictionary exclude for child column") {
     sql("DROP TABLE IF EXISTS table1")
     sql(
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
deleted file mode 100644
index 5aecb08..0000000
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggCreateCommand.scala
+++ /dev/null
@@ -1,534 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.integration.spark.testsuite.preaggregate
-
-import java.io.File
-import java.util
-import java.util.concurrent.{Callable, ExecutorService, Executors, TimeUnit}
-
-import scala.collection.JavaConverters._
-
-import org.apache.spark.sql.{AnalysisException, CarbonDatasourceHadoopRelation, Row}
-import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
-import org.apache.spark.sql.execution.datasources.LogicalRelation
-import org.apache.spark.sql.hive.CarbonRelation
-import org.apache.spark.sql.test.util.QueryTest
-import org.scalatest.BeforeAndAfterAll
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.common.exceptions.sql.{MalformedCarbonCommandException, MalformedDataMapCommandException}
-import org.apache.carbondata.core.metadata.encoder.Encoding
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable
-import org.apache.carbondata.core.metadata.schema.datamap.DataMapClassProvider.TIMESERIES
-import org.apache.carbondata.core.util.CarbonProperties
-
-class TestPreAggCreateCommand extends QueryTest with BeforeAndAfterAll {
-
-  val timeSeries = TIMESERIES.toString
-
-  override def beforeAll {
-    sql("drop database if exists otherDB cascade")
-    sql("drop table if exists PreAggMain")
-    sql("drop table if exists PreAggMain1")
-    sql("drop table if exists maintable")
-    sql("drop table if exists showTables")
-    sql("drop table if exists Preagg_twodb")
-    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 maintable (column1 int, column6 string, column5 string, column2 string, column3 int, column4 int) stored by 'carbondata' tblproperties('dictionary_include'='column1,column6', 'dictionary_exclude'='column3,column5')")
-  }
-
-  test("test pre agg create table 1") {
-    sql("create datamap preagg1 on table PreAggMain using 'preaggregate' as select a,sum(b) from PreAggMain group by a")
-    checkExistence(sql("DESCRIBE FORMATTED PreAggMain_preagg1"), true, "preaggmain_a")
-    checkExistence(sql("DESCRIBE FORMATTED PreAggMain_preagg1"), true, "preaggmain_b_sum")
-    sql("drop datamap preagg1 on table PreAggMain")
-  }
-
-  test("test pre agg create table 2") {
-    dropDataMaps("PreAggMain", "preagg2")
-    sql("create datamap preagg2 on table PreAggMain using 'preaggregate' as select a as a1,sum(b) as udfsum from PreAggMain group by a")
-    checkExistence(sql("DESCRIBE FORMATTED PreAggMain_preagg2"), true, "preaggmain_a")
-    checkExistence(sql("DESCRIBE FORMATTED PreAggMain_preagg2"), true, "preaggmain_b_sum")
-    checkExistence(sql("DESCRIBE FORMATTED PreAggMain_preagg2"), false, "preaggmain_a1")
-    checkExistence(sql("DESCRIBE FORMATTED PreAggMain_preagg2"), false, "preaggmain_udfsum")
-    sql("drop datamap preagg2 on table PreAggMain")
-  }
-
-  test("test pre agg create table 5") {
-    sql("create datamap preagg11 on table PreAggMain1 using 'preaggregate'as select a,sum(b) from PreAggMain1 group by a")
-    checkExistence(sql("DESCRIBE FORMATTED PreAggMain1_preagg11"), true, "preaggmain1_a")
-    checkExistence(sql("DESCRIBE FORMATTED PreAggMain1_preagg11"), true, "preaggmain1_b_sum")
-    sql("DESCRIBE FORMATTED PreAggMain1_preagg11").show(100, false)
-    checkExistence(sql("DESCRIBE FORMATTED PreAggMain1_preagg11"), true, "Dictionary")
-    sql("drop datamap preagg11 on table PreAggMain1")
-  }
-
-  test("test pre agg create table 6") {
-    sql("create datamap preagg12 on table PreAggMain1 using 'preaggregate' as select a as a1,sum(b) as sum from PreAggMain1 group by a")
-    checkExistence(sql("DESCRIBE FORMATTED PreAggMain1_preagg12"), true, "preaggmain1_a")
-    checkExistence(sql("DESCRIBE FORMATTED PreAggMain1_preagg12"), true, "preaggmain1_b_sum")
-    checkExistence(sql("DESCRIBE FORMATTED PreAggMain1_preagg12"), false, "preaggmain1_a1")
-    checkExistence(sql("DESCRIBE FORMATTED PreAggMain1_preagg12"), false, "preaggmain1_sum")
-    checkExistence(sql("DESCRIBE FORMATTED PreAggMain1_preagg12"), true, "Dictionary")
-    sql("drop datamap preagg12 on table PreAggMain1")
-  }
-
-  test("test pre agg create table 8") {
-    sql("create datamap preagg14 on table PreAggMain1 using 'preaggregate' as select a as a1,sum(b) as sum from PreAggMain1 group by a")
-    checkExistence(sql("DESCRIBE FORMATTED PreAggMain1_preagg14"), true, "preaggmain1_a")
-    checkExistence(sql("DESCRIBE FORMATTED PreAggMain1_preagg14"), true, "preaggmain1_b_sum")
-    checkExistence(sql("DESCRIBE FORMATTED PreAggMain1_preagg14"), false, "preaggmain1_a1")
-    checkExistence(sql("DESCRIBE FORMATTED PreAggMain1_preagg14"), false, "preaggmain1_sum")
-    checkExistence(sql("DESCRIBE FORMATTED PreAggMain1_preagg14"), true, "Dictionary")
-    sql("drop datamap preagg14 on table PreAggMain1")
-  }
-
-  test("test pre agg create table 9") {
-    sql("create datamap preagg15 on table PreAggMain using 'preaggregate' as select a,avg(b) from PreAggMain group by a")
-    checkExistence(sql("DESCRIBE FORMATTED PreAggMain_preagg15"), true, "preaggmain_a")
-    checkExistence(sql("DESCRIBE FORMATTED PreAggMain_preagg15"), true, "preaggmain_b_sum")
-    checkExistence(sql("DESCRIBE FORMATTED PreAggMain_preagg15"), true, "preaggmain_b_count")
-    checkExistence(sql("DESCRIBE FORMATTED PreAggMain_preagg15"), false, "preaggmain2_b_avg")
-    sql("drop datamap preagg15 on table PreAggMain")
-  }
-
-  test("test pre agg create table 10") {
-    sql("create datamap preagg16 on table PreAggMain using 'preaggregate' as select a as a1,max(b) from PreAggMain group by a")
-    checkExistence(sql("DESCRIBE FORMATTED PreAggMain_preagg16"), true, "preaggmain_a")
-    checkExistence(sql("DESCRIBE FORMATTED PreAggMain_preagg16"), true, "preaggmain_b_max")
-    sql("drop datamap preagg16 on table PreAggMain")
-  }
-
-  test("test pre agg create table 11") {
-    sql("create datamap preagg17 on table PreAggMain using 'preaggregate' as select a,min(b) from PreAggMain group by a")
-    checkExistence(sql("DESCRIBE FORMATTED PreAggMain_preagg17"), true, "preaggmain_a")
-    checkExistence(sql("DESCRIBE FORMATTED PreAggMain_preagg17"), true, "preaggmain_b_min")
-    sql("drop datamap preagg17 on table PreAggMain")
-  }
-
-  test("test pre agg create table 12") {
-    sql("create datamap preagg18 on table PreAggMain using 'preaggregate' as select a as a1,count(b) from PreAggMain group by a")
-    checkExistence(sql("DESCRIBE FORMATTED PreAggMain_preagg18"), true, "preaggmain_a")
-    checkExistence(sql("DESCRIBE FORMATTED PreAggMain_preagg18"), true, "preaggmain_b_count")
-    sql("drop datamap preagg18 on table PreAggMain")
-  }
-
-  test("test pre agg create table 13") {
-    val exception: Exception = intercept[MalformedCarbonCommandException] {
-      sql(
-        s"""
-           | create datamap preagg19 on table PreAggMain
-           | using 'preaggregate'
-           | as select a as a1,count(distinct b)
-           | from PreAggMain group by a
-         """.stripMargin)
-    }
-    assert(exception.getMessage.equals("Distinct is not supported On Pre Aggregation"))
-  }
-
-  test("test pre agg create table 14") {
-    val exception = intercept[MalformedCarbonCommandException] {
-      sql(
-        s"""
-           | create datamap preagg20 on table PreAggMain
-           | using 'preaggregate'
-           | as select a as a1,sum(distinct b) from PreAggMain
-           | group by a
-         """.stripMargin)
-    }
-    assert(exception.getMessage.equals("Distinct is not supported On Pre Aggregation"))
-  }
-
-  test("test pre agg create table 15: don't support where") {
-    intercept[Exception] {
-      sql(
-        s"""
-           | create datamap preagg21 on table PreAggMain
-           | using 'preaggregate'
-           | as select a as a1,sum(b)
-           | from PreAggMain
-           | where a='vishal'
-           | group by a
-         """.stripMargin)
-    }
-  }
-
-  test("test pre agg create table 16") {
-    sql("create datamap agg0 on table mainTable using 'preaggregate' as select column4, sum(column4) from maintable group by column4")
-    val df = sql("select * from maintable_agg0")
-    val carbontable = getCarbonTable(df.queryExecution.analyzed)
-    assert(carbontable.getAllMeasures.size()==2)
-    assert(carbontable.getAllDimensions.size()==0)
-    sql("drop datamap agg0 on table maintable")
-  }
-
-  test("test pre agg create table 17") {
-    sql("create datamap agg0 on table mainTable using 'preaggregate' as select column1, sum(column1),column6, sum(column6) from maintable group by column6,column1")
-    val df = sql("select * from maintable_agg0")
-    val carbontable = getCarbonTable(df.queryExecution.analyzed)
-    assert(carbontable.getAllMeasures.size()==2)
-    assert(carbontable.getAllDimensions.size()==2)
-    carbontable.getAllDimensions.asScala.foreach{ f =>
-      assert(f.getEncoder.contains(Encoding.DICTIONARY))
-    }
-    sql("drop datamap agg0 on table maintable")
-  }
-
-  test("test pre agg create table 18") {
-    sql("create datamap agg0 on table mainTable using 'preaggregate' as select column1, count(column1),column6, count(column6) from maintable group by column6,column1")
-    val df = sql("select * from maintable_agg0")
-    val carbontable = getCarbonTable(df.queryExecution.analyzed)
-    assert(carbontable.getAllMeasures.size()==2)
-    assert(carbontable.getAllDimensions.size()==2)
-    carbontable.getAllDimensions.asScala.foreach{ f =>
-      assert(f.getEncoder.contains(Encoding.DICTIONARY))
-    }
-    sql("drop datamap agg0 on table maintable")
-  }
-
-  test("test pre agg create table 19") {
-    sql("create datamap agg0 on table mainTable using 'preaggregate' as select column3, sum(column3),column5, sum(column5) from maintable group by column3,column5")
-    val df = sql("select * from maintable_agg0")
-    val carbontable = getCarbonTable(df.queryExecution.analyzed)
-    assert(carbontable.getAllMeasures.size()==2)
-    assert(carbontable.getAllDimensions.size()==2)
-    carbontable.getAllDimensions.asScala.foreach{ f =>
-      assert(!f.getEncoder.contains(Encoding.DICTIONARY))
-    }
-    sql("drop datamap agg0 on table maintable")
-  }
-
-  test("test pre agg create table 20") {
-    sql("create datamap agg0 on table mainTable using 'preaggregate' as select column3, sum(column3),column5, sum(column5) from maintable group by column3,column5,column2")
-    val df = sql("select * from maintable_agg0")
-    val carbontable = getCarbonTable(df.queryExecution.analyzed)
-    assert(carbontable.getAllMeasures.size()==2)
-    assert(carbontable.getAllDimensions.size()==3)
-    carbontable.getAllDimensions.asScala.foreach{ f =>
-      assert(!f.getEncoder.contains(Encoding.DICTIONARY))
-    }
-    sql("drop datamap agg0 on table maintable")
-  }
-
-  test("remove agg tables from show table command") {
-    CarbonProperties.getInstance().addProperty(CarbonCommonConstants.CARBON_SHOW_DATAMAPS,"false")
-    sql("DROP TABLE IF EXISTS tbl_1")
-    sql("DROP TABLE IF EXISTS sparktable")
-    sql("create table if not exists  tbl_1(imei string,age int,mac string ,prodate timestamp,update timestamp,gamepoint double,contrid double) stored by 'carbondata' ")
-    sql("create table if not exists sparktable(a int,b string)")
-    sql(
-      s"""create datamap preagg_sum on table tbl_1 using 'preaggregate' as select mac,avg(age) from tbl_1 group by mac"""
-        .stripMargin)
-    sql(
-      "create datamap agg2 on table tbl_1 using 'preaggregate' as select prodate, mac from tbl_1 group by prodate,mac")
-    checkExistence(sql("show tables"), false, "tbl_1_preagg_sum","tbl_1_agg2_day","tbl_1_agg2_hour","tbl_1_agg2_month","tbl_1_agg2_year")
-    CarbonProperties.getInstance().addProperty(CarbonCommonConstants.CARBON_SHOW_DATAMAPS,CarbonCommonConstants.CARBON_SHOW_DATAMAPS_DEFAULT)
-  }
-
-  test("test pre agg create table 22: create with preaggregate and granularity") {
-    sql("DROP TABLE IF EXISTS maintabletime")
-    sql(
-      """
-        | CREATE TABLE maintabletime(year INT,month INT,name STRING,salary INT,dob STRING)
-        | STORED BY 'carbondata'
-        | TBLPROPERTIES(
-        |   'SORT_SCOPE'='Global_sort',
-        |   'TABLE_BLOCKSIZE'='23',
-        |   'SORT_COLUMNS'='month,year,name')
-      """.stripMargin)
-    sql("INSERT INTO maintabletime SELECT 10,11,'x',12,'2014-01-01 00:00:00'")
-    sql(
-      s"""
-         | CREATE DATAMAP agg0 ON TABLE maintabletime
-         | USING 'preaggregate'
-         | AS SELECT dob,name FROM maintabletime
-         | GROUP BY dob,name
-       """.stripMargin)
-    val e = intercept[MalformedCarbonCommandException] {
-      sql(
-        s"""
-           | CREATE DATAMAP agg1 ON TABLE maintabletime
-           | USING 'preaggregate'
-           | DMPROPERTIES (
-           |  'EVENT_TIME'='dob',
-           |  'SECOND_GRANULARITY'='1')
-           | AS SELECT dob,name FROM maintabletime
-           | GROUP BY dob,name
-       """.stripMargin)
-    }
-    assert(e.getMessage.contains("Only 'path', 'partitioning' and 'long_string_columns' dmproperties "
-      + "are allowed for this datamap"))
-    sql("DROP TABLE IF EXISTS maintabletime")
-  }
-
-  test("test pre agg create table 22: using invalid datamap provider") {
-    sql("DROP DATAMAP IF EXISTS agg0 ON TABLE maintable")
-
-    val e = intercept[MalformedDataMapCommandException] {
-      sql(
-        """
-          | CREATE DATAMAP agg0 ON TABLE mainTable
-          | USING 'abc'
-          | AS SELECT column3, SUM(column3),column5, SUM(column5)
-          | FROM maintable
-          | GROUP BY column3,column5,column2
-        """.stripMargin)
-    }
-    assert(e.getMessage.contains(s"DataMap 'abc' not found"))
-    sql("DROP DATAMAP IF EXISTS agg0 ON TABLE maintable")
-  }
-
-  test("test pre agg create table 24: remove agg tables from show table command") {
-    CarbonProperties.getInstance().addProperty(CarbonCommonConstants.CARBON_SHOW_DATAMAPS, "false")
-    sql("DROP TABLE IF EXISTS tbl_1")
-    sql("create table if not exists  tbl_1(imei string,age int,mac string ,prodate timestamp,update timestamp,gamepoint double,contrid double) stored by 'carbondata' ")
-    sql("create datamap agg1 on table tbl_1 using 'preaggregate' as select mac, sum(age) from tbl_1 group by mac")
-    sql("create table if not exists  sparktable(imei string,age int,mac string ,prodate timestamp,update timestamp,gamepoint double,contrid double) ")
-    checkExistence(sql("show tables"), false, "tbl_1_agg1")
-    checkExistence(sql("show tables"), true, "sparktable", "tbl_1")
-    CarbonProperties.getInstance().addProperty(CarbonCommonConstants.CARBON_SHOW_DATAMAPS, CarbonCommonConstants.CARBON_SHOW_DATAMAPS_DEFAULT)
-  }
-
-  test("test pre agg create table 25: remove TimeSeries agg tables from show table command") {
-    sql("DROP TABLE IF EXISTS tbl_1")
-    sql("create table if not exists  tbl_1(imei string,age int,mac string ,prodate timestamp,update timestamp,gamepoint double,contrid double) stored by 'carbondata' ")
-    sql(
-      "create datamap agg2 on table tbl_1 using 'preaggregate' as select prodate, mac from tbl_1 group by prodate,mac")
-    checkExistence(sql("show tables"), false, "tbl_1_agg2_day","tbl_1_agg2_hour","tbl_1_agg2_month","tbl_1_agg2_year")
-  }
-
-  test("test pre agg create table 21: should support 'if not exists'") {
-    sql(
-      """
-        | CREATE DATAMAP IF NOT EXISTS agg0 ON TABLE mainTable
-        | USING 'preaggregate'
-        | AS SELECT
-        |   column3,
-        |   sum(column3),
-        |   column5,
-        |   sum(column5)
-        | FROM maintable
-        | GROUP BY column3,column5,column2
-      """.stripMargin)
-    sql("DROP DATAMAP IF EXISTS agg0 ON TABLE maintable")
-  }
-
-  test("test pre agg create table 22: don't support create datamap if exists'") {
-    val e: Exception = intercept[AnalysisException] {
-      sql(
-        """
-          | CREATE DATAMAP IF EXISTS agg0 ON TABLE mainTable
-          | USING 'preaggregate'
-          | AS SELECT
-          |   column3,
-          |   sum(column3),
-          |   column5,
-          |   sum(column5)
-          | FROM maintable
-          | GROUP BY column3,column5,column2
-        """.stripMargin)
-      assert(true)
-    }
-    assert(e.getMessage.contains("identifier matching regex"))
-    sql("DROP DATAMAP IF EXISTS agg0 ON TABLE maintable")
-  }
-
-  test("test show tables filtered with datamaps") {
-    CarbonProperties.getInstance().addProperty(CarbonCommonConstants.CARBON_SHOW_DATAMAPS,"false")
-    sql("create datamap preagg1 on table PreAggMain using 'preaggregate' as select a,sum(b) from PreAggMain group by a")
-    sql("show tables").show()
-    checkExistence(sql("show tables"), false, "preaggmain_preagg1")
-    CarbonProperties.getInstance().addProperty(CarbonCommonConstants.CARBON_SHOW_DATAMAPS,CarbonCommonConstants.CARBON_SHOW_DATAMAPS_DEFAULT)
-    checkExistence(sql("show tables"), true, "preaggmain_preagg1")
-  }
-
-  test("test create main and preagg table of same name in two database") {
-    sql("drop table if exists Preagg_twodb")
-    sql("create table Preagg_twodb(name string, age int) stored by 'carbondata'")
-    sql("create datamap sameName on table Preagg_twodb using 'preaggregate' as select sum(age) from Preagg_twodb")
-    sql("create database otherDB")
-    sql("use otherDB")
-    sql("drop table if exists Preagg_twodb")
-    sql("create table Preagg_twodb(name string, age int) stored by 'carbondata'")
-    try {
-      sql(
-        "create datamap sameName on table Preagg_twodb using 'preaggregate' as select sum(age) from Preagg_twodb")
-      assert(true)
-    } catch {
-      case ex: Exception =>
-        assert(false)
-    }
-    sql("use default")
-  }
-
-  // TODO: to be confirmed
-  test("test pre agg create table 26") {
-    sql("drop datamap if exists preagg2 on table PreAggMain")
-    sql("create datamap preagg2 on table PreAggMain using 'preaggregate' as select a as a1,sum(b) from PreAggMain group by a")
-    checkExistence(sql("DESCRIBE FORMATTED PreAggMain_preagg2"), true, "preaggmain_a")
-    checkExistence(sql("DESCRIBE FORMATTED PreAggMain_preagg2"), true, "preaggmain_b_sum")
-    checkExistence(sql("DESCRIBE FORMATTED PreAggMain_preagg2"), false, "preaggmain_a1")
-    intercept[Exception] {
-      sql("select a1 from PreAggMain_preagg2").show()
-    }
-    sql("drop datamap if exists preagg2 on table PreAggMain")
-  }
-
-  test("test pre agg create table 27: select * and no group by") {
-    intercept[Exception] {
-      sql(
-        """
-          | CREATE DATAMAP IF NOT EXISTS agg0 ON TABLE mainTable
-          | USING 'preaggregate'
-          | AS SELECT *  FROM maintable
-        """.stripMargin)
-    }
-  }
-
-  // TODO : to be confirmed
-  test("test pre agg create table 28: select *") {
-    intercept[Exception] {
-      sql(
-        """
-          | CREATE DATAMAP IF NOT EXISTS agg0 ON TABLE mainTable
-          | USING 'preaggregate'
-          | AS SELECT *  FROM maintable
-          | group by a
-        """.stripMargin)
-    }
-  }
-
-  test("test pre agg create table 29") {
-    intercept[Exception] {
-      sql(
-        s"""
-           | create datamap preagg21 on table PreAggMain2
-           | using 'preaggregate'
-           | as select a as a1,sum(b)
-           | from PreAggMain2
-           | where a>'vishal'
-           | group by a
-         """.stripMargin)
-    }
-  }
-
-  test("test pre agg create table 30: DESCRIBE FORMATTED") {
-    dropDataMaps("PreAggMain", "preagg2")
-    intercept[Exception] {
-      sql("DESCRIBE FORMATTED PreAggMain_preagg2").show()
-    }
-  }
-
-  test("test codegen issue with preaggregate") {
-    sql("DROP TABLE IF EXISTS PreAggMain")
-    sql("CREATE TABLE PreAggMain (id Int, date date, country string, phonetype string, " +
-        "serialname String,salary int ) STORED BY 'org.apache.carbondata.format' " +
-        "tblproperties('dictionary_include'='country')")
-    sql("create datamap PreAggSum on table PreAggMain using 'preaggregate' as " +
-        "select country,sum(salary) as sum from PreAggMain group by country")
-    sql("create datamap PreAggAvg on table PreAggMain using 'preaggregate' as " +
-        "select country,avg(salary) as avg from PreAggMain group by country")
-    sql("create datamap PreAggCount on table PreAggMain using 'preaggregate' as " +
-        "select country,count(salary) as count from PreAggMain group by country")
-    sql("create datamap PreAggMin on table PreAggMain using 'preaggregate' as " +
-        "select country,min(salary) as min from PreAggMain group by country")
-    sql("create datamap PreAggMax on table PreAggMain using 'preaggregate' as " +
-        "select country,max(salary) as max from PreAggMain group by country")
-    sql(s"LOAD DATA INPATH '$integrationPath/spark-common-test/src/test/resources/source.csv' " +
-        s"into table PreAggMain")
-    checkExistence(sql("select t1.country,sum(id) from PreAggMain t1 join (select " +
-                       "country as newcountry,sum(salary) as sum from PreAggMain group by country)" +
-                       "t2 on t1.country=t2.newcountry group by country"), true, "france")
-    sql("DROP TABLE IF EXISTS PreAggMain")
-  }
-
-  // TODO: Need to Fix
-  ignore("test creation of multiple preaggregate of same name concurrently") {
-    sql("DROP TABLE IF EXISTS tbl_concurr")
-    sql(
-      "create table if not exists  tbl_concurr(imei string,age int,mac string ,prodate timestamp," +
-      "update timestamp,gamepoint double,contrid double) stored by 'carbondata' ")
-
-    var executorService: ExecutorService = Executors.newCachedThreadPool()
-    val tasks = new util.ArrayList[Callable[String]]()
-    var i = 0
-    val count = 5
-    while (i < count) {
-      tasks
-        .add(new QueryTask(
-          s"""create datamap agg_concu1 on table tbl_concurr using
-             |'preaggregate' as select prodate, mac from tbl_concurr group by prodate,mac"""
-            .stripMargin))
-      i = i + 1
-    }
-    executorService.invokeAll(tasks).asScala
-    executorService.awaitTermination(5, TimeUnit.MINUTES)
-    checkExistence(sql("show tables"), true, "agg_concu1", "tbl_concurr")
-    executorService.shutdown()
-  }
-
-  class QueryTask(query: String) extends Callable[String] {
-    override def call(): String = {
-      var result = "SUCCESS"
-      try {
-        sql(query).collect()
-      } catch {
-        case exception: Exception => LOGGER.error(exception.getMessage)
-      }
-      result
-    }
-  }
-
-
-  def getCarbonTable(plan: LogicalPlan) : CarbonTable ={
-    var carbonTable : CarbonTable = null
-    plan.transform {
-      // first check if any preaTable1 scala function is applied it is present is in plan
-      // then call is from create preaTable1regate table class so no need to transform the query plan
-      case ca:CarbonRelation =>
-        if (ca.isInstanceOf[CarbonDatasourceHadoopRelation]) {
-          val relation = ca.asInstanceOf[CarbonDatasourceHadoopRelation]
-          carbonTable = relation.carbonTable
-        }
-        ca
-      case logicalRelation:LogicalRelation =>
-        if(logicalRelation.relation.isInstanceOf[CarbonDatasourceHadoopRelation]) {
-          val relation = logicalRelation.relation.asInstanceOf[CarbonDatasourceHadoopRelation]
-          carbonTable = relation.carbonTable
-        }
-        logicalRelation
-    }
-    carbonTable
-  }
-
-  override def afterAll {
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.CARBON_SHOW_DATAMAPS,
-        CarbonCommonConstants.CARBON_SHOW_DATAMAPS_DEFAULT)
-    sql("drop database if exists otherDB cascade")
-    sql("drop table if exists maintable")
-    sql("drop table if exists PreAggMain")
-    sql("drop table if exists PreAggMain1")
-    sql("drop table if exists maintabletime")
-    sql("drop table if exists showTables")
-    sql("drop table if exists Preagg_twodb")
-    sql("DROP TABLE IF EXISTS tbl_1")
-    sql("DROP TABLE IF EXISTS sparktable")
-  }
-}
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggStreaming.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggStreaming.scala
deleted file mode 100644
index 28b7546..0000000
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggStreaming.scala
+++ /dev/null
@@ -1,143 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.integration.spark.testsuite.preaggregate
-
-import org.apache.spark.sql.CarbonDatasourceHadoopRelation
-import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Union}
-import org.apache.spark.sql.execution.datasources.LogicalRelation
-import org.apache.spark.sql.test.util.QueryTest
-import org.scalatest.{BeforeAndAfterAll, Ignore}
-
-
-class TestPreAggStreaming extends QueryTest with BeforeAndAfterAll {
-
-
-  override def beforeAll: Unit = {
-    dropAll
-    sql("CREATE TABLE mainTable(id int, name string, city string, age string) STORED BY 'org.apache.carbondata.format' tblproperties('streaming'='true')")
-    sql("create datamap agg0 on table mainTable using 'preaggregate' as select name from mainTable group by name")
-    sql("create datamap agg1 on table mainTable using 'preaggregate' as select name,sum(age) from mainTable group by name")
-    sql("create datamap agg2 on table mainTable using 'preaggregate' as select name,avg(age) from mainTable group by name")
-    sql("create datamap agg3 on table mainTable using 'preaggregate' as select name,sum(CASE WHEN age=35 THEN id ELSE 0 END) from mainTable group by name")
-    sql("CREATE TABLE mainTableStreamingOne(id int, name string, city string, age smallint) STORED BY 'org.apache.carbondata.format' tblproperties('streaming'='true')")
-    sql("create datamap aggStreamingAvg on table mainTableStreamingOne using 'preaggregate' as select name,avg(age) from mainTableStreamingOne group by name")
-    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/measureinsertintotest.csv' into table mainTable")
-    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/measureinsertintotest.csv' into table mainTableStreamingOne")
-    sql("CREATE TABLE origin(id int, name string, city string, age string) STORED BY 'org.apache.carbondata.format' tblproperties('streaming'='true')")
-    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/measureinsertintotest.csv' into table origin")
-    sql("CREATE TABLE binary_stream(id int, label boolean, name string,image binary,autoLabel boolean) STORED BY 'org.apache.carbondata.format' tblproperties('streaming'='true')")
-    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/binaryDataBase64.csv' into table binary_stream OPTIONS('header'='false','DELIMITER'=',','binary_decoder'='baSe64')")
-    sql("create datamap agg0 on table binary_stream using 'preaggregate' as select name from binary_stream group by name")
-  }
-
-  test("Test Pre Agg Streaming with project column and group by") {
-    val df = sql("select name from maintable group by name")
-    assert(validateStreamingTablePlan(df.queryExecution.analyzed))
-    checkAnswer(df, sql("select name from origin group by name"))
-  }
-
-  test("Test binary with stream and preaggregate") {
-    val df = sql("select name from binary_stream group by name")
-    df.collect()
-    assert(validateStreamingTablePlan(df.queryExecution.analyzed))
-    checkAnswer(df, sql("select name from binary_stream group by name"))
-  }
-
-  test("Test Pre Agg Streaming table Agg Sum Aggregation") {
-    val df = sql("select name, sum(age) from maintable group by name")
-    assert(validateStreamingTablePlan(df.queryExecution.analyzed))
-    checkAnswer(df, sql("select name, sum(age) from origin group by name"))
-  }
-
-  test("Test Pre Agg Streaming table with UDF") {
-    val df = sql("select substring(name,1,1), sum(age) from maintable group by substring(name,1,1)")
-    assert(validateStreamingTablePlan(df.queryExecution.analyzed))
-    checkAnswer(df, sql("select substring(name,1,1), sum(age) from origin group by substring(name,1,1)"))
-  }
-
-  test("Test Pre Agg Streaming table with UDF Only in group by") {
-    val df = sql("select sum(age) from maintable group by substring(name,1,1)")
-    assert(validateStreamingTablePlan(df.queryExecution.analyzed))
-    checkAnswer(df, sql("select sum(age) from origin group by substring(name,1,1)"))
-  }
-
-  test("Test Pre Agg Streaming table With Sum Aggregation And Order by") {
-    val df = sql("select name, sum(age) from maintable group by name order by name")
-    assert(validateStreamingTablePlan(df.queryExecution.analyzed))
-    checkAnswer(df, sql("select name, sum(age) from origin group by name order by name"))
-  }
-
-  test("Test Pre Agg Streaming table With Avg Aggregation") {
-    val df = sql("select name, avg(age) from maintable group by name order by name")
-    assert(validateStreamingTablePlan(df.queryExecution.analyzed))
-    checkAnswer(df, sql("select name, avg(age) from origin group by name order by name"))
-  }
-
-  test("Test Pre Agg Streaming table With Expression Aggregation") {
-    val df = sql("select name, sum(CASE WHEN age=35 THEN id ELSE 0 END) from maintable group by name order by name")
-    assert(validateStreamingTablePlan(df.queryExecution.analyzed))
-    checkAnswer(df, sql("select name, sum(CASE WHEN age=35 THEN id ELSE 0 END) from origin group by name order by name"))
-  }
-
-  test("Test Pre Agg Streaming table With only aggregate expression and group by") {
-    val df = sql("select sum(age) from maintable group by name")
-    assert(validateStreamingTablePlan(df.queryExecution.analyzed))
-    checkAnswer(df, sql("select sum(age) from origin group by name"))
-  }
-
-  test("Test Pre Agg Streaming table With small int and avg") {
-    val df = sql("select name, avg(age) from mainTableStreamingOne group by name")
-    assert(validateStreamingTablePlan(df.queryExecution.analyzed))
-    checkAnswer(df, sql("select name, avg(age) from origin group by name"))
-  }
-
-  /**
-   * Below method will be used validate whether plan is already updated in case of streaming table
-   * In case of streaming table it will add UnionNode to get the data from fact and aggregate both
-   * as aggregate table will be updated after each handoff.
-   * So if plan is already updated no need to transform the plan again
-   * @param logicalPlan
-   * query plan
-   * @return whether need to update the query plan or not
-   */
-  def validateStreamingTablePlan(logicalPlan: LogicalPlan) : Boolean = {
-    var isChildTableExists: Boolean = false
-    logicalPlan.transform {
-      case union @ Union(Seq(plan1, plan2)) =>
-        plan2.collect{
-          case logicalRelation: LogicalRelation if
-          logicalRelation.relation.isInstanceOf[CarbonDatasourceHadoopRelation] &&
-          logicalRelation.relation.asInstanceOf[CarbonDatasourceHadoopRelation].carbonTable
-            .isChildDataMap =>
-            isChildTableExists = true
-        }
-        union
-    }
-    isChildTableExists
-  }
-
-  private def dropAll: Unit = {
-    sql("drop table if exists mainTable")
-    sql("drop table if exists mainTableStreamingOne")
-    sql("drop table if exists origin")
-    sql("drop table if exists binary_stream")
-  }
-
-  override def afterAll: Unit = {
-    dropAll
-  }
-}
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateCompaction.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateCompaction.scala
deleted file mode 100644
index 7bf0c35..0000000
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateCompaction.scala
+++ /dev/null
@@ -1,212 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.integration.spark.testsuite.preaggregate
-
-import org.apache.spark.sql.test.util.QueryTest
-import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach}
-import org.scalatest.Matchers._
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.util.CarbonProperties
-
-class TestPreAggregateCompaction extends QueryTest with BeforeAndAfterEach with BeforeAndAfterAll {
-
-  val testData = s"$resourcesPath/sample.csv"
-
-  override def beforeEach(): Unit = {
-    sql("drop database if exists compaction cascade")
-    sql("create database if not exists compaction")
-    sql("use compaction")
-    sql("create table testtable (id int, name string, city string, age int) STORED BY 'org.apache.carbondata.format'")
-    sql(
-      """
-        | CREATE TABLE maintable(id int, name string, city string, age int)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    sql(
-      s"""create datamap preagg_sum on table maintable using 'preaggregate' as select id,sum(age) from maintable group by id"""
-        .stripMargin)
-    sql(
-      s"""create datamap preagg_avg on table maintable using 'preaggregate' as select id,avg(age) from maintable group by id"""
-        .stripMargin)
-  }
-
-  test("test if pre-agg table is compacted with parent table minor compaction") {
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql("insert into testtable select * from maintable")
-    val sumResult = sql("select id, sum(age) from testtable group by id").collect()
-    val avgResult = sql("select id, sum(age), count(age) from testtable group by id").collect()
-    sql("alter table maintable compact 'minor'")
-    val segmentNamesSum = sql("show segments for table maintable_preagg_sum").collect().map(_.get(0).toString)
-    segmentNamesSum should equal (Array("3", "2", "1", "0.1", "0"))
-    checkAnswer(sql("select * from maintable_preagg_sum"), sumResult)
-    val segmentNamesAvg = sql("show segments for table maintable_preagg_avg").collect().map(_.get(0).toString)
-    segmentNamesAvg should equal (Array("3", "2", "1", "0.1", "0"))
-    checkAnswer(sql("select * from maintable_preagg_avg"), avgResult)
-  }
-
-  test("test if pre-agg table is compacted with parent table major compaction") {
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql("alter table maintable compact 'major'")
-    sql("insert into testtable select * from maintable")
-    val sumResult = sql("select id, sum(age) from testtable group by id").collect()
-    val avgResult = sql("select id, sum(age), count(age) from testtable group by id").collect()
-    sql("alter table maintable compact 'minor'")
-    val segmentNamesSum = sql("show segments for table maintable_preagg_sum").collect().map(_.get(0).toString)
-    segmentNamesSum should equal (Array("3", "2", "1", "0.1", "0"))
-    checkAnswer(sql("select * from maintable_preagg_sum"), sumResult)
-    val segmentNamesAvg = sql("show segments for table maintable_preagg_avg").collect().map(_.get(0).toString)
-    segmentNamesAvg should equal (Array("3", "2", "1", "0.1", "0"))
-    checkAnswer(sql("select * from maintable_preagg_avg"), avgResult)
-  }
-
-  test("test if 2nd level minor compaction is successful for pre-agg table") {
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql("alter table maintable compact 'minor'")
-    var segmentNamesSum = sql("show segments for table maintable_preagg_sum").collect().map(_.get(0).toString)
-    segmentNamesSum should equal (Array("3", "2", "1", "0.1", "0"))
-    sql("insert into testtable select * from maintable")
-    var sumResult = sql("select id, sum(age) from testtable group by id").collect()
-    var avgResult = sql("select id, sum(age), count(age) from testtable group by id").collect()
-    checkAnswer(sql("select * from maintable_preagg_sum"), sumResult)
-    var segmentNamesAvg = sql("show segments for table maintable_preagg_avg").collect().map(_.get(0).toString)
-    segmentNamesAvg should equal (Array("3", "2", "1", "0.1", "0"))
-    checkAnswer(sql("select * from maintable_preagg_avg"), avgResult)
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql("alter table maintable compact 'minor'")
-    sql("insert overwrite table testtable select * from maintable")
-    sumResult = sql("select id, sum(age) from testtable group by id").collect()
-    avgResult = sql("select id, sum(age), count(age) from testtable group by id").collect()
-    segmentNamesSum = sql("show segments for table maintable_preagg_sum").collect().map(_.get(0).toString)
-    segmentNamesSum.sorted should equal (Array("0", "0.1", "1", "2", "3", "4", "4.1", "5", "6", "7"))
-    checkAnswer(sql("select maintable_id, sum(maintable_age_sum) from maintable_preagg_sum group by maintable_id"), sumResult)
-    segmentNamesAvg = sql("show segments for table maintable_preagg_avg").collect().map(_.get(0).toString)
-    segmentNamesAvg.sorted should equal (Array("0", "0.1", "1", "2", "3", "4", "4.1", "5", "6", "7"))
-    checkAnswer(sql("select maintable_id, sum(maintable_age_sum), sum(maintable_age_count) from maintable_preagg_avg group by maintable_id"), avgResult)
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql("alter table maintable compact 'minor'")
-    sql("insert overwrite table testtable select * from maintable")
-    sumResult = sql("select id, sum(age) from testtable group by id").collect()
-    avgResult = sql("select id, sum(age), count(age) from testtable group by id").collect()
-    segmentNamesSum = sql("show segments for table maintable_preagg_sum").collect().map(_.get(0).toString)
-    segmentNamesSum should equal (Array("11", "10", "9", "8.1", "8", "7", "6", "5", "4.1", "4", "3", "2", "1", "0.2", "0.1", "0"))
-    checkAnswer(sql("select maintable_id, sum(maintable_age_sum) from maintable_preagg_sum group by maintable_id"), sumResult)
-    segmentNamesAvg = sql("show segments for table maintable_preagg_avg").collect().map(_.get(0).toString)
-    segmentNamesAvg should equal (Array("11", "10", "9", "8.1", "8", "7", "6", "5", "4.1", "4", "3", "2", "1", "0.2", "0.1", "0"))
-    checkAnswer(sql("select maintable_id, sum(maintable_age_sum), sum(maintable_age_count) from maintable_preagg_avg group by maintable_id"), avgResult)
-  }
-
-  test("test direct minor compaction on pre-agg tables") {
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql("alter table maintable_preagg_sum compact 'minor'")
-    var segmentNamesSum = sql("show segments for table maintable_preagg_sum").collect().map(_.get(0).toString)
-    segmentNamesSum should equal (Array("3", "2", "1", "0.1", "0"))
-    sql("insert into testtable select * from maintable")
-    var sumResult = sql("select id, sum(age) from testtable group by id").collect()
-    checkAnswer(sql("select * from maintable_preagg_sum"), sumResult)
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql("alter table maintable_preagg_sum compact 'minor'")
-    sql("insert overwrite table testtable select * from maintable")
-    sumResult = sql("select id, sum(age) from testtable group by id").collect()
-    segmentNamesSum = sql("show segments for table maintable_preagg_sum").collect().map(_.get(0).toString)
-    segmentNamesSum.sorted should equal (Array("0", "0.1", "1", "2", "3", "4", "4.1", "5", "6", "7"))
-    checkAnswer(sql("select maintable_id, sum(maintable_age_sum) from maintable_preagg_sum group by maintable_id"), sumResult)
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql("alter table maintable_preagg_sum compact 'minor'")
-    sql("insert overwrite table testtable select * from maintable")
-    sumResult = sql("select id, sum(age) from testtable group by id").collect()
-    segmentNamesSum = sql("show segments for table maintable_preagg_sum").collect().map(_.get(0).toString)
-    segmentNamesSum should equal (Array("11", "10", "9", "8.1", "8", "7", "6", "5", "4.1", "4", "3", "2", "1", "0.2", "0.1", "0"))
-    checkAnswer(sql("select maintable_id, sum(maintable_age_sum) from maintable_preagg_sum group by maintable_id"), sumResult)
-    val mainTableSegment = sql("SHOW SEGMENTS FOR TABLE maintable")
-    val SegmentSequenceIds = mainTableSegment.collect().map { each => (each.toSeq) (0) }
-    assert(!SegmentSequenceIds.contains("0.1"))
-  }
-
-  test("test if minor/major compaction is successful for pre-agg table") {
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql("alter table maintable_preagg_sum compact 'minor'")
-    var segmentNamesSum = sql("show segments for table maintable_preagg_sum").collect().map(_.get(0).toString)
-    segmentNamesSum should equal (Array("3","2","1","0.1", "0"))
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql("alter table maintable_preagg_sum compact 'major'")
-    segmentNamesSum = sql("show segments for table maintable_preagg_sum").collect().map(_.get(0).toString)
-    segmentNamesSum.sorted should equal (Array("0", "0.1", "0.2", "1", "2", "3", "4", "5", "6", "7"))
-  }
-
-  test("test auto compaction on aggregate table") {
-    CarbonProperties.getInstance().addProperty(CarbonCommonConstants.ENABLE_AUTO_LOAD_MERGE, "true")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    val segmentNamesSum = sql("show segments for table maintable_preagg_sum").collect().map(_.get(0).toString)
-    segmentNamesSum.sorted should equal  (Array("0", "0.1", "1", "2", "3"))
-    CarbonProperties.getInstance().addProperty(CarbonCommonConstants.ENABLE_AUTO_LOAD_MERGE, "false")
-  }
-
-  test("test minor compaction on Pre-agg tables after multiple loads") {
-    CarbonProperties.getInstance().addProperty(CarbonCommonConstants.ENABLE_AUTO_LOAD_MERGE, "true")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql("alter table maintable compact 'minor'")
-    assert(sql("show segments for table maintable").collect().map(_.get(1).toString.toLowerCase).contains("compacted"))
-  }
-
-  override def afterAll(): Unit = {
-    CarbonProperties.getInstance().addProperty(CarbonCommonConstants.ENABLE_AUTO_LOAD_MERGE, "false")
-    sql("drop database if exists compaction cascade")
-    sql("use default")
-  }
-
-}
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateDrop.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateDrop.scala
deleted file mode 100644
index f99841c..0000000
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateDrop.scala
+++ /dev/null
@@ -1,169 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.integration.spark.testsuite.preaggregate
-
-import org.apache.spark.sql.test.util.QueryTest
-import org.scalatest.BeforeAndAfterAll
-
-import org.apache.carbondata.common.exceptions.sql.NoSuchDataMapException
-import org.apache.carbondata.spark.exception.ProcessMetaDataException
-
-class TestPreAggregateDrop extends QueryTest with BeforeAndAfterAll {
-
-  override def beforeAll {
-    sql("drop table if exists maintable")
-    sql("create table maintable (a string, b string, c string) stored by 'carbondata'")
-  }
-
-  test("create and drop preaggregate table") {
-    sql(
-      "create datamap preagg1 on table maintable using 'preaggregate' as select" +
-      " a,sum(b) from maintable group by a")
-    sql("drop datamap if exists preagg1 on table maintable")
-    checkExistence(sql("SHOW DATAMAP ON TABLE maintable"), false, "maintable_preagg1")
-  }
-
-  test("dropping 1 aggregate table should not drop others") {
-    sql(
-      "create datamap preagg1 on table maintable using 'preaggregate' as select" +
-      " a,sum(b) from maintable group by a")
-    sql(
-      "create datamap preagg2 on table maintable using 'preaggregate' as select" +
-      " a,sum(c) from maintable group by a")
-    sql("drop datamap if exists preagg2 on table maintable")
-    val showdatamaps =sql("show datamap on table maintable")
-    checkExistence(showdatamaps, false, "maintable_preagg2")
-    checkExistence(showdatamaps, true, "maintable_preagg1")
-  }
-
-  test("drop datamap which is not existed") {
-    intercept[NoSuchDataMapException] {
-      sql("drop datamap newpreagg on table maintable")
-    }
-  }
-
-  test("drop datamap with same name on different tables") {
-    sql("drop table if exists maintable1")
-    sql("create datamap preagg_same on table maintable using 'preaggregate' as select" +
-    " a,sum(c) from maintable group by a")
-    sql("create table maintable1 (a string, b string, c string) stored by 'carbondata'")
-    sql("create datamap preagg_same on table maintable1 using 'preaggregate' as select" +
-    " a,sum(c) from maintable1 group by a")
-
-    sql("drop datamap preagg_same on table maintable")
-    val showDataMaps = sql("SHOW DATAMAP ON TABLE maintable1")
-    checkExistence(showDataMaps, false, "maintable_preagg_same")
-    checkExistence(showDataMaps, true, "maintable1_preagg_same")
-    sql("drop datamap preagg_same on table maintable1")
-    checkExistence(sql("SHOW DATAMAP ON TABLE maintable1"), false, "maintable1_preagg_same")
-    sql("drop table if exists maintable1")
-  }
-
-  test("drop datamap and create again with same name") {
-    sql("create datamap preagg_same1 on table maintable using 'preaggregate' as select" +
-    " a,sum(c) from maintable group by a")
-
-    sql("drop datamap preagg_same1 on table maintable")
-    checkExistence(sql("SHOW DATAMAP ON TABLE maintable"), false, "maintable_preagg_same1")
-    sql("create datamap preagg_same1 on table maintable using 'preaggregate' as select" +
-        " a,sum(c) from maintable group by a")
-    val showDatamaps =sql("show datamap on table maintable")
-    checkExistence(showDatamaps, true, "maintable_preagg_same1")
-    sql("drop datamap preagg_same1 on table maintable")
-  }
-
-  test("drop main table and check if preaggreagte is deleted") {
-    sql(
-      "create datamap preagg2 on table maintable using 'preaggregate' as select" +
-      " a,sum(c) from maintable group by a")
-    sql("drop table if exists maintable")
-    checkExistence(sql("show tables"), false, "maintable_preagg1", "maintable", "maintable_preagg2")
-  }
-
-  test("drop datamap with 'if exists' when datamap not exists") {
-    sql("DROP TABLE IF EXISTS maintable")
-    sql("CREATE TABLE maintable (a STRING, b STRING, c STRING) STORED BY 'carbondata'")
-    sql("DROP DATAMAP IF EXISTS not_exists_datamap ON TABLE maintable")
-    checkExistence(sql("DESCRIBE FORMATTED maintable"), false, "not_exists_datamap")
-  }
-
-  test("drop datamap without 'if exists' when datamap not exists") {
-    sql("DROP TABLE IF EXISTS maintable")
-    sql("CREATE TABLE maintable (a STRING, b STRING, c STRING) STORED BY 'carbondata'")
-    sql("DROP DATAMAP IF EXISTS not_exists_datamap ON TABLE maintable")
-    val e = intercept[NoSuchDataMapException] {
-      sql("DROP DATAMAP not_exists_datamap ON TABLE maintable")
-    }
-    assert(e.getMessage.equals(
-      "Datamap with name not_exists_datamap does not exist"))
-  }
-
-  test("drop datamap without 'if exists' when main table not exists") {
-    sql("DROP TABLE IF EXISTS maintable")
-    val e = intercept[ProcessMetaDataException] {
-      sql("DROP DATAMAP preagg3 ON TABLE maintable")
-    }
-    assert(e.getMessage.contains("Table or view 'maintable' not found in"))
-  }
-
-  test("drop datamap with 'if exists' when main table not exists") {
-    sql("DROP TABLE IF EXISTS maintable")
-    val e = intercept[ProcessMetaDataException] {
-      sql("DROP DATAMAP IF EXISTS preagg3 ON TABLE maintable")
-    }
-    assert(e.getMessage.contains("Table or view 'maintable' not found in"))
-  }
-
-  test("drop preaggregate datamap whose main table has other type datamaps") {
-    sql("drop table if exists maintable")
-    sql("create table maintable (a string, b string, c string) stored by 'carbondata'")
-    sql(
-      "create datamap bloom1 on table maintable using 'bloomfilter'" +
-      " dmproperties('index_columns'='a')")
-    sql(
-      "create datamap preagg1 on table maintable using 'preaggregate' as select" +
-      " a,sum(b) from maintable group by a")
-    sql("drop datamap if exists bloom1 on table maintable")
-    sql("drop datamap if exists preagg1 on table maintable")
-    checkExistence(sql("SHOW DATAMAP ON TABLE maintable"), false, "preagg1", "bloom1")
-    sql(
-      "create datamap bloom1 on table maintable using 'bloomfilter'" +
-      " dmproperties('index_columns'='a')")
-    sql(
-      "create datamap preagg1 on table maintable using 'preaggregate' as select" +
-      " a,sum(b) from maintable group by a")
-    sql("drop datamap if exists preagg1 on table maintable")
-    sql("drop datamap if exists bloom1 on table maintable")
-    checkExistence(sql("SHOW DATAMAP ON TABLE maintable"), false, "preagg1", "bloom1")
-    sql(
-      "create datamap bloom1 on table maintable using 'bloomfilter'" +
-      " dmproperties('index_columns'='a')")
-    sql(
-      "create datamap preagg1 on table maintable using 'preaggregate' as select" +
-      " a,sum(b) from maintable group by a")
-    sql("drop table if exists maintable")
-    checkExistence(sql("show tables"), false, "maintable_preagg1", "maintable")
-    checkExistence(sql("show datamap"), false, "preagg1", "bloom1")
-  }
-
-  override def afterAll() {
-    sql("drop table if exists maintable")
-    sql("drop table if exists maintable1")
-  }
-  
-}
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateExpressions.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateExpressions.scala
deleted file mode 100644
index a7511fd..0000000
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateExpressions.scala
+++ /dev/null
@@ -1,182 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.integration.spark.testsuite.preaggregate
-
-import org.apache.spark.sql.{CarbonDatasourceHadoopRelation, Row}
-import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
-import org.apache.spark.sql.execution.datasources.LogicalRelation
-import org.apache.spark.sql.hive.CarbonRelation
-import org.apache.spark.sql.test.util.QueryTest
-import org.scalatest.BeforeAndAfterAll
-
-class TestPreAggregateExpressions extends QueryTest with BeforeAndAfterAll {
-
-  override def beforeAll: Unit = {
-    sql("DROP TABLE IF EXISTS mainTable")
-    sql("CREATE TABLE mainTable(id int, name string, city string, age string) STORED BY 'org.apache.carbondata.format'")
-    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/measureinsertintotest.csv' into table mainTable")
-  }
-
-  test("test pre agg create table with expression 1") {
-    sql(
-      s"""
-         | CREATE DATAMAP agg0 ON TABLE mainTable USING 'preaggregate' AS
-         | SELECT name,
-         | count(age)
-         | FROM mainTable GROUP BY name
-         | """.stripMargin)
-    checkExistence(sql("DESCRIBE FORMATTED mainTable_agg0"), true, "maintable_age_count")
-  }
-
-  test("test pre agg create table with expression 2") {
-    sql(
-      s"""
-         | CREATE DATAMAP agg1 ON TABLE mainTable USING 'preaggregate' AS
-         | SELECT name,
-         | sum(CASE WHEN age=35 THEN id ELSE 0 END)
-         | FROM mainTable GROUP BY name
-         | """.stripMargin)
-    checkExistence(sql("DESCRIBE FORMATTED mainTable_agg1"), true, "maintable_column_0_sum")
-  }
-
-  test("test pre agg create table with expression 3") {
-    sql(
-      s"""
-         | CREATE DATAMAP agg2 ON TABLE mainTable USING 'preaggregate' AS
-         | SELECT name,
-         | sum(CASE WHEN age=35 THEN id ELSE 0 END),
-         | city
-         | FROM mainTable GROUP BY name,city
-         | """.stripMargin)
-    checkExistence(sql("DESCRIBE FORMATTED mainTable_agg2"), true, "maintable_column_0_sum")
-  }
-
-  test("test pre agg create table with expression 4") {
-    sql(
-      s"""
-         | CREATE DATAMAP agg3 ON TABLE mainTable USING 'preaggregate' AS
-         | SELECT name,
-         | sum(CASE WHEN age=27 THEN id ELSE 0 END)
-         | FROM mainTable GROUP BY name
-         | """.stripMargin)
-    checkExistence(sql("DESCRIBE FORMATTED mainTable_agg3"), true, "maintable_column_0_sum")
-  }
-
-  test("test pre agg create table with expression 5") {
-    sql(
-      s"""
-         | CREATE DATAMAP agg4 ON TABLE mainTable USING 'preaggregate' AS
-         | SELECT name,
-         | sum(CASE WHEN age=27 THEN id ELSE 0 END),
-         | SUM(CASE WHEN age=35 THEN id ELSE 0 END)
-         | FROM mainTable GROUP BY name
-         | """.stripMargin)
-    checkExistence(sql("DESCRIBE FORMATTED mainTable_agg4"), true, "maintable_column_0_sum")
-    checkExistence(sql("DESCRIBE FORMATTED mainTable_agg4"), true, "maintable_column_1_sum")
-  }
-
-  test("test pre agg create table with expression 6") {
-    sql(
-      s"""
-         | CREATE DATAMAP agg5 ON TABLE mainTable USING 'preaggregate' AS
-         | SELECT name,
-         | COUNT(CASE WHEN age=27 THEN(CASE WHEN name='eason' THEN id ELSE 0 END) ELSE 0 END)
-         | FROM mainTable GROUP BY name
-         | """.stripMargin)
-    checkExistence(sql("DESCRIBE FORMATTED mainTable_agg5"), true, "maintable_column_0_count")
-  }
-
-  test("test pre agg table selection with expression 1") {
-    val df = sql("select name as NewName, count(age) as sum from mainTable group by name order by name")
-    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg0")
-  }
-
-  test("test pre agg table selection with expression 2") {
-    val df = sql("select name as NewName, sum(case when age=35 then id else 0 end) as sum from mainTable group by name order by name")
-    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg1")
-  }
-
-  test("test pre agg table selection with expression 3") {
-    val df = sql("select sum(case when age=35 then id else 0 end) from maintable")
-    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg1")
-    checkAnswer(df, Seq(Row(6.0)))
-  }
-
-  test("test pre agg table selection with expression 4") {
-    val df = sql("select sum(case when age=27 then id else 0 end) from maintable")
-    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg3")
-    checkAnswer(df, Seq(Row(2.0)))
-  }
-
-  test("test pre agg table selection with expression 5") {
-    val df = sql("select sum(case when age=27 then id else 0 end), sum(case when age=35 then id else 0 end) from maintable")
-    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg4")
-    checkAnswer(df, Seq(Row(2.0,6.0)))
-  }
-
-  /**
-   * Below method will be used to validate the table name is present in the plan or not
-   *
-   * @param plan            query plan
-   * @param actualTableName table name to be validated
-   */
-  def preAggTableValidator(plan: LogicalPlan, actualTableName: String) : Unit ={
-    var isValidPlan = false
-    plan.transform {
-      // first check if any preaTable1 scala function is applied it is present is in plan
-      // then call is from create preaTable1regate table class so no need to transform the query plan
-      case ca:CarbonRelation =>
-        if (ca.isInstanceOf[CarbonDatasourceHadoopRelation]) {
-          val relation = ca.asInstanceOf[CarbonDatasourceHadoopRelation]
-          if(relation.carbonTable.getTableName.equalsIgnoreCase(actualTableName)) {
-            isValidPlan = true
-          }
-        }
-        ca
-      case logicalRelation:LogicalRelation =>
-        if(logicalRelation.relation.isInstanceOf[CarbonDatasourceHadoopRelation]) {
-          val relation = logicalRelation.relation.asInstanceOf[CarbonDatasourceHadoopRelation]
-          if(relation.carbonTable.getTableName.equalsIgnoreCase(actualTableName)) {
-            isValidPlan = true
-          }
-        }
-        logicalRelation
-    }
-    if(!isValidPlan) {
-      assert(false)
-    } else {
-      assert(true)
-    }
-  }
-
-  test("Test Pre_aggregate with decimal column with order by") {
-    sql("drop table if exists maintable")
-    sql("create table maintable(name string, decimal_col decimal(30,16)) stored by 'carbondata'")
-    sql("insert into table maintable select 'abc',452.564")
-    sql(
-      "create datamap ag1 on table maintable using 'preaggregate' as select name,avg(decimal_col)" +
-      " from maintable group by name")
-    checkAnswer(sql("select avg(decimal_col) from maintable group by name order by name"),
-      Seq(Row(452.56400000000000000000)))
-  }
-
-  override def afterAll: Unit = {
-    sql("DROP TABLE IF EXISTS mainTable")
-  }
-
-}
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateLoad.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateLoad.scala
deleted file mode 100644
index 75d71ec..0000000
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateLoad.scala
+++ /dev/null
@@ -1,1039 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.integration.spark.testsuite.preaggregate
-
-import org.apache.spark.sql.Row
-import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.Spark2TestQueryExecutor
-import org.apache.spark.util.SparkUtil4Test
-import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach}
-
-import org.apache.carbondata.common.exceptions.sql.MalformedDataMapCommandException
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.spark.util.SparkQueryTest
-
-class TestPreAggregateLoad extends SparkQueryTest with BeforeAndAfterAll with BeforeAndAfterEach{
-
-  val testData = s"$resourcesPath/sample.csv"
-  val p1 = CarbonProperties.getInstance()
-    .getProperty(CarbonCommonConstants.COMPACTION_SEGMENT_LEVEL_THRESHOLD,
-      CarbonCommonConstants.DEFAULT_SEGMENT_LEVEL_THRESHOLD)
-
-  override def beforeAll(): Unit = {
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.ENABLE_AUTO_LOAD_MERGE, "false")
-      .addProperty(CarbonCommonConstants.COMPACTION_SEGMENT_LEVEL_THRESHOLD,
-        CarbonCommonConstants.DEFAULT_SEGMENT_LEVEL_THRESHOLD)
-
-    SparkUtil4Test.createTaskMockUp(sqlContext)
-    sql("DROP TABLE IF EXISTS maintable")
-  }
-
-  override protected def afterAll(): Unit = {
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.ENABLE_AUTO_LOAD_MERGE,
-        CarbonCommonConstants.DEFAULT_ENABLE_AUTO_LOAD_MERGE)
-      .addProperty(CarbonCommonConstants.COMPACTION_SEGMENT_LEVEL_THRESHOLD, p1)
-    sql("DROP TABLE IF EXISTS y ")
-    sql("DROP TABLE IF EXISTS maintable")
-    sql("DROP TABLE IF EXISTS maintbl")
-    sql("DROP TABLE IF EXISTS main_table")
-  }
-
-  override protected def beforeEach(): Unit = {
-    sql("DROP TABLE IF EXISTS main_table")
-    sql("DROP TABLE IF EXISTS segmaintable")
-  }
-
-  private def createAllAggregateTables(parentTableName: String, columnName: String = "age"): Unit = {
-    sql(
-      s"""
-         | create datamap preagg_sum
-         | on table $parentTableName
-         | using 'preaggregate'
-         | as select id,sum($columnName)
-         | from $parentTableName
-         | group by id
-       """.stripMargin)
-    sql(
-      s"""create datamap preagg_avg on table $parentTableName using 'preaggregate' as select id,avg($columnName) from $parentTableName group by id"""
-        .stripMargin)
-    sql(
-      s"""create datamap preagg_count on table $parentTableName using 'preaggregate' as select id,count($columnName) from $parentTableName group by id"""
-        .stripMargin)
-    sql(
-      s"""create datamap preagg_min on table $parentTableName using 'preaggregate' as select id,min($columnName) from $parentTableName group by id"""
-        .stripMargin)
-    sql(
-      s"""create datamap preagg_max on table $parentTableName using 'preaggregate' as select id,max($columnName) from $parentTableName group by id"""
-        .stripMargin)
-  }
-
-  test("test load into main table with pre-aggregate table") {
-    sql("DROP TABLE IF EXISTS maintable")
-    sql(
-      """
-        | CREATE TABLE maintable(id int, name string, city string, age int)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    createAllAggregateTables("maintable")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    checkAnswer(sql(s"select * from maintable_preagg_sum"),
-      Seq(Row(1, 31), Row(2, 27), Row(3, 70), Row(4, 55)))
-    checkAnswer(sql(s"select * from maintable_preagg_avg"),
-      Seq(Row(1, 31, 1), Row(2, 27, 1), Row(3, 70, 2), Row(4, 55, 2)))
-    checkAnswer(sql(s"select * from maintable_preagg_count"),
-      Seq(Row(1, 1), Row(2, 1), Row(3, 2), Row(4, 2)))
-    checkAnswer(sql(s"select * from maintable_preagg_min"),
-      Seq(Row(1, 31), Row(2, 27), Row(3, 35), Row(4, 26)))
-    checkAnswer(sql(s"select * from maintable_preagg_max"),
-      Seq(Row(1, 31), Row(2, 27), Row(3, 35), Row(4, 29)))
-    sql("drop table if exists maintable")
-  }
-
-  test("test load into main table with pre-aggregate table with dictionary_include") {
-    sql("drop table if exists maintable")
-    sql(
-      """
-        | CREATE TABLE maintable(id int, name string, city string, age int)
-        | STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('dictionary_include'='id')
-      """.stripMargin)
-    createAllAggregateTables("maintable")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    checkAnswer(sql(s"select * from maintable_preagg_sum"),
-      Seq(Row(1, 31), Row(2, 27), Row(3, 70), Row(4, 55)))
-    checkAnswer(sql(s"select * from maintable_preagg_avg"),
-      Seq(Row(1, 31, 1), Row(2, 27, 1), Row(3, 70, 2), Row(4, 55,2)))
-    checkAnswer(sql(s"select * from maintable_preagg_count"),
-      Seq(Row(1, 1), Row(2, 1), Row(3, 2), Row(4, 2)))
-    checkAnswer(sql(s"select * from maintable_preagg_min"),
-      Seq(Row(1, 31), Row(2, 27), Row(3, 35), Row(4, 26)))
-    checkAnswer(sql(s"select * from maintable_preagg_max"),
-      Seq(Row(1, 31), Row(2, 27), Row(3, 35), Row(4, 29)))
-    sql("drop table if exists maintable")
-  }
-
-  test("test load into main table with pre-aggregate table with single_pass") {
-    sql("drop table if exists maintable")
-    sql(
-      """
-        | CREATE TABLE maintable(id int, name string, city string, age int)
-        | STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('dictionary_include'='id')
-      """.stripMargin)
-    createAllAggregateTables("maintable")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable options('single_pass'='true')")
-    checkAnswer(sql(s"select * from maintable_preagg_sum"),
-      Seq(Row(1, 31), Row(2, 27), Row(3, 70), Row(4, 55)))
-    checkAnswer(sql(s"select * from maintable_preagg_avg"),
-      Seq(Row(1, 31, 1), Row(2, 27, 1), Row(3, 70, 2), Row(4, 55,2)))
-    checkAnswer(sql(s"select * from maintable_preagg_count"),
-      Seq(Row(1, 1), Row(2, 1), Row(3, 2), Row(4, 2)))
-    checkAnswer(sql(s"select * from maintable_preagg_min"),
-      Seq(Row(1, 31), Row(2, 27), Row(3, 35), Row(4, 26)))
-    checkAnswer(sql(s"select * from maintable_preagg_max"),
-      Seq(Row(1, 31), Row(2, 27), Row(3, 35), Row(4, 29)))
-    sql("drop table if exists maintable")
-  }
-
-  test("test load into main table with incremental load") {
-    sql("drop table if exists maintable")
-    sql(
-      """
-        | CREATE TABLE maintable(id int, name string, city string, age int)
-        | STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('dictionary_include'='id')
-      """.stripMargin)
-    createAllAggregateTables("maintable")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    checkAnswer(sql(s"select * from maintable_preagg_sum"),
-      Seq(Row(1, 31),
-        Row(2, 27),
-        Row(3, 70),
-        Row(4, 55),
-        Row(1, 31),
-        Row(2, 27),
-        Row(3, 70),
-        Row(4, 55)))
-    checkAnswer(sql(s"select * from maintable_preagg_avg"),
-      Seq(Row(1, 31, 1),
-        Row(2, 27, 1),
-        Row(3, 70, 2),
-        Row(4, 55, 2),
-        Row(1, 31, 1),
-        Row(2, 27, 1),
-        Row(3, 70, 2),
-        Row(4, 55, 2)))
-    checkAnswer(sql(s"select * from maintable_preagg_count"),
-      Seq(Row(1, 1), Row(2, 1), Row(3, 2), Row(4, 2), Row(1, 1), Row(2, 1), Row(3, 2), Row(4, 2)))
-    checkAnswer(sql(s"select * from maintable_preagg_min"),
-      Seq(Row(1, 31),
-        Row(2, 27),
-        Row(3, 35),
-        Row(4, 26),
-        Row(1, 31),
-        Row(2, 27),
-        Row(3, 35),
-        Row(4, 26)))
-    checkAnswer(sql(s"select * from maintable_preagg_max"),
-      Seq(Row(1, 31),
-        Row(2, 27),
-        Row(3, 35),
-        Row(4, 29),
-        Row(1, 31),
-        Row(2, 27),
-        Row(3, 35),
-        Row(4, 29)))
-  }
-
-  test("test to check if exception is thrown for direct load on pre-aggregate table") {
-    sql("drop table if exists maintable")
-    sql(
-      """
-        | CREATE TABLE maintable(id int, name string, city string, age int)
-        | STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('dictionary_include'='id')
-      """.stripMargin)
-    sql(
-      s"""create datamap preagg_sum on table maintable using 'preaggregate' as select id,sum(age) from maintable group by id"""
-        .stripMargin)
-    assert(intercept[RuntimeException] {
-      sql(s"insert into maintable_preagg_sum values(1, 30)")
-    }.getMessage.equalsIgnoreCase("Cannot insert/load data directly into pre-aggregate/child table"))
-  }
-
-  test("test whether all segments are loaded into pre-aggregate table if segments are set on main table") {
-    sql("DROP TABLE IF EXISTS maintable")
-    sql(
-      """
-        | CREATE TABLE maintable(id int, name string, city string, age int)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    sql(s"insert into maintable values(1, 'xyz', 'bengaluru', 26)")
-    sql(s"insert into maintable values(1, 'xyz', 'bengaluru', 26)")
-    sql("set carbon.input.segments.default.maintable=0")
-    sql(
-      s"""create datamap preagg_sum on table maintable using 'preaggregate' as select id, sum(age) from maintable group by id"""
-        .stripMargin)
-    sqlContext.sparkSession.catalog.clearCache()
-    sql("reset")
-    checkAnswer(sql("select * from maintable_preagg_sum"), Row(1, 52))
-  }
-
-  test("test if pre-aagregate is overwritten if main table is inserted with insert overwrite") {
-    sql("DROP TABLE IF EXISTS maintable")
-    sql(
-      """
-        | CREATE TABLE maintable(id int, name string, city string, age int)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    sql(
-      s"""create datamap preagg_sum on table maintable using 'preaggregate' as select id, sum(age) from maintable group by id"""
-        .stripMargin)
-    sql(s"insert into maintable values(1, 'xyz', 'bengaluru', 26)")
-    sql(s"insert into maintable values(1, 'xyz', 'bengaluru', 26)")
-    sql(s"insert overwrite table maintable values(1, 'xyz', 'delhi', 29)")
-    checkAnswer(sql("select * from maintable_preagg_sum"), Row(1, 29))
-  }
-
-  test("test load in aggregate table with Measure col") {
-    val originalBadRecordsAction = CarbonProperties.getInstance().getProperty(CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION)
-    CarbonProperties.getInstance().addProperty(CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION, "FAIL")
-    sql("drop table if exists y ")
-    sql("create table y(year int,month int,name string,salary int) stored by 'carbondata'")
-    sql("insert into y select 10,11,'babu',12")
-    sql("create datamap y1_sum1 on table y using 'preaggregate' as select year,name,sum(salary) from y group by year,name")
-    CarbonProperties.getInstance().addProperty(CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION, originalBadRecordsAction)
-  }
-
-  test("test partition load into main table with pre-aggregate table") {
-    sql("DROP TABLE IF EXISTS maintable")
-    sql(
-      """
-        | CREATE TABLE maintable(id int, city string, age int) partitioned by(name string)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    createAllAggregateTables("maintable")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    checkAnswer(sql(s"select * from maintable_preagg_sum"),
-      Seq(Row(1, 31), Row(2, 27), Row(3, 70), Row(4, 55)))
-    checkAnswer(sql(s"select * from maintable_preagg_avg"),
-      Seq(Row(1, 31, 1), Row(2, 27, 1), Row(3, 70, 2), Row(4, 55, 2)))
-    checkAnswer(sql(s"select * from maintable_preagg_count"),
-      Seq(Row(1, 1), Row(2, 1), Row(3, 2), Row(4, 2)))
-    checkAnswer(sql(s"select * from maintable_preagg_min"),
-      Seq(Row(1, 31), Row(2, 27), Row(3, 35), Row(4, 26)))
-    checkAnswer(sql(s"select * from maintable_preagg_max"),
-      Seq(Row(1, 31), Row(2, 27), Row(3, 35), Row(4, 29)))
-    sql("drop table if exists maintable")
-  }
-
-  test("test load into preaggregate table having group by clause") {
-    sql("DROP TABLE IF EXISTS maintable")
-    sql(
-      """
-        | CREATE TABLE maintable(id int, name string, city string, age int)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    sql(s"insert into maintable values(1, 'xyz', 'bengaluru', 26)")
-    sql(s"insert into maintable values(1, 'xyz', 'bengaluru', 26)")
-    sql("set carbon.input.segments.default.maintable=0")
-    sql(
-      s"""create datamap preagg_sum on table maintable using 'preaggregate' as select id, sum(age) from maintable group by id,name"""
-        .stripMargin)
-    sqlContext.sparkSession.catalog.clearCache()
-    sql("reset")
-    checkAnswer(sql("select * from maintable_preagg_sum"), Row(1, 52, "xyz"))
-  }
-
-  test("test pregarregate with spark adaptive execution ") {
-    if (Spark2TestQueryExecutor.spark.version.startsWith("2.3")) {
-      // enable adaptive execution
-      Spark2TestQueryExecutor.conf.set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "true")
-    }
-    sql("DROP TABLE IF EXISTS maintable")
-    sql(
-      """
-        | CREATE TABLE maintable(id int, name string, city string, age int)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    sql(
-      s"""create datamap preagg_sum on table maintable using 'preaggregate' as select id, sum(age) from maintable group by id,name"""
-        .stripMargin)
-    sql(s"insert into maintable values(1, 'xyz', 'bengaluru', 20)")
-    sql(s"insert into maintable values(1, 'xyz', 'bengaluru', 30)")
-
-    checkAnswer(sql("select id, sum(age) from maintable group by id, name"), Row(1, 50))
-    sql("drop datamap preagg_sum on table maintable")
-    sql("drop table maintable")
-    if (Spark2TestQueryExecutor.spark.version.startsWith("2.3")) {
-      // disable adaptive execution
-      Spark2TestQueryExecutor.conf.set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "false")
-    }
-  }
-
-
-test("check load and select for avg double datatype") {
-  sql("drop table if exists maintbl ")
-  sql("create table maintbl(year int,month int,name string,salary double) stored by 'carbondata' tblproperties('sort_scope'='Global_sort','table_blocksize'='23','sort_columns'='month,year,name')")
-  sql("insert into maintbl select 10,11,'babu',12.89")
-  sql("insert into maintbl select 10,11,'babu',12.89")
-  sql("create datamap maintbl_double on table maintbl using 'preaggregate' as select name,avg(salary) from maintbl group by name")
-  checkAnswer(sql("select name,avg(salary) from maintbl group by name"), Row("babu", 12.89))
-}
-
-  test("check load and select for avg int datatype") {
-    sql("drop table if exists maintbl ")
-    sql("create table maintbl(year int,month int,name string,salary int) stored by 'carbondata' tblproperties('sort_scope'='Global_sort','table_blocksize'='23','sort_columns'='month,year,name')")
-    sql("insert into maintbl select 10,11,'babu',12")
-    sql("insert into maintbl select 10,11,'babu',12")
-    sql("create datamap maintbl_double on table maintbl using 'preaggregate' as select name,avg(salary) from maintbl group by name")
-    checkAnswer(sql("select name,avg(salary) from maintbl group by name"), Row("babu", 12.0))
-  }
-
-  test("check load and select for avg bigint datatype") {
-    sql("drop table if exists maintbl ")
-    sql("create table maintbl(year int,month int,name string,salary bigint) stored by 'carbondata' tblproperties('sort_scope'='Global_sort','table_blocksize'='23','sort_columns'='month,year,name')")
-    sql("insert into maintbl select 10,11,'babu',12")
-    sql("insert into maintbl select 10,11,'babu',12")
-    sql("create datamap maintbl_double on table maintbl using 'preaggregate' as select name,avg(salary) from maintbl group by name")
-    checkAnswer(sql("select name,avg(salary) from maintbl group by name"), Row("babu", 12.0))
-  }
-
-  test("check load and select for avg short datatype") {
-    sql("drop table if exists maintbl ")
-    sql("create table maintbl(year int,month int,name string,salary short) stored by 'carbondata' tblproperties('sort_scope'='Global_sort','table_blocksize'='23','sort_columns'='month,year,name')")
-    sql("insert into maintbl select 10,11,'babu',12")
-    sql("insert into maintbl select 10,11,'babu',12")
-    sql("create datamap maintbl_double on table maintbl using 'preaggregate' as select name,avg(salary) from maintbl group by name")
-    checkAnswer(sql("select name,avg(salary) from maintbl group by name"), Row("babu", 12.0))
-  }
-
-  test("check load and select for avg float datatype") {
-    sql("drop table if exists maintbl ")
-    sql("create table maintbl(year int,month int,name string,salary float) stored by 'carbondata' tblproperties('sort_scope'='Global_sort','table_blocksize'='23','sort_columns'='month,year,name')")
-    sql("insert into maintbl select 10,11,'babu',12")
-    sql("insert into maintbl select 10,11,'babu',12")
-    val rows = sql("select name,avg(salary) from maintbl group by name").collect()
-    sql("create datamap maintbl_double on table maintbl using 'preaggregate' as select name,avg(salary) from maintbl group by name")
-    checkAnswer(sql("select name,avg(salary) from maintbl group by name"), rows)
-  }
-
-  test("create datamap with 'if not exists' after load data into mainTable and create datamap") {
-    sql("DROP TABLE IF EXISTS maintable")
-    sql(
-      """
-        | CREATE TABLE maintable(id int, name string, city string, age int)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql(
-      s"""
-         | create datamap preagg_sum
-         | on table maintable
-         | using 'preaggregate'
-         | as select id,sum(age) from maintable
-         | group by id
-       """.stripMargin)
-
-    sql(
-      s"""
-         | create datamap if not exists preagg_sum
-         | on table maintable
-         | using 'preaggregate'
-         | as select id,sum(age) from maintable
-         | group by id
-       """.stripMargin)
-
-    checkAnswer(sql(s"select * from maintable_preagg_sum"),
-      Seq(Row(1, 31), Row(2, 27), Row(3, 70), Row(4, 55)))
-    sql("drop table if exists maintable")
-  }
-
-  test("create datamap with 'if not exists' after create datamap and load data into mainTable") {
-    sql("DROP TABLE IF EXISTS maintable")
-    sql(
-      """
-        | CREATE TABLE maintable(id int, name string, city string, age int)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-
-    sql(
-      s"""
-         | create datamap preagg_sum
-         | on table maintable
-         | using 'preaggregate'
-         | as select id,sum(age) from maintable
-         | group by id
-       """.stripMargin)
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql(
-      s"""
-         | create datamap if not exists preagg_sum
-         | on table maintable
-         | using 'preaggregate'
-         | as select id,sum(age) from maintable
-         | group by id
-       """.stripMargin)
-
-    checkAnswer(sql(s"select * from maintable_preagg_sum"),
-      Seq(Row(1, 31), Row(2, 27), Row(3, 70), Row(4, 55)))
-    sql("drop table if exists maintable")
-  }
-
-  test("create datamap without 'if not exists' after load data into mainTable and create datamap") {
-    sql("DROP TABLE IF EXISTS maintable")
-    sql(
-      """
-        | CREATE TABLE maintable(id int, name string, city string, age int)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql(
-      s"""
-         | create datamap preagg_sum
-         | on table maintable
-         | using 'preaggregate'
-         | as select id,sum(age) from maintable
-         | group by id
-       """.stripMargin)
-
-    val e: Exception = intercept[MalformedDataMapCommandException] {
-      sql(
-        s"""
-           | create datamap preagg_sum
-           | on table maintable
-           | using 'preaggregate'
-           | as select id,sum(age) from maintable
-           | group by id
-       """.stripMargin)
-    }
-    assert(e.getMessage.contains("DataMap name 'preagg_sum' already exist"))
-    checkAnswer(sql(s"select * from maintable_preagg_sum"),
-      Seq(Row(1, 31), Row(2, 27), Row(3, 70), Row(4, 55)))
-    sql("drop table if exists maintable")
-  }
-
-  test("check load and select for avg int datatype and group by") {
-    sql("drop table if exists maintable ")
-    sql("CREATE TABLE maintable(id int, city string, age int) stored by 'carbondata'")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    val rows = sql("select age,avg(age) from maintable group by age").collect()
-    sql("create datamap maintbl_double on table maintable using 'preaggregate' as select avg(age) from maintable group by age")
-    checkAnswer(sql("select age,avg(age) from maintable group by age"), rows)
-    sql("drop table if exists maintable ")
-  }
-
-  test("test load into main table with pre-aggregate table: string") {
-    sql(
-      """
-        | CREATE TABLE main_table(
-        |     id INT,
-        |     name STRING,
-        |     city STRING,
-        |     age STRING)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    createAllAggregateTables("main_table")
-
-    sql(s"LOAD DATA LOCAL INPATH '$testData' INTO TABLE main_table")
-    checkAnswer(sql(s"SELECT * FROM main_table_preagg_sum"),
-      Seq(Row(1, 31), Row(2, 27), Row(3, 70), Row(4, 55)))
-    checkAnswer(sql(s"SELECT * FROM main_table_preagg_avg"),
-      Seq(Row(1, 31, 1), Row(2, 27, 1), Row(3, 70, 2), Row(4, 55, 2)))
-    checkAnswer(sql(s"SELECT * FROM main_table_preagg_count"),
-      Seq(Row(1, 1), Row(2, 1), Row(3, 2), Row(4, 2)))
-    checkAnswer(sql(s"SELECT * FROM main_table_preagg_min"),
-      Seq(Row(1, "31"), Row(2, "27"), Row(3, "35"), Row(4, "26")))
-    checkAnswer(sql(s"SELECT * FROM main_table_preagg_max"),
-      Seq(Row(1, "31"), Row(2, "27"), Row(3, "35"), Row(4, "29")))
-
-    // check select and match or not match pre-aggregate table
-    checkPreAggTable(sql("SELECT id, SUM(age) FROM main_table GROUP BY id"),
-      true, "main_table_preagg_sum")
-    checkPreAggTable(sql("SELECT id, SUM(age) FROM main_table GROUP BY id"),
-      false, "main_table_preagg_avg", "main_table")
-
-    checkPreAggTable(sql("SELECT id, AVG(age) FROM main_table GROUP BY id"),
-      true, "main_table_preagg_avg")
-    checkPreAggTable(sql("SELECT id, AVG(age) from main_table GROUP BY id"),
-      false, "main_table_preagg_sum", "main_table")
-
-    checkPreAggTable(sql("SELECT id, COUNT(age) FROM main_table GROUP BY id"),
-      true, "main_table_preagg_count")
-    checkPreAggTable(sql("SELECT id, COUNT(age) FROM main_table GROUP BY id"),
-      false, "main_table_preagg_sum", "main_table")
-
-    checkPreAggTable(sql("SELECT id, MIN(age) FROM main_table GROUP BY id"),
-      true, "main_table_preagg_min")
-    checkPreAggTable(sql("SELECT id, MIN(age) FROM main_table GROUP BY id"),
-      false, "main_table_preagg_sum", "main_table")
-
-    checkPreAggTable(sql("SELECT id, MAX(age) FROM main_table GROUP BY id"),
-      true, "main_table_preagg_max")
-    checkPreAggTable(sql("SELECT id, MAX(age) FROM main_table GROUP BY id"),
-      false, "main_table_preagg_sum", "main_table")
-
-    // sub query should match pre-aggregate table
-    checkPreAggTable(sql("SELECT SUM(age) FROM main_table"),
-      true, "main_table_preagg_sum")
-    checkPreAggTable(sql("SELECT SUM(age) FROM main_table"),
-      false, "main_table_preagg_avg", "main_table")
-
-    checkPreAggTable(sql("SELECT AVG(age) FROM main_table GROUP BY id"),
-      true, "main_table_preagg_avg")
-    checkPreAggTable(sql("SELECT AVG(age) from main_table GROUP BY id"),
-      false, "main_table_preagg_sum", "main_table")
-
-    checkPreAggTable(sql("SELECT COUNT(age) FROM main_table GROUP BY id"),
-      true, "main_table_preagg_count")
-    checkPreAggTable(sql("SELECT COUNT(age) FROM main_table GROUP BY id"),
-      false, "main_table_preagg_sum", "main_table")
-
-    checkPreAggTable(sql("SELECT MIN(age) FROM main_table GROUP BY id"),
-      true, "main_table_preagg_min")
-    checkPreAggTable(sql("SELECT MIN(age) FROM main_table GROUP BY id"),
-      false, "main_table_preagg_sum", "main_table")
-
-    checkPreAggTable(sql("SELECT MAX(age) FROM main_table GROUP BY id"),
-      true, "main_table_preagg_max")
-    checkPreAggTable(sql("SELECT MAX(age) FROM main_table GROUP BY id"),
-      false, "main_table_preagg_sum", "main_table")
-  }
-
-  test("test load into main table with pre-aggregate table: sum string column") {
-    sql(
-      """
-        | CREATE TABLE main_table(
-        |     id INT,
-        |     name STRING,
-        |     city STRING,
-        |     age STRING)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    createAllAggregateTables("main_table", "name")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' INTO TABLE main_table")
-
-    checkAnswer(sql(s"SELECT * FROM main_table_preagg_sum"),
-      Seq(Row(1, null), Row(2, null), Row(3, null), Row(4, null)))
-    checkAnswer(sql(s"SELECT * FROM main_table_preagg_avg"),
-      Seq(Row(1, null, 1.0), Row(2, null, 1.0), Row(3, null, 2.0), Row(4, null, 2.0)))
-    checkAnswer(sql(s"SELECT * FROM main_table_preagg_count"),
-      Seq(Row(1, 1), Row(2, 1), Row(3, 2), Row(4, 2)))
-    checkAnswer(sql(s"SELECT * FROM main_table_preagg_min"),
-      Seq(Row(1, "david"), Row(2, "eason"), Row(3, "jarry"), Row(4, "kunal")))
-    checkAnswer(sql(s"SELECT * FROM main_table_preagg_max"),
-      Seq(Row(1, "david"), Row(2, "eason"), Row(3, "jarry"), Row(4, "vishal")))
-
-    // check select and match or not match pre-aggregate table
-    checkPreAggTable(sql("SELECT id, SUM(name) FROM main_table GROUP BY id"),
-      true, "main_table_preagg_sum")
-    checkPreAggTable(sql("SELECT id, SUM(name) FROM main_table GROUP BY id"),
-      false, "main_table_preagg_avg", "main_table")
-
-    checkPreAggTable(sql("SELECT id, AVG(name) FROM main_table GROUP BY id"),
-      true, "main_table_preagg_avg")
-    checkPreAggTable(sql("SELECT id, AVG(name) from main_table GROUP BY id"),
-      false, "main_table_preagg_sum", "main_table")
-
-    checkPreAggTable(sql("SELECT id, COUNT(name) FROM main_table GROUP BY id"),
-      true, "main_table_preagg_count")
-    checkPreAggTable(sql("SELECT id, COUNT(name) FROM main_table GROUP BY id"),
-      false, "main_table_preagg_sum", "main_table")
-
-    checkPreAggTable(sql("SELECT id, MIN(name) FROM main_table GROUP BY id"),
-      true, "main_table_preagg_min")
-    checkPreAggTable(sql("SELECT id, MIN(name) FROM main_table GROUP BY id"),
-      false, "main_table_preagg_sum", "main_table")
-
-    checkPreAggTable(sql("SELECT id, MAX(name) FROM main_table GROUP BY id"),
-      true, "main_table_preagg_max")
-    checkPreAggTable(sql("SELECT id, MAX(name) FROM main_table GROUP BY id"),
-      false, "main_table_preagg_sum", "main_table")
-
-    // sub query should match pre-aggregate table
-    checkPreAggTable(sql("SELECT SUM(name) FROM main_table"),
-      true, "main_table_preagg_sum")
-    checkPreAggTable(sql("SELECT SUM(name) FROM main_table"),
-      false, "main_table_preagg_avg", "main_table")
-
-    checkPreAggTable(sql("SELECT AVG(name) FROM main_table GROUP BY id"),
-      true, "main_table_preagg_avg")
-    checkPreAggTable(sql("SELECT AVG(name) from main_table GROUP BY id"),
-      false, "main_table_preagg_sum", "main_table")
-
-    checkPreAggTable(sql("SELECT COUNT(name) FROM main_table GROUP BY id"),
-      true, "main_table_preagg_count")
-    checkPreAggTable(sql("SELECT COUNT(name) FROM main_table GROUP BY id"),
-      false, "main_table_preagg_sum", "main_table")
-
-    checkPreAggTable(sql("SELECT MIN(name) FROM main_table GROUP BY id"),
-      true, "main_table_preagg_min")
-    checkPreAggTable(sql("SELECT MIN(name) FROM main_table GROUP BY id"),
-      false, "main_table_preagg_sum", "main_table")
-
-    checkPreAggTable(sql("SELECT MAX(name) FROM main_table GROUP BY id"),
-      true, "main_table_preagg_max")
-    checkPreAggTable(sql("SELECT MAX(name) FROM main_table GROUP BY id"),
-      false, "main_table_preagg_sum", "main_table")
-  }
-
-  test("test whether all segments are loaded into pre-aggregate table if segments are set on main table 2") {
-    sql("DROP TABLE IF EXISTS segmaintable")
-    sql(
-      """
-        | CREATE TABLE segmaintable(
-        |     id INT,
-        |     name STRING,
-        |     city STRING,
-        |     age INT)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    sql(s"INSERT INTO segmaintable VALUES(1, 'xyz', 'bengaluru', 26)")
-
-    sql("set carbon.input.segments.default.segmaintable=0")
-    checkAnswer(sql(s"SELECT id, SUM(age) FROM segmaintable GROUP BY id"),
-      Seq(Row(1, 26)))
-    sql(
-      s"""
-         | CREATE DATAMAP preagg_sum
-         | ON TABLE segmaintable
-         | USING 'preaggregate'
-         | AS SELECT id, SUM(age)
-         | FROM segmaintable
-         | GROUP BY id
-       """.stripMargin)
-    checkPreAggTable(sql("SELECT id, SUM(age) FROM segmaintable GROUP BY id"),
-      false, "segmaintable_preagg_sum")
-    sqlContext.sparkSession.catalog.clearCache()
-    sql("reset")
-    checkAnswer(sql("SELECT * FROM segmaintable_preagg_sum"), Seq(Row(1, 26)))
-    checkPreAggTable(sql("SELECT id, SUM(age) FROM segmaintable GROUP BY id"),
-      true, "segmaintable_preagg_sum")
-  }
-
-  test("test whether all segments are loaded into pre-aggregate table if segments are set on main table 3") {
-    sql("DROP TABLE IF EXISTS segmaintable")
-    sql(
-      """
-        | CREATE TABLE segmaintable(
-        |     id INT,
-        |     name STRING,
-        |     city STRING,
-        |     age INT)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    sql(s"INSERT INTO segmaintable VALUES(1, 'xyz', 'bengaluru', 26)")
-
-    sql("set carbon.input.segments.default.segmaintable=0")
-    checkAnswer(sql(s"SELECT id, SUM(age) FROM segmaintable GROUP BY id"),
-      Seq(Row(1, 26)))
-    sql(
-      s"""
-         | CREATE DATAMAP preagg_sum
-         | ON TABLE segmaintable
-         | USING 'preaggregate'
-         | AS SELECT id, SUM(age)
-         | FROM segmaintable
-         | GROUP BY id
-       """.stripMargin)
-    sql(s"INSERT INTO segmaintable VALUES(1, 'xyz', 'bengaluru', 26)")
-    sqlContext.sparkSession.catalog.clearCache()
-    sql("reset")
-    checkAnswer(sql("SELECT * FROM segmaintable_preagg_sum"), Seq(Row(1, 26), Row(1, 26)))
-    checkPreAggTable(sql("SELECT id, SUM(age) FROM segmaintable GROUP BY id"),
-      true, "segmaintable_preagg_sum")
-  }
-
-  test("test whether all segments are loaded into pre-aggregate table if segments are set on main table 4") {
-    sql("DROP TABLE IF EXISTS segmaintable")
-    sql(
-      """
-        | CREATE TABLE segmaintable(
-        |     id INT,
-        |     name STRING,
-        |     city STRING,
-        |     age INT)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    sql(s"INSERT INTO segmaintable VALUES(1, 'xyz', 'bengaluru', 26)")
-    sql(s"INSERT INTO segmaintable VALUES(1, 'xyz', 'bengaluru', 26)")
-
-    //  check value before set segments
-    checkAnswer(sql(s"SELECT id, SUM(age) FROM segmaintable GROUP BY id"),
-      Seq(Row(1, 52)))
-
-    sql("set carbon.input.segments.default.segmaintable=0")
-    //  check value after set segments
-    checkAnswer(sql(s"SELECT id, SUM(age) FROM segmaintable GROUP BY id"),
-      Seq(Row(1, 26)))
-
-    sql(
-      s"""
-         | CREATE DATAMAP preagg_sum
-         | ON TABLE segmaintable
-         | USING 'preaggregate'
-         | AS SELECT id, SUM(age)
-         | FROM segmaintable
-         | GROUP BY id
-       """.stripMargin)
-    sql(s"INSERT INTO segmaintable VALUES(1, 'xyz', 'bengaluru', 26)")
-
-    checkAnswer(sql("SELECT * FROM segmaintable_preagg_sum"), Seq(Row(1, 52), Row(1, 26)))
-    checkAnswer(sql(s"SELECT id, SUM(age) FROM segmaintable GROUP BY id"),
-      Seq(Row(1, 26)))
-    checkPreAggTable(sql("SELECT id, SUM(age) FROM segmaintable GROUP BY id"),
-      false, "segmaintable_preagg_sum")
-    sqlContext.sparkSession.catalog.clearCache()
-    // reset
-    sql("reset")
-    checkAnswer(sql(s"SELECT id, SUM(age) FROM segmaintable GROUP BY id"),
-      Seq(Row(1, 78)))
-    checkPreAggTable(sql("SELECT id, SUM(age) FROM segmaintable GROUP BY id"),
-      true, "segmaintable_preagg_sum")
-  }
-
-  test("test whether all segments are loaded into pre-aggregate table: auto merge and input segment") {
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.ENABLE_AUTO_LOAD_MERGE, "true")
-    sqlContext.sparkSession.catalog.clearCache()
-    sql("reset")
-    sql("DROP TABLE IF EXISTS segmaintable")
-    sql(
-      """
-        | CREATE TABLE segmaintable(
-        |     id INT,
-        |     name STRING,
-        |     city STRING,
-        |     age INT)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    sql(s"INSERT INTO segmaintable VALUES(1, 'xyz', 'bengaluru', 26)")
-    sql("set carbon.input.segments.default.segmaintable=0")
-    sql(s"INSERT INTO segmaintable VALUES(1, 'xyz', 'bengaluru', 26)")
-    sql(s"INSERT INTO segmaintable VALUES(1, 'xyz', 'bengaluru', 26)")
-
-    //  check value before auto merge
-    checkAnswer(sql(s"SELECT id, SUM(age) FROM segmaintable GROUP BY id"),
-      Seq(Row(1, 26)))
-
-
-    sql(s"INSERT INTO segmaintable VALUES(1, 'xyz', 'bengaluru', 26)")
-
-    //  check value after set segments and auto merge
-    checkAnswer(sql(s"SELECT id, SUM(age) FROM segmaintable GROUP BY id"),
-      Seq.empty)
-
-    checkPreAggTable(sql("SELECT id, SUM(age) FROM segmaintable GROUP BY id"),
-      false, "segmaintable_preagg_sum")
-
-    sql(
-      s"""
-         | CREATE DATAMAP preagg_sum
-         | ON TABLE segmaintable
-         | USING 'preaggregate'
-         | AS SELECT id, SUM(age)
-         | FROM segmaintable
-         | GROUP BY id
-       """.stripMargin)
-
-    sql(s"INSERT INTO segmaintable VALUES(1, 'xyz', 'bengaluru', 26)")
-    sqlContext.sparkSession.catalog.clearCache()
-    // reset
-    sql("reset")
-    checkAnswer(sql(s"SELECT id, SUM(age) FROM segmaintable GROUP BY id"),
-      Seq(Row(1, 130)))
-    checkPreAggTable(sql("SELECT id, SUM(age) FROM segmaintable GROUP BY id"),
-      true, "segmaintable_preagg_sum")
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.ENABLE_AUTO_LOAD_MERGE, "false")
-  }
-
-  //TODO: need to check and fix
-  ignore("test whether all segments are loaded into pre-aggregate table: auto merge and no input segment") {
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.ENABLE_AUTO_LOAD_MERGE, "true")
-    sqlContext.sparkSession.catalog.clearCache()
-    sql("reset")
-    sql("DROP TABLE IF EXISTS segmaintable")
-    sql(
-      """
-        | CREATE TABLE segmaintable(
-        |     id INT,
-        |     name STRING,
-        |     city STRING,
-        |     age INT)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    sql(s"INSERT INTO segmaintable VALUES(1, 'xyz', 'bengaluru', 26)")
-    sql(s"INSERT INTO segmaintable VALUES(1, 'xyz', 'bengaluru', 26)")
-    sql(s"INSERT INTO segmaintable VALUES(1, 'xyz', 'bengaluru', 26)")
-
-    sql(
-      s"""
-         | CREATE DATAMAP preagg_sum
-         | ON TABLE segmaintable
-         | USING 'preaggregate'
-         | AS SELECT id, SUM(age)
-         | FROM segmaintable
-         | GROUP BY id
-       """.stripMargin)
-
-    //  check value before auto merge
-    checkAnswer(sql(s"SELECT id, SUM(age) FROM segmaintable GROUP BY id"),
-      Seq(Row(1, 78)))
-    checkPreAggTable(sql("SELECT id, SUM(age) FROM segmaintable GROUP BY id"),
-      true, "segmaintable_preagg_sum")
-
-    sql(s"INSERT INTO segmaintable VALUES(1, 'xyz', 'bengaluru', 26)")
-
-    //  check value after auto merge
-    checkPreAggTable(sql("SELECT id, SUM(age) FROM segmaintable GROUP BY id"),
-      true, "segmaintable_preagg_sum")
-
-    sql(s"INSERT INTO segmaintable VALUES(1, 'xyz', 'bengaluru', 26)")
-
-    checkAnswer(sql(s"SELECT id, SUM(age) FROM segmaintable GROUP BY id"),
-      Seq(Row(1, 130)))
-    checkPreAggTable(sql("SELECT id, SUM(age) FROM segmaintable GROUP BY id"),
-      true, "segmaintable_preagg_sum")
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.ENABLE_AUTO_LOAD_MERGE, "false")
-  }
-
-  test("test whether all segments are loaded into pre-aggregate table: create after auto merge and no input segment") {
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.ENABLE_AUTO_LOAD_MERGE, "true")
-    sqlContext.sparkSession.catalog.clearCache()
-    sql("reset")
-    sql("DROP TABLE IF EXISTS segmaintable")
-    sql(
-      """
-        | CREATE TABLE segmaintable(
-        |     id INT,
-        |     name STRING,
-        |     city STRING,
-        |     age INT)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    sql(s"INSERT INTO segmaintable VALUES(1, 'xyz', 'bengaluru', 26)")
-    sql(s"INSERT INTO segmaintable VALUES(1, 'xyz', 'bengaluru', 26)")
-    sql(s"INSERT INTO segmaintable VALUES(1, 'xyz', 'bengaluru', 26)")
-    sql(s"INSERT INTO segmaintable VALUES(1, 'xyz', 'bengaluru', 26)")
-    sql(s"INSERT INTO segmaintable VALUES(1, 'xyz', 'bengaluru', 26)")
-    sql(
-      s"""
-         | CREATE DATAMAP preagg_sum
-         | ON TABLE segmaintable
-         | USING 'preaggregate'
-         | AS SELECT id, SUM(age)
-         | FROM segmaintable
-         | GROUP BY id
-       """.stripMargin)
-
-    checkAnswer(sql(s"SELECT id, SUM(age) FROM segmaintable GROUP BY id"),
-      Seq(Row(1, 130)))
-    checkPreAggTable(sql("SELECT id, SUM(age) FROM segmaintable GROUP BY id"),
-      true, "segmaintable_preagg_sum")
-
-    sql(s"INSERT INTO segmaintable VALUES(1, 'xyz', 'bengaluru', 26)")
-    checkAnswer(sql(s"SELECT id, SUM(age) FROM segmaintable GROUP BY id"),
-      Seq(Row(1, 156)))
-    checkPreAggTable(sql("SELECT id, SUM(age) FROM segmaintable GROUP BY id"),
-      true, "segmaintable_preagg_sum")
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.ENABLE_AUTO_LOAD_MERGE, "false")
-  }
-
-  //TODO: need to check and fix
-  ignore("test whether all segments are loaded into pre-aggregate table: mixed, load, auto merge and input segment") {
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.ENABLE_AUTO_LOAD_MERGE, "true")
-    sqlContext.sparkSession.catalog.clearCache()
-    sql("reset")
-    sql("DROP TABLE IF EXISTS main_table")
-    sql(
-      """
-        | CREATE TABLE main_table(
-        |     id INT,
-        |     name STRING,
-        |     city STRING,
-        |     age INT)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    sql(s"INSERT INTO main_table VALUES(1, 'xyz', 'bengaluru', 26)")
-    sql(s"INSERT INTO main_table VALUES(1, 'xyz', 'bengaluru', 26)")
-
-    sql(s"LOAD DATA LOCAL INPATH '$testData' INTO TABLE main_table")
-
-    createAllAggregateTables("main_table", "age")
-    sql("set carbon.input.segments.default.main_table=0")
-
-    checkPreAggTable(sql("SELECT id, SUM(age) FROM main_table GROUP BY id"),
-      false, "main_table_preagg_sum")
-    checkAnswer(sql(s"SELECT id, SUM(age) FROM main_table GROUP BY id"),
-      Seq(Row(1, 26)))
-    sqlContext.sparkSession.catalog.clearCache()
-    sql("reset")
-    checkPreAggTable(sql("SELECT id, SUM(age) FROM main_table GROUP BY id"),
-      true, "main_table_preagg_sum")
-
-    sql(s"INSERT INTO main_table VALUES(1, 'xyz', 'bengaluru', 26)")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' INTO TABLE main_table")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' INTO TABLE main_table")
-
-    checkAnswer(sql(s"SELECT id, SUM(age) FROM main_table GROUP BY id"),
-      Seq(Row(1, 171), Row(2, 81), Row(3, 210), Row(4, 165)))
-    checkPreAggTable(sql("SELECT id, SUM(age) FROM main_table GROUP BY id"),
-      true, "main_table_preagg_sum")
-
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.ENABLE_AUTO_LOAD_MERGE, "false")
-  }
-
-  //TODO: need to check and fix
-  ignore("test whether all segments are loaded into pre-aggregate table: auto merge and check pre-aggregate segment") {
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.ENABLE_AUTO_LOAD_MERGE, "true")
-    sqlContext.sparkSession.catalog.clearCache()
-    sql("reset")
-    sql("DROP TABLE IF EXISTS main_table")
-    sql(
-      """
-        | CREATE TABLE main_table(
-        |     id INT,
-        |     name STRING,
-        |     city STRING,
-        |     age INT)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-
-    sql(s"INSERT INTO main_table VALUES(1, 'xyz', 'bengaluru', 26)")
-
-    sql(
-      s"""
-         | CREATE DATAMAP preagg_sum
-         | ON TABLE main_table
-         | USING 'preaggregate'
-         | AS SELECT id, SUM(age)
-         | FROM main_table
-         | GROUP BY id
-       """.stripMargin)
-
-    sql(s"INSERT INTO main_table VALUES(1, 'xyz', 'bengaluru', 26)")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' INTO TABLE main_table")
-
-    checkExistence(sql("show segments for table main_table_preagg_sum"), false, "Compacted")
-    sql(s"INSERT INTO main_table VALUES(1, 'xyz', 'bengaluru', 26)")
-
-    // check the data whether auto merge
-    checkAnswer(sql(s"SELECT id, SUM(age) FROM main_table GROUP BY id"),
-      Seq(Row(1, 109), Row(2, 27), Row(3, 70), Row(4, 55)))
-    checkExistence(sql("show segments for table main_table_preagg_sum"), true, "Compacted")
-
-    sql(s"LOAD DATA LOCAL INPATH '$testData' INTO TABLE main_table")
-    sql(s"LOAD DATA LOCAL INPATH '$testData' INTO TABLE main_table")
-
-    checkAnswer(sql(s"SELECT id, SUM(age) FROM main_table GROUP BY id"),
-      Seq(Row(1, 171), Row(2, 81), Row(3, 210), Row(4, 165)))
-    checkPreAggTable(sql("SELECT id, SUM(age) FROM main_table GROUP BY id"),
-      true, "main_table_preagg_sum")
-
-    checkExistence(sql("show segments for table main_table_preagg_sum"), true, "Compacted")
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.ENABLE_AUTO_LOAD_MERGE, "false")
-  }
-
-  test("test deferred rebuild is not supported for preagg") {
-    val baseTable = "maintable"
-    val preagg = "preaggtable"
-    sql(s"DROP TABLE IF EXISTS $baseTable")
-    sql(
-      s"""
-        | CREATE TABLE $baseTable(id int, name string, city string, age int)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    val deferredRebuildException = intercept[MalformedDataMapCommandException] {
-      sql(
-        s"""
-           | CREATE DATAMAP $preagg ON TABLE $baseTable
-           | USING 'preaggregate'
-           | WITH DEFERRED REBUILD
-           | AS select id, sum(age) from $baseTable group by id
-       """.stripMargin)
-    }
-    assert(deferredRebuildException.getMessage.contains(
-      s"DEFERRED REBUILD is not supported on this datamap $preagg with provider preaggregate"))
-
-    sql(
-      s"""
-         | CREATE DATAMAP $preagg ON TABLE $baseTable
-         | USING 'preaggregate'
-         | AS select id, sum(age) from $baseTable group by id
-       """.stripMargin)
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table $baseTable")
-    checkExistence(sql(s"SHOW DATAMAP ON TABLE $baseTable"), true, preagg, "preaggregate")
-    val exception = intercept[MalformedDataMapCommandException] {
-      sql(s"REBUILD DATAMAP $preagg ON TABLE $baseTable").show()
-    }
-    LOGGER.error(s"XU ${exception.getMessage}")
-    assert(exception.getMessage.contains(s"Non-lazy datamap $preagg does not support rebuild"))
-    sql(s"DROP TABLE IF EXISTS $baseTable")
-  }
-}
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateMisc.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateMisc.scala
deleted file mode 100644
index 55c06c0..0000000
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateMisc.scala
+++ /dev/null
@@ -1,81 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.integration.spark.testsuite.preaggregate
-
-import org.apache.spark.sql.{CarbonEnv, Row}
-import org.apache.spark.sql.test.util.QueryTest
-import org.scalatest.{BeforeAndAfterAll, Ignore}
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-
-@Ignore
-class TestPreAggregateMisc extends QueryTest with BeforeAndAfterAll {
-  override def beforeAll: Unit = {
-    sql("drop table if exists mainTable")
-    sql("CREATE TABLE mainTable(id int, name string, city string, age string) STORED BY 'org.apache.carbondata.format'")
-    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/measureinsertintotest.csv' into table mainTable")
-    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/measureinsertintotest.csv' into table mainTable")
-  }
-
-  test("test PreAggregate With Set Segments property") {
-    sql("create datamap agg1 on table mainTable using 'preaggregate' as select name,sum(age) from mainTable group by name")
-    sql("SET carbon.input.segments.default.mainTable=0")
-    checkAnswer(
-      sql("select sum(age) from mainTable"),
-      Seq(Row(183.0)))
-    sqlContext.sparkSession.catalog.clearCache()
-    sql("RESET")
-    sql("drop datamap agg1 on table mainTable")
-
-  }
-
-  test("check preagg tbl properties sort columns inherit from main tbl") {
-    sql("drop table if exists y ")
-    sql(
-      "create table y(year int,month int,name string,salary int) stored by 'carbondata' " +
-      "tblproperties('NO_INVERTED_INDEX'='name','sort_scope'='Global_sort'," +
-      "'table_blocksize'='23','Dictionary_include'='month','Dictionary_exclude'='year,name'," +
-      "'sort_columns'='month,year,name')")
-    sql("insert into y select 10,11,'babu',12")
-    sql(
-      "create datamap y1_sum1 on table y using 'preaggregate' as select year,month,name,sum" +
-      "(salary) from y group by year,month,name")
-
-    val carbonTable = CarbonEnv.getCarbonTable(Some("default"), "y")(sqlContext.sparkSession)
-    val datamaptable = CarbonEnv
-      .getCarbonTable(Some("default"), "y_y1_sum1")(sqlContext.sparkSession)
-
-    val sortcolumns = datamaptable.getTableInfo.getFactTable.getTableProperties
-      .get(CarbonCommonConstants.SORT_COLUMNS)
-    val sortcolummatch = sortcolumns != null && sortcolumns.equals("y_month,y_year,y_name")
-
-    val sortscope = datamaptable.getTableInfo.getFactTable.getTableProperties.get("sort_scope")
-    val sortscopematch = sortscope != null && sortscope.equals(
-      carbonTable.getTableInfo.getFactTable.getTableProperties.get("sort_scope"))
-    val blockSize = datamaptable.getTableInfo.getFactTable.getTableProperties
-      .get(CarbonCommonConstants.TABLE_BLOCKSIZE)
-    val blocksizematch = blockSize != null &&
-                         blockSize.equals(carbonTable.getTableInfo.getFactTable.getTableProperties.
-                           get(CarbonCommonConstants.TABLE_BLOCKSIZE))
-    assert(sortcolummatch && sortscopematch && blocksizematch)
-  }
-
-  override def afterAll: Unit = {
-    sql("DROP TABLE IF EXISTS mainTable")
-    sql("DROP TABLE IF EXISTS y")
-  }
-}
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateTableSelection.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateTableSelection.scala
deleted file mode 100644
index 2d2bde1..0000000
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateTableSelection.scala
+++ /dev/null
@@ -1,466 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.integration.spark.testsuite.preaggregate
-
-import java.io.File
-
-import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
-import org.apache.spark.sql.execution.datasources.LogicalRelation
-import org.apache.spark.sql.hive.CarbonRelation
-import org.apache.spark.sql.{CarbonDatasourceHadoopRelation, Row}
-import org.scalatest.BeforeAndAfterAll
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.metadata.schema.datamap.DataMapClassProvider.TIMESERIES
-import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.spark.util.SparkQueryTest
-
-class TestPreAggregateTableSelection extends SparkQueryTest with BeforeAndAfterAll {
-
-  val timeSeries = TIMESERIES.toString
-
-  override def beforeAll: Unit = {
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.ENABLE_QUERY_STATISTICS, "true")
-    sql("drop table if exists mainTable")
-    sql("drop table if exists mainTableavg")
-    sql("drop table if exists agg0")
-    sql("drop table if exists agg1")
-    sql("drop table if exists agg2")
-    sql("drop table if exists agg3")
-    sql("drop table if exists agg4")
-    sql("drop table if exists agg5")
-    sql("drop table if exists agg6")
-    sql("drop table if exists agg7")
-    sql("DROP TABLE IF EXISTS maintabledict")
-    sql("CREATE TABLE mainTable(id int, name string, city string, age string) STORED BY 'org.apache.carbondata.format'")
-    sql("create datamap agg0 on table mainTable using 'preaggregate' as select name from mainTable group by name")
-    sql("create datamap agg1 on table mainTable using 'preaggregate' as select name,sum(age) from mainTable group by name")
-    sql("create datamap agg2 on table mainTable using 'preaggregate' as select name,sum(id) from mainTable group by name")
-    sql("create datamap agg3 on table mainTable using 'preaggregate' as select name,count(id) from mainTable group by name")
-    sql("create datamap agg4 on table mainTable using 'preaggregate' as select name,sum(age),count(id) from mainTable group by name")
-    sql("create datamap agg5 on table mainTable using 'preaggregate' as select name,avg(age) from mainTable group by name")
-    sql("create datamap agg6 on table mainTable using 'preaggregate' as select name,min(age) from mainTable group by name")
-    sql("create datamap agg7 on table mainTable using 'preaggregate' as select name,max(age) from mainTable group by name")
-    sql("create datamap agg8 on table maintable using 'preaggregate' as select name, sum(id), avg(id) from maintable group by name")
-    sql("create datamap agg9 on table maintable using 'preaggregate' as select name, count(*) from maintable group by name")
-    sql("CREATE TABLE mainTableavg(id int, name string, city string, age bigint) STORED BY 'org.apache.carbondata.format'")
-    sql("create datamap agg0 on table mainTableavg using 'preaggregate' as select name,sum(age), avg(age) from mainTableavg group by name")
-    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/measureinsertintotest.csv' into table mainTable")
-    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/measureinsertintotest.csv' into table mainTableavg")
-  }
-
-  test("test PreAggregate table selection 1") {
-    val df = sql("select name from mainTable group by name")
-    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg0")
-  }
-
-  test("test PreAggregate table selection with count(*)") {
-    val df = sql("select name, count(*) from mainTable group by name")
-    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg9")
-  }
-
-  test("test PreAggregate table selection 2") {
-    val df = sql("select name from mainTable where name in (select name from mainTable) group by name")
-    preAggTableValidator(df.queryExecution.analyzed, "mainTable")
-  }
-
-  test("test PreAggregate table selection 3") {
-    val df = sql("select name from mainTable where name in (select name from mainTable group by name) group by name")
-    preAggTableValidator(df.queryExecution.analyzed, "mainTable")
-  }
-
-  test("test PreAggregate table selection 4") {
-    val df = sql("select name from mainTable where name in('vishal') group by name")
-    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg0")
-  }
-
-  test("test PreAggregate table selection 5") {
-    val df = sql("select name, sum(age) from mainTable group by name")
-    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg1")
-  }
-
-  test("test PreAggregate table selection with table alias") {
-    val df = sql("select name, sum(age) from mainTable as t1 group by name")
-    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg1")
-  }
-
-  test("test PreAggregate table selection 6") {
-    val df = sql("select sum(age) from mainTable group by name")
-    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg1")
-  }
-
-  test("test PreAggregate table selection 7") {
-    val df = sql("select sum(id) from mainTable group by name")
-    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg2")
-  }
-
-  test("test PreAggregate table selection 8") {
-    val df = sql("select count(id) from mainTable group by name")
-    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg3")
-  }
-
-  test("test PreAggregate table selection 9") {
-    val df = sql("select sum(age), count(id) from mainTable group by name")
-    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg4")
-  }
-
-  test("test PreAggregate table selection 10") {
-    val df = sql("select avg(age) from mainTable group by name")
-    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg5")
-  }
-
-  test("test PreAggregate table selection 11") {
-    val df = sql("select max(age) from mainTable group by name")
-    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg7")
-  }
-
-  test("test PreAggregate table selection 12") {
-    val df = sql("select min(age) from mainTable group by name")
-    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg6")
-  }
-
-  test("test PreAggregate table selection 13") {
-    val df = sql("select name, sum(age) from mainTable where city = 'Bangalore' group by name")
-    preAggTableValidator(df.queryExecution.analyzed, "mainTable")
-  }
-
-  test("test PreAggregate table selection 14") {
-    val df = sql("select sum(age) from mainTable")
-    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg1")
-  }
-
-  test("test PreAggregate table selection 15") {
-    val df = sql("select avg(age) from mainTable")
-    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg5")
-  }
-
-  test("test PreAggregate table selection 16") {
-    val df = sql("select max(age) from mainTable")
-    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg7")
-  }
-
-  test("test PreAggregate table selection 17") {
-    val df = sql("select min(age) from mainTable")
-    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg6")
-  }
-
-  test("test PreAggregate table selection 18") {
-    val df = sql("select count(id) from mainTable")
-    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg3")
-  }
-
-  test("test PreAggregate table selection 19: test sum and avg on same column should give proper results") {
-    val df = sql("select name, sum(id), avg(id) from maintable group by name")
-    checkAnswer(df, Seq(Row("david",1,1.0), Row("jarry",6,3.0), Row("kunal",4,4.0), Row("eason",2,2.0), Row("vishal",4,4.0)))
-    checkPreAggTable(df, false, "maintable_agg5", "maintable_agg1")
-    checkPreAggTable(df, true, "maintable_agg8")
-  }
-
-  test("test PreAggregate table selection 20") {
-    val df = sql("select name from mainTable group by name order by name")
-    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg0")
-  }
-
-  test("test PreAggregate table selection 21") {
-    val df = sql("select name as NewName from mainTable group by name order by name")
-    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg0")
-  }
-
-  test("test PreAggregate table selection 22") {
-    val df = sql("select name, sum(age) from mainTable group by name order by name")
-    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg1")
-  }
-
-  test("test PreAggregate table selection 23") {
-    val df = sql("select name as NewName, sum(age) as sum from mainTable group by name order by name")
-    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg1")
-  }
-
-  test("test PreAggregate table selection 24") {
-    val df = sql("select name as NewName, sum(age) as sum from mainTable where name='vishal' group by name order by name")
-    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg1")
-  }
-
-  test("test PreAggregate table selection 25") {
-    val df = sql("select name as NewName, sum(age) as sum from mainTable where city = 'Bangalore' group by name order by name")
-    preAggTableValidator(df.queryExecution.analyzed, "maintable")
-  }
-
-  test("test PreAggregate table selection 26") {
-    val df = sql("select name from mainTable where name='vishal' group by name order by name")
-    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg0")
-  }
-
-  test("test PreAggregate table selection 27") {
-    val df = sql("select name as NewName from mainTable where name='vishal' group by name order by name")
-    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg0")
-  }
-
-  test("test PreAggregate table selection 28") {
-    val df = sql("select name as NewName, sum(case when age=2016 then 1 else 0 end) as sum from mainTable group by name")
-    preAggTableValidator(df.queryExecution.analyzed, "maintable")
-  }
-
-  def preAggTableValidator(plan: LogicalPlan, actualTableName: String) : Unit ={
-    var isValidPlan = false
-    plan.transform {
-      // first check if any preaTable1 scala function is applied it is present is in plan
-      // then call is from create preaTable1regate table class so no need to transform the query plan
-      case ca:CarbonRelation =>
-        if (ca.isInstanceOf[CarbonDatasourceHadoopRelation]) {
-          val relation = ca.asInstanceOf[CarbonDatasourceHadoopRelation]
-          if(relation.carbonTable.getTableName.equalsIgnoreCase(actualTableName)) {
-            isValidPlan = true
-          }
-        }
-        ca
-      case logicalRelation:LogicalRelation =>
-        if(logicalRelation.relation.isInstanceOf[CarbonDatasourceHadoopRelation]) {
-          val relation = logicalRelation.relation.asInstanceOf[CarbonDatasourceHadoopRelation]
-          if(relation.carbonTable.getTableName.equalsIgnoreCase(actualTableName)) {
-            isValidPlan = true
-          }
-        }
-        logicalRelation
-    }
-    if(!isValidPlan) {
-      assert(false)
-    } else {
-      assert(true)
-    }
-  }
-
-
-  test("test PreAggregate table selection 29") {
-    val df = sql("select sum(id) from mainTable group by name")
-    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg2")
-  }
-
-  test("test PreAggregate table selection 30") {
-    val df = sql("select a.name from mainTable a group by a.name")
-    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg0")
-  }
-
-  test("test PreAggregate table selection 31") {
-    val df = sql("select a.name as newName from mainTable a group by a.name")
-    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg0")
-  }
-
-  test("test PreAggregate table selection 32") {
-    val df = sql("select a.name as newName from mainTable a  where a.name='vishal' group by a.name")
-    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg0")
-  }
-
- test("test PreAggregate table selection 33: Test query with math operation hitting fact table") {
-    val df =  sql("select sum(id)+count(id) from maintable")
-    preAggTableValidator(df.queryExecution.analyzed, "maintable")
-  }
-
-  test("test PreAggregate table selection 34: test if pre-agg table is hit with filter condition") {
-    sql("DROP TABLE IF EXISTS filtertable")
-    sql(
-      """
-        | CREATE TABLE filtertable(
-        |     id INT,
-        |     name STRING,
-        |     city STRING,
-        |     age STRING)
-        | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('dictionary_include'='name,age')
-      """.stripMargin)
-    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/measureinsertintotest.csv' into table filtertable")
-    sql(
-      """
-        | CREATE DATAMAP agg9
-        | ON TABLE filtertable
-        | USING 'preaggregate'
-        | AS SELECT name, age, SUM(age)
-        |     FROM filtertable
-        |     GROUP BY name, age
-      """.stripMargin)
-    val df = sql("SELECT name, SUM(age) FROM filtertable WHERE age = '29' GROUP BY name, age")
-    preAggTableValidator(df.queryExecution.analyzed, "filtertable_agg9")
-    checkAnswer(df, Row("vishal", 29))
-  }
-
-  test("test PreAggregate table selection 35: test pre-agg table with group by condition") {
-    sql("DROP TABLE IF EXISTS grouptable")
-    sql(
-      """
-        | CREATE TABLE grouptable(
-        |     id INT,
-        |     name STRING,
-        |     city STRING,
-        |     age STRING)
-        | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('dictionary_include'='name,age')
-      """.stripMargin)
-    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/measureinsertintotest.csv' into table grouptable")
-    sql(
-      """
-        | CREATE DATAMAP agg9
-        | ON TABLE grouptable
-        | USING 'preaggregate'
-        | AS SELECT SUM(id)
-        |     FROM grouptable
-        |     GROUP BY city
-      """.stripMargin)
-    val df = sql("SELECT SUM(id) FROM grouptable GROUP BY city")
-    preAggTableValidator(df.queryExecution.analyzed, "grouptable_agg9")
-    checkAnswer(df, Seq(Row(3), Row(3), Row(4), Row(7)))
-  }
-
-  test("test PreAggregate table selection 36: test PreAggregate table selection with timeseries and normal together") {
-    sql("drop table if exists maintabletime")
-    sql(
-      "create table maintabletime(year int,month int,name string,salary int,dob timestamp) stored" +
-      " by 'carbondata' tblproperties('sort_scope'='Global_sort','table_blocksize'='23'," +
-      "'sort_columns'='month,year,name')")
-    sql("insert into maintabletime select 10,11,'babu',12,'2014-01-01 00:00:00'")
-    sql(
-      "create datamap agg0 on table maintabletime using 'preaggregate' as select dob,name from " +
-      "maintabletime group by dob,name")
-
-  sql(
-    s"""
-       | CREATE DATAMAP agg1_year ON TABLE maintabletime
-       | USING '$timeSeries'
-       | DMPROPERTIES (
-       | 'EVENT_TIME'='dob',
-       | 'YEAR_GRANULARITY'='1')
-       | AS SELECT dob, name FROM maintabletime
-       | GROUP BY dob,name
-       """.stripMargin)
-
-  val df = sql("SELECT timeseries(dob,'year') FROM maintabletime GROUP BY timeseries(dob,'year')")
-  preAggTableValidator(df.queryExecution.analyzed, "maintabletime_agg1_year")
-  sql("DROP TABLE IF EXISTS maintabletime")
-
-  }
-
-  test("test table selection when unsupported aggregate function is present") {
-    sql("DROP TABLE IF EXISTS maintabletime")
-    sql(
-      "create table maintabletime(year int,month int,name string,salary int,dob string) stored" +
-      " by 'carbondata' tblproperties('sort_scope'='Global_sort','table_blocksize'='23'," +
-      "'sort_columns'='month,year,name')")
-    sql("insert into maintabletime select 10,11,'x',12,'2014-01-01 00:00:00'")
-    sql(
-      "create datamap agg0 on table maintabletime using 'preaggregate' as select name,sum(salary) from " +
-      "maintabletime group by name")
-
-    sql("select var_samp(name) from maintabletime  where name='Mikka' ")
-  }
-
-  test("test PreAggregate table selection 38: for sum and avg in aggregate table with bigint") {
-    val df = sql("select avg(age) from mainTableavg")
-    preAggTableValidator(df.queryExecution.analyzed, "mainTableavg_agg0")
-  }
-
-  test("test PreAggregate table selection for avg with maintable containing dictionary include for group by column") {
-    sql(
-      "create table maintabledict(year int,month int,name string,salary int,dob string) stored" +
-      " by 'carbondata' tblproperties('DICTIONARY_INCLUDE'='year')")
-    sql("insert into maintabledict select 10,11,'x',12,'2014-01-01 00:00:00'")
-    sql("insert into maintabledict select 10,11,'x',12,'2014-01-01 00:00:00'")
-    sql(
-      "create datamap aggdict on table maintabledict using 'preaggregate' as select year,avg(year) from " +
-      "maintabledict group by year")
-    val df = sql("select year,avg(year) from maintabledict group by year")
-    checkAnswer(df, Seq(Row(10,10.0)))
-  }
-
-  test("explain projection query") {
-    val rows = sql("explain select name, age from mainTable").collect()
-    assertResult(
-      """== CarbonData Profiler ==
-        |Table Scan on maintable
-        | - total: 1 blocks, 1 blocklets
-        | - filter: none
-        | - pruned by Main DataMap
-        |    - skipped: 0 blocks, 0 blocklets
-        |""".stripMargin)(rows(0).getString(0))
-  }
-
-  test("explain projection query hit datamap") {
-    val rows = sql("explain select name,sum(age) from mainTable group by name").collect()
-    assertResult(
-      """== CarbonData Profiler ==
-        |Query rewrite based on DataMap:
-        | - agg1 (preaggregate)
-        |Table Scan on maintable_agg1
-        | - total: 1 blocks, 1 blocklets
-        | - filter: none
-        | - pruned by Main DataMap
-        |    - skipped: 0 blocks, 0 blocklets
-        |""".stripMargin)(rows(0).getString(0))
-  }
-
-  test("explain filter query") {
-    sql("explain select name,sum(age) from mainTable where name = 'a' group by name").show(false)
-    val rows = sql("explain select name,sum(age) from mainTable where name = 'a' group by name").collect()
-    assertResult(
-      """== CarbonData Profiler ==
-        |Query rewrite based on DataMap:
-        | - agg1 (preaggregate)
-        |Table Scan on maintable_agg1
-        | - total: 1 blocks, 1 blocklets
-        | - filter: (maintable_name <> null and maintable_name = a)
-        | - pruned by Main DataMap
-        |    - skipped: 1 blocks, 1 blocklets
-        |""".stripMargin)(rows(0).getString(0))
-
-  }
-
-  test("explain query with multiple table") {
-    val query = "explain select t1.city,sum(t1.age) from mainTable t1, mainTableavg t2 " +
-                "where t1.name = t2.name and t1.id < 3 group by t1.city"
-    sql(query).show(false)
-    val rows = sql(query).collect()
-    assert(rows(0).getString(0).contains(
-      """
-        |Table Scan on maintable
-        | - total: 1 blocks, 1 blocklets
-        | - filter: ((id <> null and id < 3) and name <> null)
-        | - pruned by Main DataMap
-        |    - skipped: 0 blocks, 0 blocklets""".stripMargin))
-    assert(rows(0).getString(0).contains(
-      """
-        |Table Scan on maintableavg
-        | - total: 1 blocks, 1 blocklets
-        | - filter: name <> null
-        | - pruned by Main DataMap
-        |    - skipped: 0 blocks, 0 blocklets""".stripMargin))
-
-  }
-
-  override def afterAll: Unit = {
-    sql("drop table if exists mainTable")
-    sql("drop table if exists mainTable_avg")
-    sql("drop table if exists lineitem")
-    sql("DROP TABLE IF EXISTS maintabletime")
-    sql("DROP TABLE IF EXISTS maintabledict")
-    sql("DROP TABLE IF EXISTS mainTableavg")
-    sql("DROP TABLE IF EXISTS filtertable")
-    sql("DROP TABLE IF EXISTS grouptable")
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.ENABLE_QUERY_STATISTICS,
-        CarbonCommonConstants.ENABLE_QUERY_STATISTICS_DEFAULT)
-  }
-
-}
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateWithSubQuery.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateWithSubQuery.scala
deleted file mode 100644
index af12be9..0000000
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateWithSubQuery.scala
+++ /dev/null
@@ -1,102 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.integration.spark.testsuite.preaggregate
-
-import org.apache.spark.sql.CarbonDatasourceHadoopRelation
-import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
-import org.apache.spark.sql.execution.datasources.LogicalRelation
-import org.apache.spark.sql.test.util.QueryTest
-import org.scalatest.BeforeAndAfterAll
-
-
-class TestPreAggregateWithSubQuery extends QueryTest with BeforeAndAfterAll {
-
-  override def beforeAll: Unit = {
-    sql("DROP TABLE IF EXISTS mainTable")
-    sql("DROP TABLE IF EXISTS mainTable1")
-    sql("CREATE TABLE mainTable(id int, name string, city string, age string) STORED BY 'org.apache.carbondata.format'")
-    sql("CREATE TABLE mainTable1(id int, name string, city string, age string) STORED BY 'org.apache.carbondata.format'")
-    sql("create datamap agg0 on table mainTable using 'preaggregate' as select name,sum(age) from mainTable group by name")
-    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/measureinsertintotest.csv' into table mainTable")
-    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/measureinsertintotest.csv' into table mainTable1")
-  }
-
-  test("test sub query PreAggregate table selection 1") {
-    val df = sql(
-      """
-        | SELECT t2.newnewname AS newname
-        | FROM mainTable1 t1
-        | JOIN (
-        |     select
-        |       name AS newnewname,
-        |       sum(age) AS sum
-        |     FROM mainTable
-        |     GROUP BY name ) t2
-        | ON t1.name = t2.newnewname
-        | GROUP BY t2.newnewname
-      """.stripMargin)
-    matchTable(collectLogicalRelation(df.queryExecution.analyzed), "maintable_agg0")
-  }
-
-  test("test sub query PreAggregate table selection 2") {
-    val df = sql("select t1.name,t1.city from mainTable1 t1 join (select name as newnewname,sum(age) as sum from mainTable group by name )t2 on t1.name=t2.newnewname")
-    matchTable(collectLogicalRelation(df.queryExecution.analyzed), "maintable_agg0")
-  }
-
-  test("test sub query PreAggregate table selection 3") {
-    val df = sql("select t1.name,t2.sum from mainTable1 t1 join (select name as newnewname,sum(age) as sum from mainTable group by name )t2 on t1.name=t2.newnewname")
-    matchTable(collectLogicalRelation(df.queryExecution.analyzed), "maintable_agg0")
-  }
-
-  test("test sub query PreAggregate table selection 4") {
-    val df = sql("select t1.name,t2.sum from mainTable1 t1 join (select name,sum(age) as sum from mainTable group by name )t2 on t1.name=t2.name group by t1.name, t2.sum")
-    matchTable(collectLogicalRelation(df.queryExecution.analyzed), "maintable_agg0")
-  }
-
-  /**
-   * Below method will be used to collect all the logical relation from logical plan
-   * @param logicalPlan
-   * query logical plan
-   * @return all the logical relation
-   */
-  def collectLogicalRelation(logicalPlan: LogicalPlan) : Seq[LogicalRelation] = {
-    logicalPlan.collect{
-      case l:LogicalRelation => l
-    }
-  }
-
-  /**
-   * Below method will be used to match the logical relation
-   * @param logicalRelations
-   * all logical relation
-   * @param tableName
-   * table name
-   */
-  def matchTable(logicalRelations: Seq[LogicalRelation], tableName: String) {
-    assert(logicalRelations.exists {
-      case l:LogicalRelation  if l.relation.isInstanceOf[CarbonDatasourceHadoopRelation] =>
-        l.relation.asInstanceOf[CarbonDatasourceHadoopRelation].carbonTable.getTableName.
-          equalsIgnoreCase(tableName)
-    })
-  }
-
-  override def afterAll: Unit = {
-    sql("DROP TABLE IF EXISTS mainTable")
-    sql("DROP TABLE IF EXISTS mainTable1")
-  }
-
-}
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/primitiveTypes/TestAdaptiveEncodingForPrimitiveTypes.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/primitiveTypes/TestAdaptiveEncodingForPrimitiveTypes.scala
index 944de37..ca1a4ee 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/primitiveTypes/TestAdaptiveEncodingForPrimitiveTypes.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/primitiveTypes/TestAdaptiveEncodingForPrimitiveTypes.scala
@@ -392,22 +392,6 @@
     sql("drop table if exists negativeTable")
   }
 
-  test("test preaggregate datamap on the adaptive encoded column") {
-    sql("drop table if exists negativeTable")
-    sql("create table negativeTable (intColumn int,stringColumn string,shortColumn short) stored by 'carbondata' TBLPROPERTIES('SORT_COLUMNS'='intColumn,shortColumn')")
-    sql(s"load data inpath '${resourcesPath + "/dataWithNegativeValues.csv"}' into table negativeTable options('FILEHEADER'='intColumn,stringColumn,shortColumn')")
-    // create preagg datamap on the encoded column
-    sql("create datamap negativeTable_preAgg1 on table negativeTable USING 'preaggregate' AS select count(intColumn),sum(intColumn) from negativeTable")
-    sql("create datamap negativeTable_preAgg2 on table negativeTable USING 'preaggregate' AS select count(shortColumn),sum(shortColumn) from negativeTable")
-    checkAnswer(sql("select * from negativeTable"), sql("select * from negativeTable_Compare"))
-    checkAnswer(sql("select count(intColumn),sum(intColumn) from negativeTable"),
-      Seq(Row(6, 20000)))
-    checkAnswer(sql("select count(shortColumn),sum(shortColumn) from negativeTable"),
-      Seq(Row(6, 200)))
-    sql("drop table if exists negativeTable")
-  }
-
-
   override def afterAll: Unit = {
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.ENABLE_UNSAFE_SORT, CarbonCommonConstants.ENABLE_UNSAFE_SORT_DEFAULT)
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/timeseries/TestTimeSeriesCreateTable.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/timeseries/TestTimeSeriesCreateTable.scala
deleted file mode 100644
index eabe0f5..0000000
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/timeseries/TestTimeSeriesCreateTable.scala
+++ /dev/null
@@ -1,604 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.integration.spark.testsuite.timeseries
-
-import org.apache.spark.sql.AnalysisException
-import org.apache.spark.sql.test.util.QueryTest
-import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach}
-
-import org.apache.carbondata.common.exceptions.sql.{MalformedCarbonCommandException, MalformedDataMapCommandException}
-import org.apache.carbondata.core.metadata.schema.datamap.DataMapClassProvider.TIMESERIES
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.util.CarbonProperties
-
-class TestTimeSeriesCreateTable extends QueryTest with BeforeAndAfterAll with BeforeAndAfterEach{
-
-  val timeSeries = TIMESERIES.toString
-  var timestampFormat: String = _
-
-  override def beforeAll: Unit = {
-    timestampFormat = CarbonProperties.getInstance()
-      .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
-        CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT)
-    sql("DROP TABLE IF EXISTS mainTable")
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
-        CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT)
-    sql("CREATE TABLE mainTable(dataTime timestamp, name string, city string, age int) STORED BY 'org.apache.carbondata.format'")
-    sql(
-      s"""
-         | CREATE DATAMAP agg0_second ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         | 'EVENT_TIME'='dataTime',
-         | 'SECOND_GRANULARITY'='1')
-         | AS SELECT dataTime, SUM(age) FROM mainTable
-         | GROUP BY dataTime
-       """.stripMargin)
-    sql(
-      s"""
-         | CREATE DATAMAP agg0_hour ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         | 'EVENT_TIME'='dataTime',
-         | 'HOUR_GRANULARITY'='1')
-         | AS SELECT dataTime, SUM(age) FROM mainTable
-         | GROUP BY dataTime
-       """.stripMargin)
-    sql(
-      s"""
-         | CREATE DATAMAP agg0_day ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         | 'EVENT_TIME'='dataTime',
-         | 'day_granularity'='1')
-         | AS SELECT dataTime, SUM(age) FROM mainTable
-         | GROUP BY dataTime
-       """.stripMargin)
-    sql(
-      s"""
-         | CREATE DATAMAP agg0_month ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         | 'EVENT_TIME'='dataTime',
-         | 'month_granularity'='1')
-         | AS SELECT dataTime, SUM(age) FROM mainTable
-         | GROUP BY dataTime
-       """.stripMargin)
-    sql(
-      s"""
-         | CREATE DATAMAP agg0_year ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         | 'EVENT_TIME'='dataTime',
-         | 'year_granularity'='1')
-         | AS SELECT dataTime, SUM(age) FROM mainTable
-         | GROUP BY dataTime
-       """.stripMargin)
-  }
-
-  override def afterEach(): Unit = {
-    dropDataMaps("mainTable", "agg1_second", "agg1_minute",
-      "agg1_hour", "agg1_day", "agg1_month", "agg1_year")
-  }
-
-  test("test timeseries create table 1") {
-    checkExistence(sql("DESCRIBE FORMATTED mainTable_agg0_second"), true, "maintable_agg0_second")
-    sql("DROP DATAMAP agg0_second ON TABLE mainTable")
-  }
-
-  test("test timeseries create table 2") {
-    checkExistence(sql("DESCRIBE FORMATTED mainTable_agg0_hour"), true, "maintable_agg0_hour")
-    sql("DROP DATAMAP agg0_hour ON TABLE mainTable")
-  }
-
-  test("test timeseries create table 3") {
-    checkExistence(sql("DESCRIBE FORMATTED maintable_agg0_day"), true, "maintable_agg0_day")
-    sql("DROP DATAMAP agg0_day ON TABLE mainTable")
-  }
-
-  test("test timeseries create table 4") {
-    checkExistence(sql("DESCRIBE FORMATTED mainTable_agg0_month"), true, "maintable_agg0_month")
-    sql("DROP DATAMAP agg0_month ON TABLE mainTable")
-  }
-
-  test("test timeseries create table 5") {
-    checkExistence(sql("DESCRIBE FORMATTED mainTable_agg0_year"), true, "maintable_agg0_year")
-    sql("DROP DATAMAP agg0_year ON TABLE mainTable")
-  }
-
-  test("test timeseries create table 6: TIMESERIES should define time granularity") {
-    sql("DROP DATAMAP IF EXISTS agg0_second ON TABLE mainTable")
-    val e = intercept[MalformedCarbonCommandException] {
-      sql(
-        s"""CREATE DATAMAP agg0_second ON TABLE mainTable USING '$timeSeries'
-           |DMPROPERTIES (
-           |   'event_time'='dataTime',
-           |   'SEC_GRANULARITY'='1')
-           |AS SELECT dataTime, SUM(age) FROM mainTable
-           |GROUP BY dataTime
-        """.stripMargin)
-    }
-    assert(e.getMessage.contains("TIMESERIES should define time granularity"))
-  }
-
-  test("test timeseries create table 7: Granularity only support 1") {
-    sql("DROP DATAMAP IF EXISTS agg0_second ON TABLE mainTable")
-    val e = intercept[MalformedDataMapCommandException] {
-      sql(
-        s"""
-           | CREATE DATAMAP agg0_second ON TABLE mainTable
-           | USING '$timeSeries'
-           | DMPROPERTIES (
-           | 'EVENT_TIME'='dataTime',
-           | 'DAY_GRANULARITY'='1.5')
-           | AS SELECT dataTime, SUM(age) FROM mainTable
-           | GROUP BY dataTime
-       """.stripMargin)
-    }
-    assert(e.getMessage.equals("Granularity only support 1"))
-  }
-
-  test("test timeseries create table 8: Granularity only support 1") {
-    dropDataMaps("mainTable", "agg1_hour")
-    val e = intercept[MalformedCarbonCommandException] {
-      sql(
-        s"""CREATE DATAMAP agg1_hour ON TABLE mainTable USING '$timeSeries'
-           |DMPROPERTIES (
-           |   'event_time'='dataTime',
-           |   'HOUR_GRANULARITY'='hour=-2')
-           |AS SELECT dataTime, SUM(age) FROM mainTable
-           |GROUP BY dataTime
-        """.stripMargin)
-    }
-    assert(e.getMessage.contains("Granularity only support "))
-    checkExistence(sql("SHOW DATAMAP ON TABLE mainTable"), false, "maintable_agg1_hour")
-  }
-
-  test("test timeseries create table 9: SECOND_GRANULARITY is null") {
-    sql("DROP DATAMAP IF EXISTS agg1 ON TABLE mainTable")
-    val e = intercept[MalformedCarbonCommandException] {
-      sql(
-        s"""CREATE DATAMAP agg0_hour ON TABLE mainTable
-           |USING '$timeSeries'
-           |DMPROPERTIES (
-           |   'event_time'='dataTime',
-           |   'HOUR_GRANULARITY'='')
-           |AS SELECT dataTime, SUM(age) FROM mainTable
-           |GROUP BY dataTime
-        """.stripMargin)
-    }
-    assert(e.getMessage.contains("Granularity only support 1"))
-  }
-
-  test("test timeseries create table 10: Table already exists in database") {
-    val e = intercept[MalformedDataMapCommandException] {
-      sql(
-        s"""CREATE DATAMAP agg1_hour ON TABLE mainTable USING '$timeSeries'
-           |DMPROPERTIES (
-           |   'event_time'='dataTime',
-           |   'HOUR_GRANULARITY'='1')
-           |AS SELECT dataTime, SUM(age) FROM mainTable
-           |GROUP BY dataTime
-        """.stripMargin)
-      sql(
-        s"""CREATE DATAMAP agg1_hour ON TABLE mainTable USING '$timeSeries'
-           |DMPROPERTIES (
-           |   'event_time'='dataTime',
-           |   'HOUR_GRANULARITY'='1')
-           |AS SELECT dataTime, SUM(age) FROM mainTable
-           |GROUP BY dataTime
-        """.stripMargin)
-    }
-    assert(e.getMessage.contains(
-      "DataMap name 'agg1_hour' already exist"))
-  }
-
-  test("test timeseries create table 11: don't support create timeseries table on non timestamp") {
-    sql("DROP DATAMAP IF EXISTS agg0_hour ON TABLE mainTable")
-    val e = intercept[MalformedCarbonCommandException] {
-      sql(
-        s"""
-           | CREATE DATAMAP agg0_hour ON TABLE mainTable
-           | USING '$timeSeries'
-           | DMPROPERTIES (
-           | 'EVENT_TIME'='name',
-           | 'HOUR_GRANULARITY'='1')
-           | AS SELECT dataTime, SUM(age) FROM mainTable
-           | GROUP BY dataTime
-       """.stripMargin)
-    }
-    assert(e.getMessage.equals("Timeseries event time is only supported on Timestamp column"))
-  }
-
-  test("test timeseries create table 12: Time series column dataTime does not exists in select") {
-    sql("DROP DATAMAP IF EXISTS agg0_hour ON TABLE mainTable")
-    val e = intercept[MalformedCarbonCommandException] {
-      sql(
-        s"""
-           | CREATE DATAMAP agg0_hour ON TABLE mainTable
-           | USING '$timeSeries'
-           | DMPROPERTIES (
-           | 'EVENT_TIME'='dataTime',
-           | 'HOUR_GRANULARITY'='1')
-           | AS SELECT name, SUM(age) FROM mainTable
-           | GROUP BY name
-         """.stripMargin)
-    }
-    assert(e.getMessage.equals("Time series column dataTime does not exists in select"))
-  }
-
-  test("test timeseries create table 13: don't support create timeseries table on non timestamp") {
-    sql("DROP DATAMAP IF EXISTS agg0_hour ON TABLE mainTable")
-    val e = intercept[MalformedCarbonCommandException] {
-      sql(
-        s"""CREATE DATAMAP agg0_hour ON TABLE mainTable
-           |USING '$timeSeries'
-           |DMPROPERTIES (
-           |   'event_time'='name',
-           |   'HOUR_GRANULARITY'='1')
-           |AS SELECT name, SUM(age) FROM mainTable
-           |GROUP BY name
-        """.stripMargin)
-    }
-    assert(e.getMessage.contains("Timeseries event time is only supported on Timestamp column"))
-  }
-
-  test("test timeseries create table 14: USING") {
-    val e: Exception = intercept[MalformedDataMapCommandException] {
-      sql(
-        """CREATE DATAMAP agg0_hour ON TABLE mainTable
-          | USING 'abc'
-          | DMPROPERTIES (
-          |   'EVENT_TIME'='dataTime',
-          |   'HOUR_GRANULARITY'='1')
-          | AS SELECT dataTime, SUM(age) FROM mainTable
-          | GROUP BY dataTime
-        """.stripMargin)
-    }
-    assert(e.getMessage.equals("DataMap 'abc' not found"))
-  }
-
-  test("test timeseries create table 15: USING and catch MalformedCarbonCommandException") {
-    val e: Exception = intercept[MalformedCarbonCommandException] {
-      sql(
-        """CREATE DATAMAP agg0_hour ON TABLE mainTable
-          | USING 'abc'
-          | DMPROPERTIES (
-          |   'EVENT_TIME'='dataTime',
-          |   'HOUR_GRANULARITY'='1')
-          | AS SELECT dataTime, SUM(age) FROM mainTable
-          | GROUP BY dataTime
-        """.stripMargin)
-    }
-    assert(e.getMessage.equals("DataMap 'abc' not found"))
-  }
-
-  test("test timeseries create table 16: Only one granularity level can be defined 1") {
-    sql("DROP DATAMAP IF EXISTS agg0_hour ON TABLE mainTable")
-    val e: Exception = intercept[MalformedCarbonCommandException] {
-      sql(
-        s"""
-           | CREATE DATAMAP agg0_hour ON TABLE mainTable
-           | USING '$timeSeries'
-           | DMPROPERTIES (
-           | 'EVENT_TIME'='dataTime',
-           | 'SECOND_GRANULARITY'='1',
-           | 'HOUR_GRANULARITY'='1',
-           | 'DAY_GRANULARITY'='1',
-           | 'MONTH_GRANULARITY'='1',
-           | 'YEAR_GRANULARITY'='1')
-           | AS SELECT dataTime, SUM(age) FROM mainTable
-           | GROUP BY dataTime
-       """.stripMargin)
-    }
-    e.printStackTrace()
-    assert(e.getMessage.equals("Only one granularity level can be defined"))
-  }
-
-  test("test timeseries create table 17: Only one granularity level can be defined 2") {
-    sql("DROP DATAMAP IF EXISTS agg0_hour ON TABLE mainTable")
-    val e: Exception = intercept[MalformedDataMapCommandException] {
-      sql(
-        s"""
-           | CREATE DATAMAP agg0_hour ON TABLE mainTable
-           | USING '$timeSeries'
-           | DMPROPERTIES (
-           | 'EVENT_TIME'='dataTime',
-           | 'SECOND_GRANULARITY'='1',
-           | 'HOUR_GRANULARITY'='1')
-           | AS SELECT dataTime, SUM(age) FROM mainTable
-           | GROUP BY dataTime
-       """.stripMargin)
-    }
-    assert(e.getMessage.equals("Only one granularity level can be defined"))
-  }
-
-  test("test timeseries create table 18: Only one granularity level can be defined 3") {
-    sql("DROP DATAMAP IF EXISTS agg0_hour ON TABLE mainTable")
-    val e: Exception = intercept[MalformedDataMapCommandException] {
-      sql(
-        s"""
-           | CREATE DATAMAP agg0_hour ON TABLE mainTable
-           | USING '$timeSeries'
-           | DMPROPERTIES (
-           | 'EVENT_TIME'='dataTime',
-           | 'DAY_GRANULARITY'='1',
-           | 'HOUR_GRANULARITY'='1')
-           | AS SELECT dataTime, SUM(age) FROM mainTable
-           | GROUP BY dataTime
-       """.stripMargin)
-    }
-    assert(e.getMessage.equals("Only one granularity level can be defined"))
-  }
-
-  test("test timeseries create table 19: timeSeries should define time granularity") {
-    sql("DROP DATAMAP IF EXISTS agg0_hour ON TABLE mainTable")
-    val e = intercept[MalformedDataMapCommandException] {
-      sql(
-        s"""
-           | CREATE DATAMAP agg0_hour ON TABLE mainTable
-           | USING '$timeSeries'
-           | DMPROPERTIES (
-           | 'EVENT_TIME'='dataTime')
-           | AS SELECT dataTime, SUM(age) FROM mainTable
-           | GROUP BY dataTime
-       """.stripMargin)
-    }
-    assert(e.getMessage.equals(s"$timeSeries should define time granularity"))
-  }
-
-  test("test timeseries create table 20: should support if not exists, create when same table exists") {
-    sql("DROP DATAMAP IF EXISTS agg1 ON TABLE mainTable")
-
-    sql(
-      s"""
-         | CREATE DATAMAP agg1 ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         |   'EVENT_TIME'='dataTime',
-         |   'MONTH_GRANULARITY'='1')
-         | AS SELECT dataTime, SUM(age) FROM mainTable
-         | GROUP BY dataTime
-        """.stripMargin)
-    sql(
-      s"""
-         | CREATE DATAMAP IF NOT EXISTS agg1 ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         |   'EVENT_TIME'='dataTime',
-         |   'MONTH_GRANULARITY'='1')
-         |AS SELECT dataTime, SUM(age) FROM mainTable
-         |GROUP BY dataTime
-        """.stripMargin)
-    checkExistence(sql("SHOW DATAMAP ON TABLE mainTable"), true, "agg1")
-    checkExistence(sql("DESC FORMATTED mainTable_agg1"), true, "maintable_age_sum")
-  }
-
-  test("test timeseries create table 32: should support if not exists, create when same table not exists") {
-    sql("DROP DATAMAP IF EXISTS agg1_year ON TABLE mainTable")
-    sql(
-      s"""
-         |CREATE DATAMAP if not exists agg1_year ON TABLE mainTable
-         |USING '$timeSeries'
-         |DMPROPERTIES (
-         |   'event_time'='dataTime',
-         |   'YEAR_GRANULARITY'='1')
-         |AS SELECT dataTime, SUM(age) FROM mainTable
-         |GROUP BY dataTime
-        """.stripMargin)
-    checkExistence(sql("SHOW DATAMAP ON TABLE mainTable"), true, "agg1_year")
-    checkExistence(sql("DESC FORMATTED mainTable_agg1_year"), true, "maintable_age_sum")
-  }
-
-  test("test timeseries create table 20: don't support 'create datamap if exists'") {
-    val e: Exception = intercept[AnalysisException] {
-      sql(
-        s"""CREATE DATAMAP IF EXISTS agg2 ON TABLE mainTable
-          | USING '$timeSeries'
-          | DMPROPERTIES (
-          |   'EVENT_TIME'='dataTime',
-          |   'MONTH_GRANULARITY'='1')
-          | AS SELECT dataTime, SUM(age) FROM mainTable
-          | GROUP BY dataTime
-        """.stripMargin)
-    }
-    assert(e.getMessage.contains("identifier matching regex"))
-  }
-
-  test("test timeseries create table 26: test different data type") {
-    sql("drop table if exists dataTable")
-    sql(
-      s"""
-         | CREATE TABLE dataTable(
-         | shortField SHORT,
-         | booleanField BOOLEAN,
-         | intField INT,
-         | bigintField LONG,
-         | doubleField DOUBLE,
-         | stringField STRING,
-         | decimalField DECIMAL(18,2),
-         | charField CHAR(5),
-         | floatField FLOAT,
-         | dataTime timestamp
-         | )
-         | STORED BY 'carbondata'
-       """.stripMargin)
-
-
-    sql(
-      s"""CREATE DATAMAP agg0_hour ON TABLE dataTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         |   'event_time'='dataTime',
-         |   'HOUR_GRANULARITY'='1')
-         | AS SELECT
-         |   dataTime,
-         |   SUM(intField),
-         |   shortField,
-         |   booleanField,
-         |   intField,
-         |   bigintField,
-         |   doubleField,
-         |   stringField,
-         |   decimalField,
-         |   charField,
-         |   floatField
-         | FROM dataTable
-         | GROUP BY
-         |   dataTime,
-         |   shortField,
-         |   booleanField,
-         |   intField,
-         |   bigintField,
-         |   doubleField,
-         |   stringField,
-         |   decimalField,
-         |   charField,
-         |   floatField
-        """.stripMargin)
-    checkExistence(sql("SHOW DATAMAP ON TABLE dataTable"), true, "datatable_agg0_hour")
-    sql("DROP TABLE IF EXISTS dataTable")
-  }
-
-  test("test timeseries create table 27: test data map name") {
-    sql(
-      s"""CREATE DATAMAP agg1_hour ON TABLE mainTable
-         |USING '$timeSeries'
-         |DMPROPERTIES (
-         |   'event_time'='dataTime',
-         |   'HOUR_GRANULARITY'='1')
-         |AS SELECT dataTime, SUM(age) FROM mainTable
-         |GROUP BY dataTime
-        """.stripMargin)
-    checkExistence(sql("SHOW DATAMAP ON TABLE mainTable"), true, "agg1_hour")
-    checkExistence(sql("DESC FORMATTED mainTable_agg1_hour"), true, "maintable_age_sum")
-  }
-
-  test("test timeseries create table 28: event_time is null") {
-    sql("DROP DATAMAP IF EXISTS agg1 ON TABLE mainTable")
-    intercept[NullPointerException] {
-      sql(
-        s"""CREATE DATAMAP agg1 ON TABLE mainTable
-           |USING '$timeSeries'
-           |DMPROPERTIES (
-           |   'event_time'='',
-           |   'HOUR_GRANULARITY'='1')
-           |AS SELECT name, SUM(age) FROM mainTable
-           |GROUP BY name
-        """.stripMargin)
-    }
-  }
-
-  test("test timeseries create table 29: table not exists") {
-    sql("DROP DATAMAP IF EXISTS agg1 ON TABLE mainTable")
-    val e = intercept[AnalysisException] {
-      sql(
-        s"""CREATE DATAMAP agg1 ON TABLE mainTable
-           |USING '$timeSeries'
-           |DMPROPERTIES (
-           |   'event_time'='dataTime',
-           |   'HOUR_GRANULARITY'='1')
-           |AS SELECT dataTime, SUM(age) FROM mainTableNo
-           |GROUP BY dataTime
-        """.stripMargin)
-    }
-    assert(e.getMessage.contains("Table or view not found: mainTableNo"))
-  }
-
-  test("test timeseries create table 33: support event_time and granularity key with space") {
-    sql("DROP DATAMAP IF EXISTS agg1_month ON TABLE maintable")
-    sql(
-      s"""CREATE DATAMAP agg1_month ON TABLE mainTable
-         |USING '$timeSeries'
-         |DMPROPERTIES (
-         |   ' event_time '='dataTime',
-         |   ' MONTH_GRANULARITY '='1')
-         |AS SELECT dataTime, SUM(age) FROM mainTable
-         |GROUP BY dataTime
-        """.stripMargin)
-    checkExistence(sql("SHOW DATAMAP ON TABLE maintable"), true, "maintable_agg1_month")
-    sql("DROP DATAMAP IF EXISTS agg1_month ON TABLE maintable")
-  }
-
-
-  test("test timeseries create table 34: support event_time value with space") {
-    sql("DROP DATAMAP IF EXISTS agg1_month ON TABLE maintable")
-    sql(
-      s"""CREATE DATAMAP agg1_month ON TABLE mainTable
-         |USING '$timeSeries'
-         |DMPROPERTIES (
-         |   'event_time '=' dataTime',
-         |   'MONTH_GRANULARITY '='1')
-         |AS SELECT dataTime, SUM(age) FROM mainTable
-         |GROUP BY dataTime
-        """.stripMargin)
-    checkExistence(sql("SHOW DATAMAP ON TABLE maintable"), true, "maintable_agg1_month")
-    sql("DROP DATAMAP IF EXISTS agg1_month ON TABLE maintable")
-  }
-
-  test("test timeseries create table 35: support granularity value with space") {
-    sql("DROP DATAMAP IF EXISTS agg1_month ON TABLE maintable")
-    sql(
-      s"""CREATE DATAMAP agg1_month ON TABLE mainTable
-         |USING '$timeSeries'
-         |DMPROPERTIES (
-         |   'event_time '='dataTime',
-         |   'MONTH_GRANULARITY '=' 1')
-         |AS SELECT dataTime, SUM(age) FROM mainTable
-         |GROUP BY dataTime
-        """.stripMargin)
-    checkExistence(sql("SHOW DATAMAP ON TABLE maintable"), true, "maintable_agg1_month")
-    sql("DROP DATAMAP IF EXISTS agg1_month ON TABLE maintable")
-  }
-
-  test("test timeseries create table 36: support event_time and granularity value with space") {
-    sql("DROP DATAMAP IF EXISTS agg1_month ON TABLE maintable")
-    sql(
-      s"""
-         | CREATE DATAMAP agg1_month ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         |   'EVENT_TIME'='dataTime   ',
-         |   'MONTH_GRANULARITY'=' 1  ')
-         | AS SELECT dataTime, SUM(age) FROM mainTable
-         | GROUP BY dataTime
-        """.stripMargin)
-    checkExistence(sql("SHOW DATAMAP ON TABLE maintable"), true, "maintable_agg1_month")
-  }
-
-  test("test timeseries create table 37: unsupport event_time error value") {
-    sql("DROP DATAMAP IF EXISTS agg1_month ON TABLE maintable")
-    intercept[NullPointerException] {
-      sql(
-        s"""CREATE DATAMAP agg1_month ON TABLE mainTable USING '$timeSeries'
-           |DMPROPERTIES (
-           |   'event_time'='data Time',
-           |   'MONTH_GRANULARITY'='1')
-           |AS SELECT dataTime, SUM(age) FROM mainTable
-           |GROUP BY dataTime
-         """.stripMargin)
-    }
-    sql("DROP DATAMAP IF EXISTS agg1_month ON TABLE maintable")
-  }
-
-  override def afterAll: Unit = {
-    dropTable("mainTable")
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, timestampFormat)
-  }
-}
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/timeseries/TestTimeSeriesDropSuite.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/timeseries/TestTimeSeriesDropSuite.scala
deleted file mode 100644
index 33ac413..0000000
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/timeseries/TestTimeSeriesDropSuite.scala
+++ /dev/null
@@ -1,231 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.integration.spark.testsuite.timeseries
-
-import org.apache.spark.sql.AnalysisException
-import org.apache.spark.sql.test.util.QueryTest
-import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach}
-
-import org.apache.carbondata.common.exceptions.sql.{MalformedCarbonCommandException, NoSuchDataMapException}
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.spark.exception.ProcessMetaDataException
-
-class TestTimeSeriesDropSuite extends QueryTest with BeforeAndAfterAll with BeforeAndAfterEach {
-
-  val timeSeries = "timeseries"
-  var timestampFormat: String = _
-
-  override def beforeAll: Unit = {
-    timestampFormat = CarbonProperties.getInstance()
-      .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
-        CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT)
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
-        CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT)
-    dropTable("mainTable")
-    sql(
-      """
-        | CREATE TABLE mainTable(
-        |   dataTime TIMESTAMP,
-        |   name STRING,
-        |   city STRING,
-        |   age INT)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-  }
-
-  override def afterEach(): Unit = {
-    dropDataMaps("mainTable", "agg1_second", "agg1_minute",
-      "agg1_hour", "agg1_day", "agg1_month", "agg1_year")
-  }
-
-  test("test timeseries drop datamap 1: drop datamap should throw exception, maintable hasn't datamap") {
-    // DROP DATAMAP DataMapName if the DataMapName not exists
-    checkExistence(sql("SHOW DATAMAP ON TABLE mainTable"), false, "agg1_month")
-    val e: Exception = intercept[NoSuchDataMapException] {
-      sql(s"DROP DATAMAP agg1_month ON TABLE mainTable")
-    }
-    assert(e.getMessage.equals(
-      "Datamap with name agg1_month does not exist"))
-  }
-
-  test("test timeseries drop datamap 2: should support drop datamap IF EXISTS, maintable hasn't datamap") {
-    // DROP DATAMAP IF EXISTS DataMapName
-    checkExistence(sql("SHOW DATAMAP ON TABLE mainTable"), false, "agg1_month")
-    sql(s"DROP DATAMAP IF EXISTS agg1_month ON TABLE mainTable")
-    assert(true)
-  }
-
-  test("test timeseries drop datamap 3: should support drop datamap, maintable has datamap") {
-    sql(
-      s"""
-         | CREATE DATAMAP agg1_month ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         |   'event_time'='dataTime',
-         |   'MONTH_GRANULARITY'='1')
-         | AS SELECT dataTime, SUM(age) from mainTable
-         | GROUP BY dataTime
-       """.stripMargin)
-
-    // Before DROP DATAMAP
-    checkExistence(sql("SHOW DATAMAP ON TABLE mainTable"), true, "agg1_month")
-
-    // DROP DATAMAP DataMapName
-    sql(s"DROP DATAMAP agg1_month ON TABLE mainTable")
-    checkExistence(sql("SHOW DATAMAP ON TABLE mainTable"), false, "agg1_month")
-    val e: Exception = intercept[NoSuchDataMapException] {
-      sql(s"DROP DATAMAP agg1_month ON TABLE mainTable")
-    }
-    assert(e.getMessage.equals(
-      "Datamap with name agg1_month does not exist"))
-  }
-
-  test("test timeseries drop datamap 4: should support drop datamap with IF EXISTS, maintable has datamap") {
-    sql(
-      s"""
-         | CREATE DATAMAP agg1_month ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         |   'event_time'='dataTime',
-         |   'MONTH_GRANULARITY'='1')
-         | AS SELECT dataTime, SUM(age) from mainTable
-         | GROUP BY dataTime
-       """.stripMargin)
-    // DROP DATAMAP IF EXISTS DataMapName
-    checkExistence(sql("SHOW DATAMAP ON TABLE mainTable"), true, "agg1_month")
-
-    // DROP DATAMAP DataMapName
-    sql(s"DROP DATAMAP agg1_month ON TABLE mainTable")
-    checkExistence(sql("SHOW DATAMAP ON TABLE mainTable"), false, "agg1_month")
-    val e: Exception = intercept[MalformedCarbonCommandException] {
-      sql(s"DROP DATAMAP agg1_month ON TABLE mainTable")
-    }
-    assert(e.getMessage.equals(
-      "Datamap with name agg1_month does not exist"))
-  }
-
-  test("test timeseries drop datamap 5: drop datamap without IF EXISTS when table not exists, catch MalformedCarbonCommandException") {
-    // DROP DATAMAP DataMapName if the DataMapName not exists
-    checkExistence(sql("SHOW DATAMAP ON TABLE mainTable"), false, "agg1_month")
-    val e: Exception = intercept[MalformedCarbonCommandException] {
-      sql(s"DROP DATAMAP agg1_month ON TABLE mainTableNotExists")
-    }
-    assert(e.getMessage.contains(
-      "Dropping datamap agg1_month failed: Table or view 'maintablenotexists' not found"))
-  }
-
-  test("test timeseries drop datamap 6: drop datamap with IF EXISTS when table not exists, catch MalformedCarbonCommandException") {
-    // DROP DATAMAP DataMapName if the DataMapName not exists
-    checkExistence(sql("SHOW DATAMAP ON TABLE mainTable"), false, "agg1_month")
-    val e: Exception = intercept[MalformedCarbonCommandException] {
-      sql(s"DROP DATAMAP IF EXISTS agg1_month ON TABLE mainTableNotExists")
-    }
-    assert(e.getMessage.contains(
-      "Dropping datamap agg1_month failed: Table or view 'maintablenotexists' not found"))
-  }
-
-  test("test timeseries drop datamap 7: drop datamap should throw exception if table not exist, catch ProcessMetaDataException") {
-    // DROP DATAMAP DataMapName if the DataMapName not exists and
-    checkExistence(sql("SHOW DATAMAP ON TABLE mainTable"), false, "agg1_month")
-    val e: Exception = intercept[ProcessMetaDataException] {
-      sql(s"DROP DATAMAP agg1_month ON TABLE mainTableNotExists")
-    }
-    assert(e.getMessage.contains(
-      "Dropping datamap agg1_month failed: Table or view 'maintablenotexists' not found "))
-  }
-
-  test("test timeseries drop datamap 8: should throw exception if table not exist with IF EXISTS, catch ProcessMetaDataException") {
-    // DROP DATAMAP DataMapName if the DataMapName not exists
-    // DROP DATAMAP should throw exception if table not exist, even though there is IF EXISTS"
-    checkExistence(sql("SHOW DATAMAP ON TABLE mainTable"), false, "agg1_month")
-    val e: Exception = intercept[ProcessMetaDataException] {
-      sql(s"DROP DATAMAP IF EXISTS agg1_month ON TABLE mainTableNotExists")
-    }
-    assert(e.getMessage.contains(
-      "Dropping datamap agg1_month failed: Table or view 'maintablenotexists' not found "))
-  }
-
-  test("test timeseries drop datamap 9: drop datamap when table not exists, there are datamap in database") {
-    sql(
-      s"""CREATE DATAMAP agg1_minute ON TABLE mainTable
-         |USING '$timeSeries'
-         |DMPROPERTIES (
-         |   'event_time'='dataTime',
-         |   'minute_GRANULARITY'='1')
-         |AS SELECT dataTime, SUM(age) from mainTable
-         |GROUP BY dataTime
-      """.stripMargin)
-
-    // DROP DATAMAP DataMapName if the DataMapName not exists
-    checkExistence(sql("SHOW DATAMAP ON TABLE mainTable"), false, "agg1_month")
-    val e: Exception = intercept[ProcessMetaDataException] {
-      sql(s"DROP DATAMAP agg1_month ON TABLE mainTableNotExists")
-    }
-    assert(e.getMessage.contains(
-      "Dropping datamap agg1_month failed: Table or view 'maintablenotexists' not found"))
-  }
-
-
-  test("test timeseries drop datamap 10: drop datamap when table not exists, there are datamap in database") {
-    sql(
-      s"""CREATE DATAMAP agg3 ON TABLE mainTable
-         |USING '$timeSeries'
-         |DMPROPERTIES (
-         |   'event_time'='dataTime',
-         |   'month_GRANULARITY'='1')
-         |AS SELECT dataTime, SUM(age) from mainTable
-         |GROUP BY dataTime
-      """.stripMargin)
-
-    // DROP DATAMAP DataMapName if the DataMapName not exists
-    checkExistence(sql("SHOW DATAMAP ON TABLE mainTable"), false, "agg1_month")
-    val e: Exception = intercept[ProcessMetaDataException] {
-      sql(s"DROP DATAMAP IF EXISTS agg1_month ON TABLE mainTableNotExists")
-    }
-    assert(e.getMessage.contains(
-      "Dropping datamap agg1_month failed: Table or view 'maintablenotexists' not found"))
-  }
-
-  test("test timeseries drop datamap 11: drop datamap when table not exists, there are datamap in database") {
-    sql(
-      s"""
-         |CREATE DATAMAP agg4 ON TABLE mainTable
-         |USING '$timeSeries'
-         |DMPROPERTIES (
-         |   'event_time'='dataTime',
-         |   'month_GRANULARITY'='1')
-         |AS SELECT dataTime, SUM(age) from mainTable
-         |GROUP BY dataTime
-      """.stripMargin)
-
-    // DROP DATAMAP DataMapName if the DataMapName not exists
-    checkExistence(sql("SHOW DATAMAP ON TABLE mainTable"), false, "agg1_month")
-    val e: Exception = intercept[AnalysisException] {
-      sql(s"DROP DATAMAP IF NOT EXISTS agg1_month ON TABLE mainTableNotExists")
-    }
-    assert(e.getMessage.contains("failure"))
-  }
-
-  override def afterAll: Unit = {
-    dropTable("mainTable")
-    dropTable("mainTableNotExists")
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, timestampFormat)
-  }
-}
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/timeseries/TestTimeSeriesMatchStrategySuite.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/timeseries/TestTimeSeriesMatchStrategySuite.scala
deleted file mode 100644
index 55aa264..0000000
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/timeseries/TestTimeSeriesMatchStrategySuite.scala
+++ /dev/null
@@ -1,401 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.integration.spark.testsuite.timeseries
-
-import java.sql.Timestamp
-
-import org.apache.spark.sql.{CarbonDatasourceHadoopRelation, Row}
-import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
-import org.apache.spark.sql.execution.datasources.LogicalRelation
-import org.apache.spark.sql.hive.CarbonRelation
-import org.apache.spark.sql.test.util.QueryTest
-import org.apache.spark.util.SparkUtil4Test
-import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach}
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.util.CarbonProperties
-
-class TestTimeSeriesMatchStrategySuite extends QueryTest with BeforeAndAfterAll with BeforeAndAfterEach {
-
-  var timestampFormat: String = _
-
-  override def beforeAll: Unit = {
-    timestampFormat = CarbonProperties.getInstance()
-      .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
-        CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT)
-    SparkUtil4Test.createTaskMockUp(sqlContext)
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
-        CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT)
-
-  }
-
-  override protected def beforeEach(): Unit = {
-    sql("drop table if exists mainTable")
-    sql(
-      """
-        | CREATE TABLE mainTable(
-        |     mytime TIMESTAMP,
-        |     name STRING,
-        |     age INT)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/timeseriestest.csv' INTO TABLE mainTable")
-  }
-
-  val timeSeries = "TIMESERIES"
-
-  test("test timeseries match 1: select small one when create big_agg and then create small_agg") {
-
-    dropDataMaps("maintable", "big_agg", "small_agg")
-    sql(
-      s"""
-         | CREATE DATAMAP big_agg ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         |   'event_time'='mytime',
-         |   'MINUTE_GRANULARITY'='1')
-         | AS SELECT mytime, SUM(age), count(age), max(age), min(age)
-         | FROM mainTable
-         | GROUP BY mytime
-      """.stripMargin)
-
-    val df1 = sql(
-      """
-        | SELECT
-        |   timeseries(mytime,'minute') AS minuteLevel,
-        |   SUM(age) AS SUM
-        | FROM mainTable
-        | WHERE
-        |     timeseries(mytime,'minute')<'2016-02-23 09:02:00' and
-        |     timeseries(mytime,'minute')>='2016-02-23 09:01:00'
-        | GROUP BY
-        |   timeseries(mytime,'minute')
-        | ORDER BY
-        |   timeseries(mytime,'minute')
-      """.stripMargin)
-
-    checkAnswer(df1, Seq(Row(Timestamp.valueOf("2016-02-23 09:01:00"), 60)))
-
-    preAggTableValidator(df1.queryExecution.analyzed, "maintable_big_agg")
-
-    sql(
-      s"""
-         | CREATE DATAMAP small_agg ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         |   'event_time'='mytime',
-         |   'MINUTE_GRANULARITY'='1')
-         | AS SELECT mytime, SUM(age)
-         | FROM mainTable
-         | GROUP BY mytime
-      """.stripMargin)
-
-    val df2 = sql(
-      """
-        | SELECT
-        |   timeseries(mytime,'minute') AS minuteLevel,
-        |   SUM(age) AS SUM
-        | FROM mainTable
-        | WHERE
-        |     timeseries(mytime,'minute')<'2016-02-23 09:02:00' and
-        |     timeseries(mytime,'minute')>='2016-02-23 09:01:00'
-        | GROUP BY
-        |   timeseries(mytime,'minute')
-        | ORDER BY
-        |   timeseries(mytime,'minute')
-      """.stripMargin)
-
-    checkAnswer(df2, Seq(Row(Timestamp.valueOf("2016-02-23 09:01:00"), 60)))
-
-    preAggTableValidator(df2.queryExecution.analyzed, "maintable_small_agg")
-  }
-
-  test("test timeseries match 2: select small one when create small_agg and then create big_agg") {
-    dropDataMaps("maintable", "big_agg", "small_agg")
-
-    sql(
-      s"""
-         | CREATE DATAMAP small_agg ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         |   'event_time'='mytime',
-         |   'MINUTE_GRANULARITY'='1')
-         | AS SELECT mytime, SUM(age)
-         | FROM mainTable
-         | GROUP BY mytime
-      """.stripMargin)
-
-    val df2 = sql(
-      """
-        | SELECT
-        |   timeseries(mytime,'minute') AS minuteLevel,
-        |   SUM(age) AS SUM
-        | FROM mainTable
-        | WHERE
-        |     timeseries(mytime,'minute')<'2016-02-23 09:02:00' and
-        |     timeseries(mytime,'minute')>='2016-02-23 09:01:00'
-        | GROUP BY
-        |   timeseries(mytime,'minute')
-        | ORDER BY
-        |   timeseries(mytime,'minute')
-      """.stripMargin)
-
-    checkAnswer(df2, Seq(Row(Timestamp.valueOf("2016-02-23 09:01:00"), 60)))
-    preAggTableValidator(df2.queryExecution.analyzed, "maintable_small_agg")
-
-    sql(
-      s"""
-         | CREATE DATAMAP big_agg ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         |   'event_time'='mytime',
-         |   'MINUTE_GRANULARITY'='1')
-         | AS SELECT mytime, SUM(age), count(age), max(age), min(age)
-         | FROM mainTable
-         | GROUP BY mytime
-      """.stripMargin)
-
-    val df1 = sql(
-      """
-        | SELECT
-        |   timeseries(mytime,'minute') AS minuteLevel,
-        |   SUM(age) AS SUM
-        | FROM mainTable
-        | WHERE
-        |     timeseries(mytime,'minute')<'2016-02-23 09:02:00' and
-        |     timeseries(mytime,'minute')>='2016-02-23 09:01:00'
-        | GROUP BY
-        |   timeseries(mytime,'minute')
-        | ORDER BY
-        |   timeseries(mytime,'minute')
-      """.stripMargin)
-
-    checkAnswer(df1, Seq(Row(Timestamp.valueOf("2016-02-23 09:01:00"), 60)))
-
-    intercept[Exception]{
-      preAggTableValidator(df1.queryExecution.analyzed, "maintable_big_agg")
-    }
-    preAggTableValidator(df1.queryExecution.analyzed, "maintable_small_agg")
-  }
-
-  test("test timeseries match 3: select small one when create big_agg and then create small_agg") {
-
-    dropDataMaps("maintable", "big_agg", "small_agg")
-    sql(
-      s"""
-         | CREATE DATAMAP big_agg ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         |   'event_time'='mytime',
-         |   'MINUTE_GRANULARITY'='1')
-         | AS SELECT mytime, SUM(age), count(age), max(age), min(age)
-         | FROM mainTable
-         | GROUP BY mytime
-      """.stripMargin)
-
-    val df1 = sql(
-      """
-        | SELECT
-        |   timeseries(mytime,'minute') AS minuteLevel,
-        |   SUM(age) AS SUM
-        | FROM mainTable
-        | WHERE
-        |     timeseries(mytime,'minute')<'2016-02-23 09:02:00' and
-        |     timeseries(mytime,'minute')>='2016-02-23 09:01:00'
-        | GROUP BY
-        |   timeseries(mytime,'minute')
-        | ORDER BY
-        |   timeseries(mytime,'minute')
-      """.stripMargin)
-
-    checkAnswer(df1, Seq(Row(Timestamp.valueOf("2016-02-23 09:01:00"), 60)))
-
-    preAggTableValidator(df1.queryExecution.analyzed, "maintable_big_agg")
-
-    sql(
-      s"""
-         | CREATE DATAMAP small_agg ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         |   'event_time'='mytime',
-         |   'MINUTE_GRANULARITY'='1')
-         | AS SELECT mytime, SUM(age)
-         | FROM mainTable
-         | GROUP BY mytime
-      """.stripMargin)
-
-    val df2 = sql(
-      """
-        | SELECT
-        |   timeseries(mytime,'minute') AS minuteLevel,
-        |   SUM(age) AS SUM
-        | FROM mainTable
-        | WHERE
-        |     timeseries(mytime,'minute')<'2016-02-23 09:02:00' and
-        |     timeseries(mytime,'minute')>='2016-02-23 09:01:00'
-        | GROUP BY
-        |   timeseries(mytime,'minute')
-        | ORDER BY
-        |   timeseries(mytime,'minute')
-      """.stripMargin)
-
-    checkAnswer(df2, Seq(Row(Timestamp.valueOf("2016-02-23 09:01:00"), 60)))
-
-    preAggTableValidator(df2.queryExecution.analyzed, "maintable_small_agg")
-    intercept[Exception] {
-      preAggTableValidator(df1.queryExecution.analyzed, "maintable_small_agg")
-    }
-    preAggTableValidator(df1.queryExecution.analyzed, "maintable_big_agg")
-  }
-
-  test("test timeseries match 4: select small one when create big_agg, small_agg, and middle_agg") {
-
-    dropDataMaps("maintable", "big_agg", "small_agg", "middle_agg")
-    sql(
-      s"""
-         | CREATE DATAMAP big_agg ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         |   'event_time'='mytime',
-         |   'MINUTE_GRANULARITY'='1')
-         | AS SELECT mytime, SUM(age), count(age), max(age), min(age)
-         | FROM mainTable
-         | GROUP BY mytime
-      """.stripMargin)
-
-    val df1 = sql(
-      """
-        | SELECT
-        |   timeseries(mytime,'minute') AS minuteLevel,
-        |   SUM(age) AS SUM
-        | FROM mainTable
-        | WHERE
-        |     timeseries(mytime,'minute')<'2016-02-23 09:02:00' and
-        |     timeseries(mytime,'minute')>='2016-02-23 09:01:00'
-        | GROUP BY
-        |   timeseries(mytime,'minute')
-        | ORDER BY
-        |   timeseries(mytime,'minute')
-      """.stripMargin)
-
-    checkAnswer(df1, Seq(Row(Timestamp.valueOf("2016-02-23 09:01:00"), 60)))
-
-    preAggTableValidator(df1.queryExecution.analyzed, "maintable_big_agg")
-
-    sql(
-      s"""
-         | CREATE DATAMAP small_agg ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         |   'event_time'='mytime',
-         |   'MINUTE_GRANULARITY'='1')
-         | AS SELECT mytime, SUM(age)
-         | FROM mainTable
-         | GROUP BY mytime
-      """.stripMargin)
-
-    val df2 = sql(
-      """
-        | SELECT
-        |   timeseries(mytime,'minute') AS minuteLevel,
-        |   SUM(age) AS SUM
-        | FROM mainTable
-        | WHERE
-        |     timeseries(mytime,'minute')<'2016-02-23 09:02:00' and
-        |     timeseries(mytime,'minute')>='2016-02-23 09:01:00'
-        | GROUP BY
-        |   timeseries(mytime,'minute')
-        | ORDER BY
-        |   timeseries(mytime,'minute')
-      """.stripMargin)
-
-    checkAnswer(df2, Seq(Row(Timestamp.valueOf("2016-02-23 09:01:00"), 60)))
-
-    preAggTableValidator(df2.queryExecution.analyzed, "maintable_small_agg")
-    intercept[Exception] {
-      preAggTableValidator(df1.queryExecution.analyzed, "maintable_small_agg")
-    }
-    preAggTableValidator(df1.queryExecution.analyzed, "maintable_big_agg")
-
-    sql(
-      s"""
-         | CREATE DATAMAP middle_agg ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         |   'event_time'='mytime',
-         |   'MINUTE_GRANULARITY'='1')
-         | AS SELECT mytime, SUM(age), count(age)
-         | FROM mainTable
-         | GROUP BY mytime
-      """.stripMargin)
-
-    val df3 = sql(
-      """
-        | SELECT
-        |   timeseries(mytime,'minute') AS minuteLevel,
-        |   SUM(age) AS SUM
-        | FROM mainTable
-        | WHERE
-        |     timeseries(mytime,'minute')<'2016-02-23 09:02:00' and
-        |     timeseries(mytime,'minute')>='2016-02-23 09:01:00'
-        | GROUP BY
-        |   timeseries(mytime,'minute')
-        | ORDER BY
-        |   timeseries(mytime,'minute')
-      """.stripMargin)
-
-    preAggTableValidator(df3.queryExecution.analyzed, "maintable_small_agg")
-  }
-
-  def preAggTableValidator(plan: LogicalPlan, actualTableName: String) : Unit ={
-    var isValidPlan = false
-    plan.transform {
-      // first check if any preaTable1 scala function is applied it is present is in plan
-      // then call is FROM create preaTable1regate table class so no need to transform
-      // the query plan
-      case ca:CarbonRelation =>
-        if (ca.isInstanceOf[CarbonDatasourceHadoopRelation]) {
-          val relation = ca.asInstanceOf[CarbonDatasourceHadoopRelation]
-          if(relation.carbonTable.getTableName.equalsIgnoreCase(actualTableName)) {
-            isValidPlan = true
-          }
-        }
-        ca
-      case logicalRelation:LogicalRelation =>
-        if(logicalRelation.relation.isInstanceOf[CarbonDatasourceHadoopRelation]) {
-          val relation = logicalRelation.relation.asInstanceOf[CarbonDatasourceHadoopRelation]
-          if(relation.carbonTable.getTableName.equalsIgnoreCase(actualTableName)) {
-            isValidPlan = true
-          }
-        }
-        logicalRelation
-    }
-    if(!isValidPlan) {
-      assert(false)
-    } else {
-      assert(true)
-    }
-  }
-
-  override def afterAll: Unit = {
-    dropDataMaps("maintable", "agg0_second", "agg0_hour", "agg0_day", "agg0_month", "agg0_year")
-    sql("drop table if exists mainTable")
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, timestampFormat)
-  }
-}
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/timeseries/TestTimeSeriesUnsupportedSuite.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/timeseries/TestTimeSeriesUnsupportedSuite.scala
deleted file mode 100644
index 7fa2672..0000000
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/timeseries/TestTimeSeriesUnsupportedSuite.scala
+++ /dev/null
@@ -1,265 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.integration.spark.testsuite.timeseries
-
-import java.sql.Timestamp
-
-import org.apache.spark.sql.Row
-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.util.CarbonProperties
-
-class TestTimeSeriesUnsupportedSuite extends QueryTest with BeforeAndAfterAll with BeforeAndAfterEach {
-
-  val timeSeries = "TIMESERIES"
-  var timestampFormat: String = _
-
-  override def beforeAll: Unit = {
-    timestampFormat = CarbonProperties.getInstance()
-      .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
-        CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT)
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
-        CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT)
-  }
-
-  override def beforeEach(): Unit = {
-    sql("drop table if exists mainTable")
-    sql(
-      """
-        | CREATE TABLE mainTable(
-        |   mytime TIMESTAMP,
-        |   name STRING,
-        |   age INT)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-  }
-
-  test("test timeseries unsupported 1: don't support insert") {
-    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/timeseriestest.csv' INTO TABLE mainTable")
-    sql(
-      s"""CREATE DATAMAP agg1_minute ON TABLE mainTable
-         |USING '$timeSeries'
-         |DMPROPERTIES (
-         |   'event_time'='mytime',
-         |   'MINUTE_GRANULARITY'='1')
-         |AS SELECT mytime, SUM(age) FROM mainTable
-         |GROUP BY mytime
-        """.stripMargin)
-    checkExistence(sql("SHOW DATAMAP ON TABLE mainTable"), true, "maintable_agg1_minute")
-    checkAnswer(sql("SELECT * FROM maintable_agg1_minute"),
-      Seq(Row(Timestamp.valueOf("2016-02-23 09:01:00.0"), 60),
-        Row(Timestamp.valueOf("2016-02-23 09:02:00.0"), 140)))
-    val e = intercept[UnsupportedOperationException] {
-      sql(s"INSERT INTO maintable_agg1_minute VALUES('2016-02-23 09:01:00.0', 60)")
-    }
-    assert(e.getMessage.equalsIgnoreCase(
-      "Cannot insert/load data directly into pre-aggregate/child table"))
-
-    // check value after inserting
-    checkAnswer(sql("SELECT * FROM maintable_agg1_minute"),
-      Seq(Row(Timestamp.valueOf("2016-02-23 09:01:00.0"), 60),
-        Row(Timestamp.valueOf("2016-02-23 09:02:00.0"), 140)))
-  }
-
-  test("test timeseries unsupported 2: don't support insert") {
-    sql(
-      s"""CREATE DATAMAP agg1_minute ON TABLE mainTable
-         |USING '$timeSeries'
-         |DMPROPERTIES (
-         |   'event_time'='mytime',
-         |   'MINUTE_GRANULARITY'='1')
-         |AS SELECT mytime, SUM(age) FROM mainTable
-         |GROUP BY mytime
-        """.stripMargin)
-    checkExistence(sql("SHOW DATAMAP ON TABLE mainTable"), true, "maintable_agg1_minute")
-    val e = intercept[UnsupportedOperationException] {
-      sql(s"INSERT INTO maintable_agg1_minute VALUES('2016-02-23 09:01:00.0', 60)")
-    }
-    assert(e.getMessage.equalsIgnoreCase(
-      "Cannot insert/load data directly into pre-aggregate/child table"))
-  }
-
-  test("test timeseries unsupported 3: don't support insert") {
-    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/timeseriestest.csv' INTO TABLE mainTable")
-    sql(
-      s"""CREATE DATAMAP agg1_minute ON TABLE mainTable
-         |USING '$timeSeries'
-         |DMPROPERTIES (
-         |   'event_time'='mytime',
-         |   'MINUTE_GRANULARITY'='1')
-         |AS SELECT
-         |    mytime,
-         |    name,
-         |    SUM(age)
-         |from mainTable
-         |GROUP BY mytime, name
-        """.stripMargin)
-    checkExistence(sql("SHOW DATAMAP ON TABLE mainTable"), true, "maintable_agg1_minute")
-
-    val e = intercept[UnsupportedOperationException] {
-      sql(s"INSERT INTO maintable_agg1_minute VALUES('2016-02-23 09:01:00.0', 'hello', 60)")
-    }
-    assert(e.getMessage.equalsIgnoreCase(
-      "Cannot insert/load data directly into pre-aggregate/child table"))
-  }
-
-  test("test timeseries unsupported 4: don't support load") {
-    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/timeseriestest.csv' INTO TABLE mainTable")
-    sql(
-      s"""CREATE DATAMAP agg1_minute ON TABLE mainTable
-         |USING '$timeSeries'
-         |DMPROPERTIES (
-         |   'event_time'='mytime',
-         |   'MINUTE_GRANULARITY'='1')
-         |AS SELECT
-         |    mytime,
-         |    name,
-         |    SUM(age) AS age
-         |from mainTable
-         |GROUP BY mytime, name
-        """.stripMargin)
-    checkExistence(sql("SHOW DATAMAP ON TABLE mainTable"), true, "maintable_agg1_minute")
-
-    val e = intercept[UnsupportedOperationException] {
-      sql(
-        s"""
-           | LOAD DATA LOCAL INPATH '$resourcesPath/timeseriestest.csv'
-           | INTO TABLE maintable_agg1_minute
-           | OPTIONS('FILEHEADER'='maintable_mytime,maintable_name,maintable_age_SUM')
-         """.stripMargin)
-    }
-    assert(e.getMessage.equalsIgnoreCase(
-      "Cannot insert/load data directly into pre-aggregate/child table"))
-  }
-
-  test("test timeseries unsupported 5: don't support update") {
-    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/timeseriestest.csv' INTO TABLE mainTable")
-    sql(
-      s"""CREATE DATAMAP agg1_minute ON TABLE mainTable
-         |USING '$timeSeries'
-         |DMPROPERTIES (
-         |   'event_time'='mytime',
-         |   'MINUTE_GRANULARITY'='1')
-         |AS SELECT mytime, SUM(age) FROM mainTable
-         |GROUP BY mytime
-        """.stripMargin)
-    checkAnswer(sql("SELECT * FROM maintable_agg1_minute"),
-      Seq(Row(Timestamp.valueOf("2016-02-23 09:01:00.0"), 60),
-        Row(Timestamp.valueOf("2016-02-23 09:02:00.0"), 140)))
-    val e = intercept[Exception] {
-      sql("update maintable_agg1_minute SET (maintable_age_SUM) = (maintable_age_SUM+1)").show()
-    }
-    assert(e.getMessage.equalsIgnoreCase(
-      "Update operation is not supported for pre-aggregate table"))
-
-    // check value after inserting
-    checkAnswer(sql("SELECT * FROM maintable_agg1_minute"),
-      Seq(Row(Timestamp.valueOf("2016-02-23 09:01:00.0"), 60),
-        Row(Timestamp.valueOf("2016-02-23 09:02:00.0"), 140)))
-  }
-
-  test("test timeseries unsupported 6: don't support update") {
-    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/timeseriestest.csv' INTO TABLE mainTable")
-    sql(
-      s"""CREATE DATAMAP agg1_minute ON TABLE mainTable
-         |USING '$timeSeries'
-         |DMPROPERTIES (
-         |   'event_time'='mytime',
-         |   'MINUTE_GRANULARITY'='1')
-         |AS SELECT mytime, SUM(age) FROM mainTable
-         |GROUP BY mytime
-        """.stripMargin)
-    checkAnswer(sql("SELECT * FROM maintable_agg1_minute"),
-      Seq(Row(Timestamp.valueOf("2016-02-23 09:01:00.0"), 60),
-        Row(Timestamp.valueOf("2016-02-23 09:02:00.0"), 140)))
-    val e = intercept[Exception] {
-      sql(
-        """
-          | update maintable_agg1_minute
-          | SET (maintable_mytime, maintable_age_SUM)=('2016-02-23 09:11:00.0', 160)
-          | WHERE maintable_age_SUM = '60'
-        """.stripMargin).show
-    }
-    checkAnswer(sql("SELECT * FROM maintable_agg1_minute"),
-      Seq(Row(Timestamp.valueOf("2016-02-23 09:01:00.0"), 60),
-        Row(Timestamp.valueOf("2016-02-23 09:02:00.0"), 140)))
-    assert(e.getMessage.equalsIgnoreCase(
-      "Update operation is not supported for pre-aggregate table"))
-  }
-
-  test("test timeseries unsupported 7: don't support delete") {
-    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/timeseriestest.csv' INTO TABLE mainTable")
-    sql(
-      s"""CREATE DATAMAP agg1_minute ON TABLE mainTable
-         |USING '$timeSeries'
-         |DMPROPERTIES (
-         |   'event_time'='mytime',
-         |   'MINUTE_GRANULARITY'='1')
-         |AS SELECT mytime, SUM(age) FROM mainTable
-         |GROUP BY mytime
-        """.stripMargin)
-    checkAnswer(sql("SELECT * FROM maintable_agg1_minute"),
-      Seq(Row(Timestamp.valueOf("2016-02-23 09:01:00.0"), 60),
-        Row(Timestamp.valueOf("2016-02-23 09:02:00.0"), 140)))
-    val e = intercept[UnsupportedOperationException] {
-      sql("delete FROM maintable_agg1_minute")
-    }
-
-    assert(e.getMessage.equalsIgnoreCase(
-      "Delete operation is not supported for pre-aggregate table"))
-
-    // check value after inserting
-    checkAnswer(sql("SELECT * FROM maintable_agg1_minute"),
-      Seq(Row(Timestamp.valueOf("2016-02-23 09:01:00.0"), 60),
-        Row(Timestamp.valueOf("2016-02-23 09:02:00.0"), 140)))
-  }
-
-  test("test timeseries unsupported 8: don't support alter") {
-    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/timeseriestest.csv' INTO TABLE mainTable")
-    sql(
-      s"""CREATE DATAMAP agg1_minute ON TABLE mainTable
-         |USING '$timeSeries'
-         |DMPROPERTIES (
-         |   'event_time'='mytime',
-         |   'MINUTE_GRANULARITY'='1')
-         |AS SELECT mytime, SUM(age) FROM mainTable
-         |GROUP BY mytime
-        """.stripMargin)
-    // before alter
-    checkExistence(sql("SHOW DATAMAP ON TABLE mainTable"), true, "maintable_agg1_minute")
-
-    // alter
-    val e = intercept[Exception] {
-      sql("alter table maintable_agg1_minute rename to maintable_agg1_minute_new")
-    }
-    assert(e.getMessage.contains(
-      "Rename operation for datamaps is not supported."))
-
-    // check datamap after alter
-    checkExistence(sql("SHOW DATAMAP ON TABLE mainTable"), true, "maintable_agg1_minute")
-  }
-
-  override def afterAll: Unit = {
-    dropTable("mainTable")
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, timestampFormat)
-  }
-}
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/timeseries/TestTimeseriesCompaction.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/timeseries/TestTimeseriesCompaction.scala
deleted file mode 100644
index 2642b03..0000000
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/timeseries/TestTimeseriesCompaction.scala
+++ /dev/null
@@ -1,136 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.integration.spark.testsuite.timeseries
-
-import org.apache.spark.sql.test.util.QueryTest
-import org.apache.spark.util.SparkUtil4Test
-import org.scalatest.BeforeAndAfterAll
-import org.scalatest.Matchers._
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.metadata.schema.datamap.DataMapClassProvider.TIMESERIES
-import org.apache.carbondata.core.util.CarbonProperties
-
-class TestTimeseriesCompaction extends QueryTest with BeforeAndAfterAll {
-
-  var isCompactionEnabled = false
-  val timeSeries = TIMESERIES.toString
-
-  override def beforeAll: Unit = {
-    SparkUtil4Test.createTaskMockUp(sqlContext)
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
-        CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT)
-    isCompactionEnabled = CarbonProperties.getInstance()
-      .getProperty(CarbonCommonConstants.ENABLE_AUTO_LOAD_MERGE,"false").toBoolean
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.ENABLE_AUTO_LOAD_MERGE, "true")
-    sql("drop table if exists mainTable")
-    sql("CREATE TABLE mainTable(mytime timestamp, name string, age int) STORED BY 'org.apache.carbondata.format'")
-    sql(
-      s"""
-         | CREATE DATAMAP agg0_second ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         | 'EVENT_TIME'='mytime',
-         | 'SECOND_GRANULARITY'='1')
-         | AS SELECT mytime, SUM(age) FROM mainTable
-         | GROUP BY mytime
-       """.stripMargin)
-    sql(
-      s"""
-         | CREATE DATAMAP agg0_minute ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         | 'EVENT_TIME'='mytime',
-         | 'MINUTE_GRANULARITY'='1')
-         | AS SELECT mytime, SUM(age) FROM mainTable
-         | GROUP BY mytime
-       """.stripMargin)
-    sql(
-      s"""
-         | CREATE DATAMAP agg0_hour ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         | 'EVENT_TIME'='mytime',
-         | 'HOUR_GRANULARITY'='1')
-         | AS SELECT mytime, SUM(age) FROM mainTable
-         | GROUP BY mytime
-       """.stripMargin)
-    sql(
-      s"""
-         | CREATE DATAMAP agg0_day ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         | 'EVENT_TIME'='mytime',
-         | 'DAY_GRANULARITY'='1')
-         | AS SELECT mytime, SUM(age) FROM mainTable
-         | GROUP BY mytime
-       """.stripMargin)
-    sql(
-      s"""
-         | CREATE DATAMAP agg0_month ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         | 'EVENT_TIME'='mytime',
-         | 'MONTH_GRANULARITY'='1')
-         | AS SELECT mytime, SUM(age) FROM mainTable
-         | GROUP BY mytime
-       """.stripMargin)
-    sql(
-      s"""
-         | CREATE DATAMAP agg0_year ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         | 'EVENT_TIME'='mytime',
-         | 'YEAR_GRANULARITY'='1')
-         | AS SELECT mytime, SUM(age) FROM mainTable
-         | GROUP BY mytime
-       """.stripMargin)
-
-    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/timeseriestest.csv' into table mainTable")
-    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/timeseriestest.csv' into table mainTable")
-    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/timeseriestest.csv' into table mainTable")
-    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/timeseriestest.csv' into table mainTable")
-  }
-
-  test("test if pre-agg table is compacted with parent table minor compaction") {
-    val segmentNamesSecond = sql("show segments for table maintable_agg0_second").collect().map(_.get(0).toString)
-    segmentNamesSecond should equal (Array("3", "2", "1", "0.1", "0"))
-
-    val segmentNamesMinute = sql("show segments for table maintable_agg0_minute").collect().map(_.get(0).toString)
-    segmentNamesMinute should equal (Array("3", "2", "1", "0.1", "0"))
-
-    val segmentNamesHour = sql("show segments for table maintable_agg0_hour").collect().map(_.get(0).toString)
-    segmentNamesHour should equal (Array("3", "2", "1", "0.1", "0"))
-
-    val segmentNamesday = sql("show segments for table maintable_agg0_day").collect().map(_.get(0).toString)
-    segmentNamesday should equal (Array("3", "2", "1", "0.1", "0"))
-
-    val segmentNamesmonth = sql("show segments for table maintable_agg0_month").collect().map(_.get(0).toString)
-    segmentNamesmonth should equal (Array("3", "2", "1", "0.1", "0"))
-
-    val segmentNamesyear = sql("show segments for table maintable_agg0_year").collect().map(_.get(0).toString)
-    segmentNamesyear should equal (Array("3", "2", "1", "0.1", "0"))
-  }
-
-  override def afterAll: Unit = {
-    sql("drop table if exists mainTable")
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.ENABLE_AUTO_LOAD_MERGE, isCompactionEnabled+"")
-  }
-}
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/timeseries/TestTimeseriesDataLoad.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/timeseries/TestTimeseriesDataLoad.scala
deleted file mode 100644
index 244cc44..0000000
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/timeseries/TestTimeseriesDataLoad.scala
+++ /dev/null
@@ -1,597 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.integration.spark.testsuite.timeseries
-
-import java.sql.Timestamp
-
-import org.apache.spark.sql.Row
-import org.apache.spark.sql.test.util.QueryTest
-import org.apache.spark.util.SparkUtil4Test
-import org.scalatest.BeforeAndAfterAll
-
-import org.apache.carbondata.common.exceptions.sql.MalformedDataMapCommandException
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.metadata.schema.datamap.DataMapClassProvider.TIMESERIES
-import org.apache.carbondata.core.util.CarbonProperties
-
-class TestTimeseriesDataLoad extends QueryTest with BeforeAndAfterAll {
-
-  val timeSeries = TIMESERIES.toString
-  var timestampFormat: String = _
-
-  override def beforeAll: Unit = {
-    timestampFormat = CarbonProperties.getInstance()
-      .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
-        CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT)
-    SparkUtil4Test.createTaskMockUp(sqlContext)
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT)
-    sql("DROP TABLE IF EXISTS mainTable")
-    sql("DROP TABLE IF EXISTS table_03")
-    sql("CREATE TABLE mainTable(mytime timestamp, name string, age int) STORED BY 'org.apache.carbondata.format'")
-    sql(
-      s"""
-         | CREATE DATAMAP agg0_second ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         | 'EVENT_TIME'='mytime',
-         | 'SECOND_GRANULARITY'='1')
-         | AS SELECT mytime, SUM(age) FROM mainTable
-         | GROUP BY mytime
-       """.stripMargin)
-    sql(
-      s"""
-         | CREATE DATAMAP agg0_minute ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         | 'EVENT_TIME'='mytime',
-         | 'minute_granularity'='1')
-         | AS SELECT mytime, SUM(age) FROM mainTable
-         | GROUP BY mytime
-       """.stripMargin)
-    sql(
-      s"""
-         | CREATE DATAMAP agg0_hour ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         | 'EVENT_TIME'='mytime',
-         | 'HOUR_GRANULARITY'='1')
-         | AS SELECT mytime, SUM(age) FROM mainTable
-         | GROUP BY mytime
-       """.stripMargin)
-    sql(
-      s"""
-         | CREATE DATAMAP agg0_day ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         | 'EVENT_TIME'='mytime',
-         | 'DAY_GRANULARITY'='1')
-         | AS SELECT mytime, SUM(age) FROM mainTable
-         | GROUP BY mytime
-       """.stripMargin)
-    sql(
-      s"""
-         | CREATE DATAMAP agg0_month ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         | 'EVENT_TIME'='mytime',
-         | 'MONTH_GRANULARITY'='1')
-         | AS SELECT mytime, SUM(age) FROM mainTable
-         | GROUP BY mytime
-       """.stripMargin)
-    sql(
-      s"""
-         | CREATE DATAMAP agg0_year ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         | 'EVENT_TIME'='mytime',
-         | 'year_granularity'='1')
-         | AS SELECT mytime, SUM(age) FROM mainTable
-         | GROUP BY mytime
-       """.stripMargin)
-
-    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/timeseriestest.csv' into table mainTable")
-    sql("CREATE TABLE table_03 (imei string,age int,mac string,productdate timestamp,updatedate timestamp,gamePointId double,contractid double ) STORED BY 'org.apache.carbondata.format'")
-    sql(s"LOAD DATA inpath '$resourcesPath/data_sort.csv' INTO table table_03 options ('DELIMITER'=',', 'QUOTECHAR'='','FILEHEADER'='imei,age,mac,productdate,updatedate,gamePointId,contractid')")
-
-    sql(
-      s"""
-         | CREATE DATAMAP ag1_second ON TABLE table_03
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         |    'EVENT_TIME'='productdate',
-         |    'SECOND_GRANULARITY'='1')
-         | AS SELECT productdate,mac,SUM(age) FROM table_03
-         | GROUP BY productdate,mac
-       """.stripMargin)
-    sql(
-      s"""
-         | CREATE DATAMAP ag1_minute ON TABLE table_03
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         |    'EVENT_TIME'='productdate',
-         |    'minute_granularity'='1')
-         | AS SELECT productdate,mac,SUM(age) FROM table_03
-         | GROUP BY productdate,mac
-       """.stripMargin)
-    sql(
-      s"""
-         | CREATE DATAMAP ag1_hour ON TABLE table_03
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         |   'EVENT_TIME'='productdate',
-         |    'HOUR_GRANULARITY'='1')
-         | AS SELECT productdate,mac,SUM(age) FROM table_03
-         | GROUP BY productdate,mac
-       """.stripMargin)
-    sql(
-      s"""
-         | CREATE DATAMAP ag1_day ON TABLE table_03
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         |    'EVENT_TIME'='productdate',
-         |    'DAY_GRANULARITY'='1')
-         | AS SELECT productdate,mac,SUM(age) FROM table_03
-         | GROUP BY productdate,mac
-       """.stripMargin)
-    sql(
-      s"""
-         | CREATE DATAMAP ag1_month ON TABLE table_03
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         |    'EVENT_TIME'='productdate',
-         |    'month_granularity'='1')
-         | AS SELECT productdate,mac,SUM(age) FROM table_03
-         | GROUP BY productdate,mac
-       """.stripMargin)
-    sql(
-      s"""
-         | CREATE DATAMAP ag1_year ON TABLE table_03
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         |    'EVENT_TIME'='productdate',
-         |    'year_granularity'='1')
-         | AS SELECT productdate,mac,SUM(age) FROM table_03
-         | GROUP BY productdate,mac
-       """.stripMargin)
-
-  }
-
-  test("test timeseries table selection 1: year level timeseries data validation1 ") {
-    checkAnswer(sql("SELECT COUNT(*) FROM table_03_ag1_year"),
-      Seq(Row(4)))
-  }
-
-  test("test timeseries table selection 2: month level timeseries data validation1 ") {
-    checkAnswer(sql("SELECT COUNT(*) FROM table_03_ag1_month"),
-      Seq(Row(4)))
-  }
-
-  test("test timeseries table selection 3: day level timeseries data validation1 ") {
-    checkAnswer(sql("SELECT COUNT(*) FROM table_03_ag1_day"),
-      Seq(Row(12)))
-  }
-
-  test("test timeseries table selection 4: year level timeseries data validation") {
-    checkAnswer(sql("SELECT * FROM maintable_agg0_year"),
-      Seq(Row(Timestamp.valueOf("2016-01-01 00:00:00.0"), 200)))
-  }
-
-  test("test timeseries table selection 5: month level timeseries data validation") {
-    checkAnswer(sql("SELECT * FROM maintable_agg0_month"),
-      Seq(Row(Timestamp.valueOf("2016-02-01 00:00:00.0"), 200)))
-  }
-
-  test("test timeseries table selection 6: day level timeseries data validation") {
-    checkAnswer(sql("SELECT * FROM maintable_agg0_day"),
-      Seq(Row(Timestamp.valueOf("2016-02-23 00:00:00.0"), 200)))
-  }
-
-  test("test timeseries table selection 7: hour level timeseries data validation") {
-    checkAnswer(sql("SELECT * FROM maintable_agg0_hour"),
-      Seq(Row(Timestamp.valueOf("2016-02-23 09:00:00.0"), 200)))
-  }
-
-  test("test timeseries table selection 8: minute level timeseries data validation") {
-    checkAnswer(sql("SELECT * FROM maintable_agg0_minute"),
-      Seq(Row(Timestamp.valueOf("2016-02-23 09:01:00.0"), 60),
-        Row(Timestamp.valueOf("2016-02-23 09:02:00.0"), 140)))
-  }
-
-  test("test timeseries table selection 9: second level timeseries data validation") {
-    checkAnswer(sql("SELECT * FROM maintable_agg0_second"),
-      Seq(Row(Timestamp.valueOf("2016-02-23 09:01:30.0"), 10),
-        Row(Timestamp.valueOf("2016-02-23 09:01:40.0"), 20),
-        Row(Timestamp.valueOf("2016-02-23 09:01:50.0"), 30),
-        Row(Timestamp.valueOf("2016-02-23 09:02:30.0"), 40),
-        Row(Timestamp.valueOf("2016-02-23 09:02:40.0"), 50),
-        Row(Timestamp.valueOf("2016-02-23 09:02:50.0"), 50)))
-  }
-
-  test("test timeseries table selection 10: if timeseries load is successful ON TABLE creation") {
-    sql("DROP TABLE IF EXISTS mainTable")
-    sql("CREATE TABLE mainTable(mytime timestamp, name string, age int) STORED BY 'org.apache.carbondata.format'")
-    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/timeseriestest.csv' into table mainTable")
-    sql(
-      s"""
-         | CREATE DATAMAP agg0_second ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         | 'EVENT_TIME'='mytime',
-         | 'SECOND_GRANULARITY'='1')
-         | AS SELECT mytime, SUM(age) FROM mainTable
-         | GROUP BY mytime
-       """.stripMargin)
-    checkAnswer( sql("SELECT * FROM maintable_agg0_second"),
-      Seq(Row(Timestamp.valueOf("2016-02-23 09:01:30.0"),10),
-        Row(Timestamp.valueOf("2016-02-23 09:01:40.0"),20),
-        Row(Timestamp.valueOf("2016-02-23 09:01:50.0"),30),
-        Row(Timestamp.valueOf("2016-02-23 09:02:30.0"),40),
-        Row(Timestamp.valueOf("2016-02-23 09:02:40.0"),50),
-        Row(Timestamp.valueOf("2016-02-23 09:02:50.0"),50)))
-  }
-
-  test("test timeseries table selection 11: if timeseries load twice is successful ON TABLE creation") {
-    sql("DROP TABLE IF EXISTS mainTable")
-    sql(
-      """
-        | CREATE TABLE mainTable(
-        |   mytime TIMESTAMP,
-        |   name STRING,
-        |   age INT)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/timeseriestest.csv' INTO TABLE mainTable")
-    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/timeseriestest.csv' INTO TABLE mainTable")
-    sql(
-      s"""
-         | CREATE DATAMAP agg0_second ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         |   'event_time'='mytime',
-         |   'SECOND_GRANULARITY'='1')
-         | AS SELECT mytime, SUM(age)
-         | FROM mainTable
-         | GROUP BY mytime
-      """.stripMargin)
-    checkAnswer(sql("SELECT * FROM maintable_agg0_second"),
-      Seq(Row(Timestamp.valueOf("2016-02-23 09:01:30.0"), 20),
-        Row(Timestamp.valueOf("2016-02-23 09:01:40.0"), 40),
-        Row(Timestamp.valueOf("2016-02-23 09:01:50.0"), 60),
-        Row(Timestamp.valueOf("2016-02-23 09:02:30.0"), 80),
-        Row(Timestamp.valueOf("2016-02-23 09:02:40.0"), 100),
-        Row(Timestamp.valueOf("2016-02-23 09:02:50.0"), 100)))
-  }
-
-  test("test timeseries table selection 12: create datamap without 'if not exists' after load data into mainTable and create datamap") {
-    sql("DROP TABLE IF EXISTS mainTable")
-    sql(
-      """
-        | CREATE TABLE mainTable(
-        |   mytime timestamp,
-        |   name string,
-        |   age int)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/timeseriestest.csv' into table mainTable")
-
-    sql(
-      s"""
-         | CREATE DATAMAP agg0_second ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         |   'EVENT_TIME'='mytime',
-         |   'second_granularity'='1')
-         | AS SELECT mytime, SUM(age) FROM mainTable
-         | GROUP BY mytime
-        """.stripMargin)
-
-    checkAnswer(sql("select * from maintable_agg0_second"),
-      Seq(Row(Timestamp.valueOf("2016-02-23 09:01:30.0"), 10),
-        Row(Timestamp.valueOf("2016-02-23 09:01:40.0"), 20),
-        Row(Timestamp.valueOf("2016-02-23 09:01:50.0"), 30),
-        Row(Timestamp.valueOf("2016-02-23 09:02:30.0"), 40),
-        Row(Timestamp.valueOf("2016-02-23 09:02:40.0"), 50),
-        Row(Timestamp.valueOf("2016-02-23 09:02:50.0"), 50)))
-    val e: Exception = intercept[MalformedDataMapCommandException] {
-      sql(
-        s"""
-           | CREATE DATAMAP agg0_second ON TABLE mainTable
-           | USING '$timeSeries'
-           | DMPROPERTIES (
-           |   'EVENT_TIME'='mytime',
-           |   'second_granularity'='1')
-           | AS SELECT mytime, SUM(age) FROM mainTable
-           | GROUP BY mytime
-        """.stripMargin)
-    }
-    assert(e.getMessage.contains("DataMap name 'agg0_second' already exist"))
-    sql("DROP DATAMAP agg0_second ON TABLE mainTable")
-  }
-
-  test("test timeseries table selection 13: create datamap with 'if not exists' after load data into mainTable and create datamap") {
-    sql("drop table if exists mainTable")
-    sql(
-      """
-        | CREATE TABLE mainTable(
-        |   mytime timestamp,
-        |   name string,
-        |   age int)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/timeseriestest.csv' INTO TABLE mainTable")
-
-    sql(
-      s"""
-         | CREATE DATAMAP agg0_second ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         |   'EVENT_TIME'='mytime',
-         |   'second_granularity'='1')
-         | AS SELECT mytime, SUM(age) FROM mainTable
-         | GROUP BY mytime
-        """.stripMargin)
-
-    checkAnswer(sql("select * from maintable_agg0_second"),
-      Seq(Row(Timestamp.valueOf("2016-02-23 09:01:30.0"), 10),
-        Row(Timestamp.valueOf("2016-02-23 09:01:40.0"), 20),
-        Row(Timestamp.valueOf("2016-02-23 09:01:50.0"), 30),
-        Row(Timestamp.valueOf("2016-02-23 09:02:30.0"), 40),
-        Row(Timestamp.valueOf("2016-02-23 09:02:40.0"), 50),
-        Row(Timestamp.valueOf("2016-02-23 09:02:50.0"), 50)))
-
-    sql(
-      s"""
-         | CREATE DATAMAP IF NOT EXISTS  agg0_second ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         |   'EVENT_TIME'='mytime',
-         |   'second_granularity'='1')
-         | AS SELECT mytime, SUM(age) FROM mainTable
-         | GROUP BY mytime
-        """.stripMargin)
-
-    checkAnswer(sql("select * from maintable_agg0_second"),
-      Seq(Row(Timestamp.valueOf("2016-02-23 09:01:30.0"), 10),
-        Row(Timestamp.valueOf("2016-02-23 09:01:40.0"), 20),
-        Row(Timestamp.valueOf("2016-02-23 09:01:50.0"), 30),
-        Row(Timestamp.valueOf("2016-02-23 09:02:30.0"), 40),
-        Row(Timestamp.valueOf("2016-02-23 09:02:40.0"), 50),
-        Row(Timestamp.valueOf("2016-02-23 09:02:50.0"), 50)))
-  }
-
-  test("test timeseries table selection 14: load data into mainTable after create timeseries datamap ON TABLE and SELECT sub table") {
-    sql("DROP TABLE IF EXISTS main_table")
-    sql(
-      """
-        | CREATE TABLE main_table(
-        |   mytime timestamp,
-        |   name string,
-        |   age int)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/timeseriestest.csv' INTO TABLE main_table")
-    sql(
-      s"""
-         | CREATE DATAMAP agg0_second ON TABLE main_table
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         |   'event_time'='mytime',
-         |   'SECOND_GRANULARITY'='1')
-         | AS SELECT mytime, SUM(age)
-         | FROM main_table
-         | GROUP BY mytime""".stripMargin)
-
-
-    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/timeseriestest.csv' INTO TABLE main_table")
-
-    checkAnswer(sql("SELECT * FROM main_table_agg0_second"),
-      Seq(Row(Timestamp.valueOf("2016-02-23 09:01:30.0"), 10),
-        Row(Timestamp.valueOf("2016-02-23 09:01:40.0"), 20),
-        Row(Timestamp.valueOf("2016-02-23 09:01:50.0"), 30),
-        Row(Timestamp.valueOf("2016-02-23 09:02:30.0"), 40),
-        Row(Timestamp.valueOf("2016-02-23 09:02:40.0"), 50),
-        Row(Timestamp.valueOf("2016-02-23 09:02:50.0"), 50),
-        Row(Timestamp.valueOf("2016-02-23 09:01:30.0"), 10),
-        Row(Timestamp.valueOf("2016-02-23 09:01:40.0"), 20),
-        Row(Timestamp.valueOf("2016-02-23 09:01:50.0"), 30),
-        Row(Timestamp.valueOf("2016-02-23 09:02:30.0"), 40),
-        Row(Timestamp.valueOf("2016-02-23 09:02:40.0"), 50),
-        Row(Timestamp.valueOf("2016-02-23 09:02:50.0"), 50)))
-  }
-
-  test("test timeseries table selection 15: load data into main_table after create timeseries datamap ON TABLE 1") {
-    sql("DROP TABLE IF EXISTS main_table")
-    sql(
-      """
-        | CREATE TABLE main_table(
-        |   mytime timestamp,
-        |   name string,
-        |   age int)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    sql(s"LOAD DATA INPATH '$resourcesPath/timeseriestest.csv' INTO TABLE main_table")
-
-    sql(
-      s"""
-        | CREATE DATAMAP agg0_minute ON TABLE main_table
-        | USING '$timeSeries'
-        | DMPROPERTIES (
-        |   'event_time'='mytime',
-        |   'MINUTE_GRANULARITY'='1')
-        | AS SELECT mytime, SUM(age)
-        | FROM main_table
-        | GROUP BY mytime""".stripMargin)
-
-    sql(s"LOAD DATA INPATH '$resourcesPath/timeseriestest.csv' INTO TABLE main_table")
-    val df = sql(
-      """
-        | SELECT
-        |   timeseries(mytime,'minute') AS minuteLevel,
-        |   SUM(age) AS SUM
-        | FROM main_table
-        | WHERE timeseries(mytime,'minute')>='2016-02-23 09:01:00'
-        | GROUP BY
-        |   timeseries(mytime,'minute')
-        | ORDER BY
-        |   timeseries(mytime,'minute')
-      """.stripMargin)
-
-    checkAnswer(df,
-      Seq(Row(Timestamp.valueOf("2016-02-23 09:01:00"), 120),
-        Row(Timestamp.valueOf("2016-02-23 09:02:00"), 280)))
-  }
-
-  test("test timeseries table selection 16: load data into main_table after create timeseries datamap ON TABLE 2") {
-    sql("DROP TABLE IF EXISTS main_table")
-    sql(
-      """
-        | CREATE TABLE main_table(
-        |   mytime timestamp,
-        |   name string,
-        |   age int)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-
-    sql(s"LOAD DATA INPATH '$resourcesPath/timeseriestest.csv' INTO TABLE main_table")
-    sql(s"LOAD DATA INPATH '$resourcesPath/timeseriestest.csv' INTO TABLE main_table")
-    sql(
-      s"""
-        | CREATE DATAMAP agg0_minute ON TABLE main_table
-        | USING '$timeSeries'
-        | DMPROPERTIES (
-        |   'event_time'='mytime',
-        |   'MINUTE_GRANULARITY'='1')
-        | AS SELECT mytime, SUM(age)
-        | FROM main_table
-        | GROUP BY mytime""".stripMargin)
-
-
-    val df = sql(
-      """
-        | SELECT
-        |   timeseries(mytime,'minute') AS minuteLevel,
-        |   SUM(age) AS SUM
-        | FROM main_table
-        | WHERE timeseries(mytime,'minute')>='2016-02-23 09:01:00'
-        | GROUP BY
-        |   timeseries(mytime,'minute')
-        | ORDER BY
-        |   timeseries(mytime,'minute')
-      """.stripMargin)
-
-    checkAnswer(df,
-      Seq(Row(Timestamp.valueOf("2016-02-23 09:01:00"), 120),
-        Row(Timestamp.valueOf("2016-02-23 09:02:00"), 280)))
-  }
-
-  test("test timeseries table selection 17: load data into main_table after create timeseries datamap ON TABLE 3") {
-    sql("DROP TABLE IF EXISTS main_table")
-    sql(
-      """
-        | CREATE TABLE main_table(
-        |   mytime timestamp,
-        |   name string,
-        |   age int)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-
-    sql(s"LOAD DATA INPATH '$resourcesPath/timeseriestest.csv' INTO TABLE main_table")
-    sql(s"LOAD DATA INPATH '$resourcesPath/timeseriestest.csv' INTO TABLE main_table")
-    sql(s"LOAD DATA INPATH '$resourcesPath/timeseriestest.csv' INTO TABLE main_table")
-    sql(
-      s"""
-        | CREATE DATAMAP agg0_minute ON TABLE main_table
-        | USING '$timeSeries'
-        | DMPROPERTIES (
-        |   'event_time'='mytime',
-        |   'MINUTE_GRANULARITY'='1')
-        | AS SELECT mytime, SUM(age)
-        | FROM main_table
-        | GROUP BY mytime""".stripMargin)
-
-
-    val df = sql(
-      """
-        | SELECT
-        |   timeseries(mytime,'minute') AS minuteLevel,
-        |   SUM(age) AS SUM
-        | FROM main_table
-        | WHERE timeseries(mytime,'minute')>='2016-02-23 09:01:00'
-        | GROUP BY
-        |   timeseries(mytime,'minute')
-        | ORDER BY
-        |   timeseries(mytime,'minute')
-      """.stripMargin)
-
-    checkAnswer(df,
-      Seq(Row(Timestamp.valueOf("2016-02-23 09:01:00"), 180),
-        Row(Timestamp.valueOf("2016-02-23 09:02:00"), 420)))
-  }
-
-  test("test timeseries table selection 18: load data into main_table after create timeseries datamap ON TABLE 4") {
-    sql("DROP TABLE IF EXISTS main_table")
-    sql(
-      """
-        | CREATE TABLE main_table(
-        |   mytime timestamp,
-        |   name string,
-        |   age int)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-
-    sql(s"LOAD DATA INPATH '$resourcesPath/timeseriestest.csv' INTO TABLE main_table")
-
-    sql(
-      s"""
-        | CREATE DATAMAP agg0_minute ON TABLE main_table
-        | USING '$timeSeries'
-        | DMPROPERTIES (
-        |   'event_time'='mytime',
-        |   'MINUTE_GRANULARITY'='1')
-        | AS SELECT mytime, SUM(age)
-        | FROM main_table
-        | GROUP BY mytime""".stripMargin)
-
-    sql(s"LOAD DATA INPATH '$resourcesPath/timeseriestest.csv' INTO TABLE main_table")
-    sql(s"LOAD DATA INPATH '$resourcesPath/timeseriestest.csv' INTO TABLE main_table")
-    val df = sql(
-      """
-        | SELECT
-        |   timeseries(mytime,'minute') AS minuteLevel,
-        |   SUM(age) AS SUM
-        | FROM main_table
-        | WHERE timeseries(mytime,'minute')>='2016-02-23 09:01:00'
-        | GROUP BY
-        |   timeseries(mytime,'minute')
-        | ORDER BY
-        |   timeseries(mytime,'minute')
-      """.stripMargin)
-
-    checkAnswer(df,
-      Seq(Row(Timestamp.valueOf("2016-02-23 09:01:00"), 180),
-        Row(Timestamp.valueOf("2016-02-23 09:02:00"), 420)))
-  }
-
-  override def afterAll: Unit = {
-    sql("DROP TABLE IF EXISTS main_table")
-    sql("DROP TABLE IF EXISTS mainTable")
-    sql("DROP TABLE IF EXISTS table_03")
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, timestampFormat)
-  }
-}
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/timeseries/TestTimeseriesTableSelection.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/timeseries/TestTimeseriesTableSelection.scala
deleted file mode 100644
index 7083c54..0000000
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/timeseries/TestTimeseriesTableSelection.scala
+++ /dev/null
@@ -1,882 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.integration.spark.testsuite.timeseries
-
-import java.sql.Timestamp
-
-import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
-import org.apache.spark.sql.execution.datasources.LogicalRelation
-import org.apache.spark.sql.hive.CarbonRelation
-import org.apache.spark.sql.test.util.QueryTest
-import org.apache.spark.sql.{CarbonDatasourceHadoopRelation, Row}
-import org.apache.spark.util.SparkUtil4Test
-import org.scalatest.BeforeAndAfterAll
-
-import org.apache.carbondata.core.metadata.schema.datamap.DataMapClassProvider.TIMESERIES
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.util.CarbonProperties
-
-class TestTimeseriesTableSelection extends QueryTest with BeforeAndAfterAll {
-
-  val timeSeries = TIMESERIES.toString
-  var timestampFormat: String = _
-
-  override def beforeAll: Unit = {
-    timestampFormat = CarbonProperties.getInstance()
-      .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
-        CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT)
-    SparkUtil4Test.createTaskMockUp(sqlContext)
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
-        CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT)
-    sql("drop table if exists mainTable")
-    sql(
-      """
-        | CREATE TABLE mainTable(mytime timestamp, name string, age int)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    sql(
-      s"""
-         | CREATE DATAMAP agg0_second ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         | 'EVENT_TIME'='mytime',
-         | 'SECOND_GRANULARITY'='1')
-         | AS SELECT mytime, SUM(age) FROM mainTable
-         | GROUP BY mytime
-       """.stripMargin)
-    sql(
-      s"""
-         | CREATE DATAMAP agg0_minute ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         | 'EVENT_TIME'='mytime',
-         | 'minute_granularity'='1')
-         | AS SELECT mytime, SUM(age) FROM mainTable
-         | GROUP BY mytime
-       """.stripMargin)
-    sql(
-      s"""
-         | CREATE DATAMAP agg0_hour ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         | 'EVENT_TIME'='mytime',
-         | 'HOUR_GRANULARITY'='1')
-         | AS SELECT mytime, SUM(age) FROM mainTable
-         | GROUP BY mytime
-       """.stripMargin)
-    sql(
-      s"""
-         | CREATE DATAMAP agg0_day ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         | 'EVENT_TIME'='mytime',
-         | 'DAY_GRANULARITY'='1')
-         | AS SELECT mytime, SUM(age) FROM mainTable
-         | GROUP BY mytime
-       """.stripMargin)
-    sql(
-      s"""
-         | CREATE DATAMAP agg0_month ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         | 'EVENT_TIME'='mytime',
-         | 'MONTH_GRANULARITY'='1')
-         | AS SELECT mytime, SUM(age) FROM mainTable
-         | GROUP BY mytime
-       """.stripMargin)
-    sql(
-      s"""
-         | CREATE DATAMAP agg0_year ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         | 'EVENT_TIME'='mytime',
-         | 'YEAR_GRANULARITY'='1')
-         | AS SELECT mytime, SUM(age) FROM mainTable
-         | GROUP BY mytime
-       """.stripMargin)
-
-    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/timeseriestest.csv' into table mainTable")
-  }
-
-  test("test timeseries table selection 1") {
-    val df = sql("SELECT mytime FROM mainTable GROUP BY mytime")
-    preAggTableValidator(df.queryExecution.analyzed, "maintable")
-  }
-
-  test("test timeseries table selection 2") {
-    val df = sql("SELECT TIMESERIES(mytime,'hour') FROM mainTable " +
-                 "GROUP BY TIMESERIES(mytime,'hour')")
-    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg0_hour")
-  }
-
-  test("test timeseries table selection 3: No enum constant MILLI") {
-    val e = intercept[Exception] {
-      val df = sql(
-        """
-          | SELECT TIMESERIES(mytime,'milli')
-          | FROM mainTable
-          | GROUP BY TIMESERIES(mytime,'milli')
-        """.stripMargin)
-      preAggTableValidator(df.queryExecution.analyzed, "maintable")
-      df.show()
-    }
-    assert(e.getMessage.contains(
-      "No enum constant org.apache.carbondata.core.preagg.TimeSeriesFunctionEnum.MILLI"))
-  }
-
-  test("test timeseries table selection 4") {
-    val df = sql("SELECT TIMESERIES(mytime,'year') FROM mainTable GROUP BY TIMESERIES(mytime,'year')")
-    preAggTableValidator(df.queryExecution.analyzed,"maintable_agg0_year")
-  }
-
-  test("test timeseries table selection 5") {
-    val df = sql("SELECT TIMESERIES(mytime,'day') FROM mainTable GROUP BY TIMESERIES(mytime,'day')")
-    preAggTableValidator(df.queryExecution.analyzed,"maintable_agg0_day")
-  }
-
-  test("test timeseries table selection 6") {
-    val df = sql("SELECT TIMESERIES(mytime,'month') FROM mainTable GROUP BY TIMESERIES(mytime,'month')")
-    preAggTableValidator(df.queryExecution.analyzed,"maintable_agg0_month")
-  }
-
-  test("test timeseries table selection 7") {
-    val df = sql("SELECT TIMESERIES(mytime,'minute') FROM mainTable GROUP BY TIMESERIES(mytime,'minute')")
-    preAggTableValidator(df.queryExecution.analyzed,"maintable_agg0_minute")
-  }
-
-  test("test timeseries table selection 8") {
-    val df = sql("SELECT TIMESERIES(mytime,'second') FROM mainTable GROUP BY TIMESERIES(mytime,'second')")
-    preAggTableValidator(df.queryExecution.analyzed,"maintable_agg0_second")
-  }
-
-  test("test timeseries table selection 9") {
-    val df = sql(
-      """
-        | SELECT TIMESERIES(mytime,'hour')
-        | FROM mainTable
-        | WHERE TIMESERIES(mytime,'hour')='x'
-        | GROUP BY TIMESERIES(mytime,'hour')
-      """.stripMargin)
-    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg0_hour")
-  }
-
-  test("test timeseries table selection 10") {
-    val df = sql(
-      """
-        | SELECT TIMESERIES(mytime,'hour')
-        | FROM mainTable
-        | WHERE TIMESERIES(mytime,'hour')='x'
-        | GROUP BY TIMESERIES(mytime,'hour')
-        | ORDER BY TIMESERIES(mytime,'hour')
-      """.stripMargin)
-    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg0_hour")
-  }
-
-  test("test timeseries table selection 11") {
-    val df = sql(
-      """
-        | SELECT TIMESERIES(mytime,'hour'),SUM(age)
-        | FROM mainTable
-        | WHERE TIMESERIES(mytime,'hour')='x'
-        | GROUP BY TIMESERIES(mytime,'hour')
-        | ORDER BY TIMESERIES(mytime,'hour')
-      """.stripMargin)
-    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg0_hour")
-  }
-
-  test("test timeseries table selection 12") {
-    val df = sql(
-      """
-        | SELECT TIMESERIES(mytime,'hour') AS hourlevel,SUM(age) AS SUM
-        | FROM mainTable
-        | WHERE TIMESERIES(mytime,'hour')='x'
-        | GROUP BY TIMESERIES(mytime,'hour')
-        | ORDER BY TIMESERIES(mytime,'hour')
-      """.stripMargin)
-    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg0_hour")
-  }
-
-  test("test timeseries table selection 13") {
-    val df = sql(
-      """
-        | SELECT TIMESERIES(mytime,'hour')as hourlevel,SUM(age) AS SUM
-        | FROM mainTable
-        | WHERE TIMESERIES(mytime,'hour')='x' AND name='vishal'
-        | GROUP BY TIMESERIES(mytime,'hour')
-        | ORDER BY TIMESERIES(mytime,'hour')
-      """.stripMargin)
-    preAggTableValidator(df.queryExecution.analyzed, "maintable")
-  }
-
-  test("test timeseries table selection 14: TIMESERIES(mytime,'hour') match") {
-    val df = sql(
-      """
-        | SELECT TIMESERIES(mytime,'hour')
-        | FROM mainTable
-        | WHERE TIMESERIES(mytime,'hour')='2016-02-23 09:00:00'
-        | GROUP BY TIMESERIES(mytime,'hour')
-      """.stripMargin)
-    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg0_hour")
-    checkAnswer(df, Row(Timestamp.valueOf("2016-02-23 09:00:00.0")))
-  }
-
-  test("test timeseries table selection 15: TIMESERIES(mytime,'hour') not match") {
-    val df = sql(
-      """
-        | SELECT TIMESERIES(mytime,'hour')
-        | FROM mainTable
-        | WHERE TIMESERIES(mytime,'hour')='2016-02-23 09:01:00'
-        | GROUP BY TIMESERIES(mytime,'hour')
-      """.stripMargin)
-    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg0_hour")
-    checkExistence(df, false, "2016-02-23 09:00:00", "2016-02-23 09:01:00")
-  }
-
-  test("test timeseries table selection 16: TIMESERIES(mytime,'minute') match") {
-    checkExistence(sql("SELECT * FROM mainTable"), true,
-      "2016-02-23 09:01:30", "2016-02-23 09:02:40")
-    checkExistence(sql("SELECT * FROM mainTable"), false,
-      "2016-02-23 09:02:00", "2016-02-23 09:01:00")
-    val df = sql(
-      """
-        |SELECT TIMESERIES(mytime,'minute') 
-        |FROM mainTable 
-        |GROUP BY TIMESERIES(mytime,'minute')
-      """.stripMargin)
-    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg0_minute")
-    checkExistence(df, true, "2016-02-23 09:02:00", "2016-02-23 09:01:00")
-    checkAnswer(df,
-      Seq(Row(Timestamp.valueOf("2016-02-23 09:02:00.0")),
-        Row(Timestamp.valueOf("2016-02-23 09:01:00.0"))))
-
-    val df2 = sql(
-      """
-        | SELECT
-        |   TIMESERIES(mytime,'minute')as minutelevel,
-        |   SUM(age) AS SUM
-        | FROM mainTable
-        | WHERE TIMESERIES(mytime,'minute')='2016-02-23 09:01:00'
-        | GROUP BY TIMESERIES(mytime,'minute')
-        | ORDER BY TIMESERIES(mytime,'minute')
-      """.stripMargin)
-    preAggTableValidator(df2.queryExecution.analyzed, "maintable_agg0_minute")
-    checkAnswer(df2, Seq(Row(Timestamp.valueOf("2016-02-23 09:01:00"), 60)))
-  }
-
-  test("test timeseries table selection 17: TIMESERIES(mytime,'minute') not match pre agg") {
-    val df = sql(
-      """
-        | SELECT
-        |   TIMESERIES(mytime,'minute')as minutelevel,
-        |   SUM(age) AS SUM
-        | FROM mainTable
-        | WHERE TIMESERIES(mytime,'minute')='2016-02-23 09:01:00' AND name='vishal'
-        | GROUP BY TIMESERIES(mytime,'minute')
-        | ORDER BY TIMESERIES(mytime,'minute')
-      """.stripMargin)
-    checkAnswer(df, Seq(Row(Timestamp.valueOf("2016-02-23 09:01:00"), 10)))
-    preAggTableValidator(df.queryExecution.analyzed, "maintable")
-  }
-
-  test("test timeseries table selection 18: select with many GROUP BY AND one filter") {
-    val df = sql(
-      """
-        | SELECT
-        |   TIMESERIES(mytime,'year') AS yearLevel,
-        |   TIMESERIES(mytime,'month') AS monthLevel,
-        |   TIMESERIES(mytime,'day') AS dayLevel,
-        |   TIMESERIES(mytime,'hour') AS hourLevel,
-        |   TIMESERIES(mytime,'minute') AS minuteLevel,
-        |   TIMESERIES(mytime,'second') AS secondLevel,
-        |   SUM(age) AS SUM
-        | FROM mainTable
-        | WHERE TIMESERIES(mytime,'minute')='2016-02-23 09:01:00'
-        | GROUP BY
-        |   TIMESERIES(mytime,'year'),
-        |   TIMESERIES(mytime,'month'),
-        |   TIMESERIES(mytime,'day'),
-        |   TIMESERIES(mytime,'hour'),
-        |   TIMESERIES(mytime,'minute'),
-        |   TIMESERIES(mytime,'second')
-        | ORDER BY
-        |   TIMESERIES(mytime,'year'),
-        |   TIMESERIES(mytime,'month'),
-        |   TIMESERIES(mytime,'day'),
-        |   TIMESERIES(mytime,'hour'),
-        |   TIMESERIES(mytime,'minute'),
-        |   TIMESERIES(mytime,'second')
-      """.stripMargin)
-
-    checkExistence(df, true,
-      "2016-01-01 00:00:00",
-      "2016-02-01 00:00:00",
-      "2016-02-23 09:00:00",
-      "2016-02-23 09:01:00",
-      "2016-02-23 09:01:50",
-      "30"
-    )
-  }
-
-  test("test timeseries table selection 19: select with many GROUP BY AND many filter") {
-    val df = sql(
-      """
-        | SELECT
-        |   TIMESERIES(mytime,'year') AS yearLevel,
-        |   TIMESERIES(mytime,'month') AS monthLevel,
-        |   TIMESERIES(mytime,'day') AS dayLevel,
-        |   TIMESERIES(mytime,'hour') AS hourLevel,
-        |   TIMESERIES(mytime,'minute') AS minuteLevel,
-        |   TIMESERIES(mytime,'second') AS secondLevel,
-        |   SUM(age) AS SUM
-        | FROM mainTable
-        | WHERE
-        |   TIMESERIES(mytime,'second')='2016-02-23 09:01:50' and
-        |   TIMESERIES(mytime,'minute')='2016-02-23 09:01:00' and
-        |   TIMESERIES(mytime,'hour')='2016-02-23 09:00:00' and
-        |   TIMESERIES(mytime,'month')='2016-02-01 00:00:00' and
-        |   TIMESERIES(mytime,'year')='2016-01-01 00:00:00'
-        | GROUP BY
-        |   TIMESERIES(mytime,'year'),
-        |   TIMESERIES(mytime,'month'),
-        |   TIMESERIES(mytime,'day'),
-        |   TIMESERIES(mytime,'hour'),
-        |   TIMESERIES(mytime,'minute'),
-        |   TIMESERIES(mytime,'second')
-        | ORDER BY
-        |   TIMESERIES(mytime,'year'),
-        |   TIMESERIES(mytime,'month'),
-        |   TIMESERIES(mytime,'day'),
-        |   TIMESERIES(mytime,'hour'),
-        |   TIMESERIES(mytime,'minute'),
-        |   TIMESERIES(mytime,'second')
-      """.stripMargin)
-
-    checkExistence(df, true,
-      "2016-01-01 00:00:00",
-      "2016-02-01 00:00:00",
-      "2016-02-23 09:00:00",
-      "2016-02-23 09:01:00",
-      "2016-02-23 09:01:50",
-      "30"
-    )
-  }
-
-  test("test timeseries table selection 21: filter < AND >") {
-    val df = sql(
-      """
-        | SELECT
-        |   TIMESERIES(mytime,'minute') AS minuteLevel,
-        |   SUM(age) AS SUM
-        | FROM mainTable
-        | WHERE TIMESERIES(mytime,'minute')<'2016-02-23 09:04:00'
-        |   AND TIMESERIES(mytime,'minute')>'2016-02-23 09:01:00'
-        | GROUP BY
-        |   TIMESERIES(mytime,'minute')
-        | ORDER BY
-        |   TIMESERIES(mytime,'minute')
-      """.stripMargin)
-
-    checkAnswer(df, Seq(Row(Timestamp.valueOf("2016-02-23 09:02:00"), 140)))
-
-    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg0_minute")
-  }
-
-  test("test timeseries table selection 22: filter <= AND >=") {
-    val df = sql(
-      """
-        | SELECT
-        |   TIMESERIES(mytime,'minute') AS minuteLevel,
-        |   SUM(age) AS SUM
-        | FROM mainTable
-        | WHERE TIMESERIES(mytime,'minute')<='2016-02-23 09:02:00'
-        |   AND TIMESERIES(mytime,'minute')>='2016-02-23 09:01:00'
-        | GROUP BY
-        |   TIMESERIES(mytime,'minute')
-        | ORDER BY
-        |   TIMESERIES(mytime,'minute')
-      """.stripMargin)
-
-    checkAnswer(df,
-      Seq(Row(Timestamp.valueOf("2016-02-23 09:01:00"), 60),
-        Row(Timestamp.valueOf("2016-02-23 09:02:00"), 140)))
-
-    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg0_minute")
-  }
-
-  test("test timeseries table selection 23: filter < AND >=") {
-    val df = sql(
-      """
-        | SELECT
-        |   TIMESERIES(mytime,'minute') AS minuteLevel,
-        |   SUM(age) AS SUM
-        | FROM mainTable
-        | WHERE TIMESERIES(mytime,'minute')<'2016-02-23 09:02:00'
-        |   AND TIMESERIES(mytime,'minute')>='2016-02-23 09:01:00'
-        | GROUP BY
-        |   TIMESERIES(mytime,'minute')
-        | ORDER BY
-        |   TIMESERIES(mytime,'minute')
-      """.stripMargin)
-
-    checkAnswer(df, Seq(Row(Timestamp.valueOf("2016-02-23 09:01:00"), 60)))
-
-    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg0_minute")
-  }
-
-  test("test timeseries table selection 24: filter < AND >=") {
-    val df = sql(
-      """
-        | SELECT
-        |   TIMESERIES(mytime,'minute') AS minuteLevel,
-        |   SUM(age) AS SUM
-        | FROM mainTable
-        | WHERE TIMESERIES(mytime,'minute')<'2016-02-23 09:01:00'
-        |   AND TIMESERIES(mytime,'minute')>='2016-02-23 09:01:00'
-        | GROUP BY
-        |   TIMESERIES(mytime,'minute')
-        | ORDER BY
-        |   TIMESERIES(mytime,'minute')
-      """.stripMargin)
-
-    checkAnswer(df, Seq.empty)
-
-    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg0_minute")
-  }
-
-  test("test timeseries table selection 25: filter many column") {
-    val df = sql(
-      """
-        | SELECT
-        |   TIMESERIES(mytime,'minute') AS minuteLevel,
-        |   SUM(age) AS SUM
-        | FROM mainTable
-        | WHERE
-        |   TIMESERIES(mytime,'minute')<'2016-02-23 09:02:00' and
-        |   TIMESERIES(mytime,'hour')>='2016-02-23 09:00:00' and
-        |   name='vishal'
-        | GROUP BY
-        |   TIMESERIES(mytime,'minute')
-        | ORDER BY
-        |   TIMESERIES(mytime,'minute')
-      """.stripMargin)
-
-    checkAnswer(df, Seq(Row(Timestamp.valueOf("2016-02-23 09:01:00"), 10)))
-  }
-
-  test("test timeseries table selection 26: filter < AND >=, avg") {
-    val df = sql(
-      """
-        | SELECT
-        |   TIMESERIES(mytime,'minute') AS minuteLevel,
-        |   avg(age) AS avg
-        | FROM mainTable
-        | WHERE TIMESERIES(mytime,'minute')<'2016-02-23 09:02:00'
-        |   AND TIMESERIES(mytime,'minute')>='2016-02-23 09:01:00'
-        | GROUP BY
-        |   TIMESERIES(mytime,'minute')
-        | ORDER BY
-        |   TIMESERIES(mytime,'minute')
-      """.stripMargin)
-
-    checkAnswer(df, Seq(Row(Timestamp.valueOf("2016-02-23 09:01:00"), 20.0)))
-
-  }
-
-  test("test timeseries table selection 27: filter < AND >=, max") {
-    val df = sql(
-      """
-        | SELECT
-        |   TIMESERIES(mytime,'second') AS secondLevel,
-        |   max(age) AS maxValue
-        | FROM mainTable
-        | WHERE TIMESERIES(mytime,'second')<'2016-02-23 09:02:00'
-        |   AND TIMESERIES(mytime,'second')>='2016-02-23 09:01:00'
-        | GROUP BY
-        |   TIMESERIES(mytime,'second')
-        | ORDER BY
-        |   TIMESERIES(mytime,'second')
-      """.stripMargin)
-
-    checkAnswer(df, Seq(Row(Timestamp.valueOf("2016-02-23 09:01:30"), 10),
-      Row(Timestamp.valueOf("2016-02-23 09:01:40"), 20),
-      Row(Timestamp.valueOf("2016-02-23 09:01:50"), 30)))
-  }
-
-  test("test timeseries table selection 28: filter < AND >=, min") {
-    val df = sql(
-      """
-        | SELECT
-        |   TIMESERIES(mytime,'second') AS secondLevel,
-        |   min(age) AS minValue
-        | FROM mainTable
-        | WHERE TIMESERIES(mytime,'second')<'2016-02-23 09:02:00'
-        |   AND TIMESERIES(mytime,'second')>='2016-02-23 09:01:00'
-        | GROUP BY
-        |   TIMESERIES(mytime,'second')
-        | ORDER BY
-        |   TIMESERIES(mytime,'second')
-      """.stripMargin)
-
-    checkAnswer(df, Seq(Row(Timestamp.valueOf("2016-02-23 09:01:30"), 10),
-      Row(Timestamp.valueOf("2016-02-23 09:01:40"), 20),
-      Row(Timestamp.valueOf("2016-02-23 09:01:50"), 30)))
-  }
-
-  test("test timeseries table selection 29: count, max, min, sum") {
-    dropDataMaps("maintable", "agg1_second", "agg1_minute")
-    sql(
-      s"""
-         | CREATE DATAMAP agg1_minute ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         |   'event_time'='mytime',
-         |   'MINUTE_GRANULARITY'='1')
-         | AS SELECT mytime, SUM(age), count(age), max(age), min(age)
-         | FROM mainTable
-         | GROUP BY mytime
-      """.stripMargin)
-
-    val df = sql(
-      """
-        | SELECT
-        |   TIMESERIES(mytime,'minute') AS minuteLevel,
-        |   count(age) AS count
-        | FROM mainTable
-        | WHERE TIMESERIES(mytime,'minute')<'2016-02-23 09:02:00'
-        |   AND TIMESERIES(mytime,'minute')>='2016-02-23 09:01:00'
-        | GROUP BY
-        |   TIMESERIES(mytime,'minute')
-        | ORDER BY
-        |   TIMESERIES(mytime,'minute')
-      """.stripMargin)
-
-    checkAnswer(df, Seq(Row(Timestamp.valueOf("2016-02-23 09:01:00"), 3)))
-
-    preAggTableValidator(df.queryExecution.analyzed, "maintable_agg1_minute")
-
-    val df1 = sql(
-      """
-        | SELECT
-        |   TIMESERIES(mytime,'minute') AS minuteLevel,
-        |   max(age) AS maxValue
-        | FROM mainTable
-        | WHERE TIMESERIES(mytime,'minute')<'2016-02-23 09:02:00'
-        |   AND TIMESERIES(mytime,'minute')>='2016-02-23 09:01:00'
-        | GROUP BY
-        |   TIMESERIES(mytime,'minute')
-        | ORDER BY
-        |   TIMESERIES(mytime,'minute')
-      """.stripMargin)
-
-    checkAnswer(df1, Seq(Row(Timestamp.valueOf("2016-02-23 09:01:00"), 30)))
-
-    preAggTableValidator(df1.queryExecution.analyzed, "maintable_agg1_minute")
-
-    val df2 = sql(
-      """
-        | SELECT
-        |   TIMESERIES(mytime,'minute') AS minuteLevel,
-        |   min(age) AS minValue
-        | FROM mainTable
-        | WHERE TIMESERIES(mytime,'minute')<'2016-02-23 09:02:00'
-        |   AND TIMESERIES(mytime,'minute')>='2016-02-23 09:01:00'
-        | GROUP BY
-        |   TIMESERIES(mytime,'minute')
-        | ORDER BY
-        |   TIMESERIES(mytime,'minute')
-      """.stripMargin)
-
-    checkAnswer(df2, Seq(Row(Timestamp.valueOf("2016-02-23 09:01:00"), 10)))
-
-    preAggTableValidator(df2.queryExecution.analyzed, "maintable_agg1_minute")
-
-    dropDataMaps("maintable", "agg1_second", "agg1_minute")
-
-    val df4 = sql(
-      """
-        | SELECT
-        |   TIMESERIES(mytime,'minute') AS minuteLevel,
-        |   SUM(age) AS SUM
-        | FROM mainTable
-        | WHERE TIMESERIES(mytime,'minute')<'2016-02-23 09:02:00'
-        |   AND TIMESERIES(mytime,'minute')>='2016-02-23 09:01:00'
-        | GROUP BY
-        |   TIMESERIES(mytime,'minute')
-        | ORDER BY
-        |   TIMESERIES(mytime,'minute')
-      """.stripMargin)
-
-    checkAnswer(df4, Seq(Row(Timestamp.valueOf("2016-02-23 09:01:00"), 60)))
-  }
-
-  test("test timeseries table selection 30: max, no create") {
-    val df1 = sql(
-      """
-        | SELECT
-        |   TIMESERIES(mytime,'minute') AS minuteLevel,
-        |   max(age) AS maxValue
-        | FROM mainTable
-        | WHERE TIMESERIES(mytime,'minute')<'2016-02-23 09:02:00'
-        |   AND TIMESERIES(mytime,'minute')>='2016-02-23 09:01:00'
-        | GROUP BY
-        |   TIMESERIES(mytime,'minute')
-        | ORDER BY
-        |   TIMESERIES(mytime,'minute')
-      """.stripMargin)
-
-    checkAnswer(df1, Seq(Row(Timestamp.valueOf("2016-02-23 09:01:00"), 30)))
-  }
-
-  test("test timeseries table selection 31: min, no create") {
-    val df1 = sql(
-      """
-        | SELECT
-        |   TIMESERIES(mytime,'minute') AS minuteLevel,
-        |   min(age) AS minValue
-        | FROM mainTable
-        | WHERE TIMESERIES(mytime,'minute')<'2016-02-23 09:02:00'
-        |   AND TIMESERIES(mytime,'minute')>='2016-02-23 09:01:00'
-        | GROUP BY
-        |   TIMESERIES(mytime,'minute')
-        | ORDER BY
-        |   TIMESERIES(mytime,'minute')
-      """.stripMargin)
-
-    checkAnswer(df1, Seq(Row(Timestamp.valueOf("2016-02-23 09:01:00"), 10)))
-  }
-
-  test("test timeseries table selection 32: filter < AND >=, min") {
-    val df = sql(
-      """
-        | SELECT
-        |   TIMESERIES(mytime,'second') AS secondLevel,
-        |   min(age) AS minValue
-        | FROM mainTable
-        | WHERE TIMESERIES(mytime,'second')<'2016-02-23 09:02:00'
-        |   AND TIMESERIES(mytime,'second')>='2016-02-23 09:01:00'
-        | GROUP BY
-        |   TIMESERIES(mytime,'second')
-        | ORDER BY
-        |   TIMESERIES(mytime,'second')
-      """.stripMargin)
-
-    checkAnswer(df, Seq(Row(Timestamp.valueOf("2016-02-23 09:01:30"), 10),
-      Row(Timestamp.valueOf("2016-02-23 09:01:40"), 20),
-      Row(Timestamp.valueOf("2016-02-23 09:01:50"), 30)))
-  }
-
-  test("test timeseries table selection 33: max") {
-    dropDataMaps("maintable", "agg1_minute")
-    sql(
-      s"""
-         | CREATE DATAMAP agg1_minute ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         |   'event_time'='mytime',
-         |   'MINUTE_GRANULARITY'='1')
-         | AS SELECT mytime, SUM(age), count(age), max(age), min(age)
-         | FROM mainTable
-         | GROUP BY mytime
-      """.stripMargin)
-
-    val df1 = sql(
-      """
-        | SELECT
-        |   TIMESERIES(mytime,'minute') AS minuteLevel,
-        |   max(age) AS maxValue
-        | FROM mainTable
-        | WHERE TIMESERIES(mytime,'minute')<'2016-02-23 09:02:00'
-        |   AND TIMESERIES(mytime,'minute')>='2016-02-23 09:01:00'
-        | GROUP BY
-        |   TIMESERIES(mytime,'minute')
-        | ORDER BY
-        |   TIMESERIES(mytime,'minute')
-      """.stripMargin)
-
-    checkAnswer(df1, Seq(Row(Timestamp.valueOf("2016-02-23 09:01:00"), 30)))
-
-    preAggTableValidator(df1.queryExecution.analyzed, "maintable_agg1_minute")
-  }
-
-  test("test timeseries table selection 34: min") {
-    dropDataMaps("maintable", "agg1_second", "agg1_minute")
-    sql(
-      s"""
-         | CREATE DATAMAP agg1_minute ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         |   'event_time'='mytime',
-         |   'MINUTE_GRANULARITY'='1')
-         | AS SELECT mytime, SUM(age), count(age), max(age), min(age)
-         | FROM mainTable
-         | GROUP BY mytime
-      """.stripMargin)
-
-    val df1 = sql(
-      """
-        | SELECT
-        |   TIMESERIES(mytime,'minute') AS minuteLevel,
-        |   min(age) AS minValue
-        | FROM mainTable
-        | WHERE TIMESERIES(mytime,'minute')<'2016-02-23 09:02:00'
-        |   AND TIMESERIES(mytime,'minute')>='2016-02-23 09:01:00'
-        | GROUP BY
-        |   TIMESERIES(mytime,'minute')
-        | ORDER BY
-        |   TIMESERIES(mytime,'minute')
-      """.stripMargin)
-
-    checkAnswer(df1, Seq(Row(Timestamp.valueOf("2016-02-23 09:01:00"), 10)))
-
-    preAggTableValidator(df1.queryExecution.analyzed, "maintable_agg1_minute")
-  }
-
-  test("test timeseries table selection 35: sum") {
-    dropDataMaps("maintable", "agg1_second", "agg1_minute")
-    sql(
-      s"""
-         | CREATE DATAMAP agg1_minute ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         |   'event_time'='mytime',
-         |   'MINUTE_GRANULARITY'='1')
-         | AS SELECT mytime, SUM(age), count(age), max(age), min(age)
-         | FROM mainTable
-         | GROUP BY mytime
-      """.stripMargin)
-
-    val df1 = sql(
-      """
-        | SELECT
-        |   TIMESERIES(mytime,'minute') AS minuteLevel,
-        |   SUM(age) AS SUMValue
-        | FROM mainTable
-        | WHERE TIMESERIES(mytime,'minute')<'2016-02-23 09:02:00'
-        |   AND TIMESERIES(mytime,'minute')>='2016-02-23 09:01:00'
-        | GROUP BY
-        |   TIMESERIES(mytime,'minute')
-        | ORDER BY
-        |   TIMESERIES(mytime,'minute')
-      """.stripMargin)
-
-    checkAnswer(df1, Seq(Row(Timestamp.valueOf("2016-02-23 09:01:00"), 60)))
-  }
-
-  test("test timeseries table selection 36: count") {
-    dropDataMaps("maintable", "agg1_second", "agg1_minute")
-    sql(
-      s"""
-         | CREATE DATAMAP agg1_minute ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         |   'event_time'='mytime',
-         |   'MINUTE_GRANULARITY'='1')
-         | AS SELECT mytime, SUM(age), count(age), max(age), min(age)
-         | FROM mainTable
-         | GROUP BY mytime
-      """.stripMargin)
-
-    val df1 = sql(
-      """
-        | SELECT
-        |   TIMESERIES(mytime,'minute') AS minuteLevel,
-        |   count(age) AS countValue
-        | FROM mainTable
-        | WHERE TIMESERIES(mytime,'minute')<'2016-02-23 09:02:00'
-        |   AND TIMESERIES(mytime,'minute')>='2016-02-23 09:01:00'
-        | GROUP BY
-        |   TIMESERIES(mytime,'minute')
-        | ORDER BY
-        |   TIMESERIES(mytime,'minute')
-      """.stripMargin)
-
-    checkAnswer(df1, Seq(Row(Timestamp.valueOf("2016-02-23 09:01:00"), 3)))
-
-    preAggTableValidator(df1.queryExecution.analyzed, "maintable_agg1_minute")
-
-  }
-
-  test("test timeseries table selection 37: avg") {
-    dropDataMaps("maintable", "agg1_second", "agg1_minute")
-    sql(
-      s"""
-         | CREATE DATAMAP agg1_minute ON TABLE mainTable
-         | USING '$timeSeries'
-         | DMPROPERTIES (
-         |   'event_time'='mytime',
-         |   'MINUTE_GRANULARITY'='1')
-         | AS SELECT mytime, avg(age)
-         | FROM mainTable
-         | GROUP BY mytime
-      """.stripMargin)
-
-    val df1 = sql(
-      """
-        | SELECT
-        |   TIMESERIES(mytime,'minute') AS minuteLevel,
-        |   avg(age) AS avgValue
-        | FROM mainTable
-        | WHERE TIMESERIES(mytime,'minute')<'2016-02-23 09:02:00'
-        |   AND TIMESERIES(mytime,'minute')>='2016-02-23 09:01:00'
-        | GROUP BY
-        |   TIMESERIES(mytime,'minute')
-        | ORDER BY
-        |   TIMESERIES(mytime,'minute')
-      """.stripMargin)
-
-    checkAnswer(df1, Seq(Row(Timestamp.valueOf("2016-02-23 09:01:00"), 20)))
-
-    preAggTableValidator(df1.queryExecution.analyzed, "maintable_agg1_minute")
-  }
-
-  def preAggTableValidator(plan: LogicalPlan, actualTableName: String) : Unit ={
-    var isValidPlan = false
-    plan.transform {
-      // first check if any preaTable1 scala function is applied it is present is in plan
-      // then call is FROM create preaTable1regate table class so no need to transform the query plan
-      case ca:CarbonRelation =>
-        if (ca.isInstanceOf[CarbonDatasourceHadoopRelation]) {
-          val relation = ca.asInstanceOf[CarbonDatasourceHadoopRelation]
-          if(relation.carbonTable.getTableName.equalsIgnoreCase(actualTableName)) {
-            isValidPlan = true
-          }
-        }
-        ca
-      case logicalRelation:LogicalRelation =>
-        if(logicalRelation.relation.isInstanceOf[CarbonDatasourceHadoopRelation]) {
-          val relation = logicalRelation.relation.asInstanceOf[CarbonDatasourceHadoopRelation]
-          if(relation.carbonTable.getTableName.equalsIgnoreCase(actualTableName)) {
-            isValidPlan = true
-          }
-        }
-        logicalRelation
-    }
-    if(!isValidPlan) {
-      assert(false)
-    } else {
-      assert(true)
-    }
-  }
-
-  override def afterAll: Unit = {
-    dropDataMaps("maintable", "agg0_second", "agg0_hour", "agg0_day", "agg0_month", "agg0_year")
-    sql("drop table if exists mainTable")
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, timestampFormat)
-  }
-}
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/alterTable/TestAlterTableSortColumnsProperty.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/alterTable/TestAlterTableSortColumnsProperty.scala
index 2ac6889..c38d198 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/alterTable/TestAlterTableSortColumnsProperty.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/alterTable/TestAlterTableSortColumnsProperty.scala
@@ -131,12 +131,10 @@
       "alter_sc_agg",
       Map("sort_scope"->"local_sort", "sort_columns"->"intField")
     )
-    createAggDataMap("alter_sc_agg", "alter_sc_agg_dm1")
     createTable(
       "alter_sc_agg_base",
       Map("sort_scope"->"local_sort", "sort_columns"->"intField")
     )
-    createAggDataMap("alter_sc_agg_base", "alter_sc_agg_base_dm1")
 
     createTable(
       "alter_sc_cli",
@@ -221,19 +219,6 @@
        """.stripMargin)
   }
 
-  private def createAggDataMap(tableName: String, dataMapName: String): Unit = {
-    sql(s"create datamap PreAggSum$dataMapName on table $tableName using 'preaggregate' as " +
-        s"select stringField,sum(intField) as sum from $tableName group by stringField")
-    sql(s"create datamap PreAggAvg$dataMapName on table $tableName using 'preaggregate' as " +
-        s"select stringField,avg(intField) as avg from $tableName group by stringField")
-    sql(s"create datamap PreAggCount$dataMapName on table $tableName using 'preaggregate' as " +
-        s"select stringField,count(intField) as count from $tableName group by stringField")
-    sql(s"create datamap PreAggMin$dataMapName on table $tableName using 'preaggregate' as " +
-        s"select stringField,min(intField) as min from $tableName group by stringField")
-    sql(s"create datamap PreAggMax$dataMapName on table $tableName using 'preaggregate' as " +
-        s"select stringField,max(intField) as max from $tableName group by stringField")
-  }
-
   private def loadData(tableNames: String*): Unit = {
     tableNames.foreach { tableName =>
       sql(
@@ -563,23 +548,6 @@
     checkAnswer(sql(s"select * from $tableName where smallIntField = 3 order by floatField"), sql(s"select * from $baseTableName where smallIntField = 3 order by floatField"))
   }
 
-  test("pre-aggregate") {
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.ENABLE_QUERY_STATISTICS, "true")
-    val tableName = "alter_sc_agg"
-    val dataMapName = "alter_sc_agg_dm1"
-    val baseTableName = "alter_sc_agg_base"
-    loadData(tableName, baseTableName)
-    checkExistence(sql(s"SHOW DATAMAP ON TABLE $tableName"), true, "preaggregate", dataMapName)
-    checkExistence(sql(s"EXPLAIN select stringField,sum(intField) as sum from $tableName where stringField = 'abc2' group by stringField"), true, "preaggregate", dataMapName)
-    checkAnswer(sql(s"select stringField,sum(intField) as sum from $tableName where stringField = 'abc2' group by stringField"), sql(s"select stringField,sum(intField) as sum from $baseTableName where stringField = 'abc2' group by stringField"))
-
-    sql(s"alter table $tableName set tblproperties('sort_scope'='global_sort', 'sort_columns'='smallIntField, charField')")
-    loadData(tableName, baseTableName)
-    checkExistence(sql(s"EXPLAIN select stringField,max(intField) as sum from $tableName where stringField = 'abc2' group by stringField"), true, "preaggregate", dataMapName)
-    checkAnswer(sql(s"select stringField,max(intField) as sum from $tableName where stringField = 'abc2' group by stringField"), sql(s"select stringField,max(intField) as sum from $baseTableName where stringField = 'abc2' group by stringField"))
-  }
-
   test("carboncli -cmd sort_columns -p <segment folder>") {
     val tableName = "alter_sc_cli"
     // no_sort
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/alterTable/TestAlterTableWithColumnMetCacheAndCacheLevelProperty.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/alterTable/TestAlterTableWithColumnMetCacheAndCacheLevelProperty.scala
index 4c045c5..f4664b4 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/alterTable/TestAlterTableWithColumnMetCacheAndCacheLevelProperty.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/alterTable/TestAlterTableWithColumnMetCacheAndCacheLevelProperty.scala
@@ -160,25 +160,6 @@
     checkExistence(descResult, true, "Min/Max Index Cache Level")
   }
 
-  test("validate column_meta_cache and cache_level on child dataMap- ALTER_CACHE_LEVEL_07") {
-    intercept [Exception] {
-      sql("CREATE DATAMAP agg1 ON TABLE alter_column_meta_cache USING 'preaggregate' DMPROPERTIES('column_meta_cache'='c2') AS SELECT c2,sum(c3) FROM alter_column_meta_cache GROUP BY c2")
-    }
-
-    intercept [Exception] {
-      sql("CREATE DATAMAP agg1 ON TABLE alter_column_meta_cache USING 'preaggregate' DMPROPERTIES('cache_level'='blocklet') AS SELECT c2,sum(c3) FROM alter_column_meta_cache GROUP BY c2")
-    }
-
-    // create datamap
-    sql("CREATE DATAMAP agg1 ON TABLE alter_column_meta_cache USING 'preaggregate' AS SELECT c2,sum(c3) FROM alter_column_meta_cache GROUP BY c2")
-    intercept [Exception] {
-      sql("Alter table alter_column_meta_cache_agg1 SET TBLPROPERTIES('column_meta_cache'='c2')")
-    }
-    intercept [Exception] {
-      sql("Alter table alter_column_meta_cache_agg1 SET TBLPROPERTIES('cache_level'='BLOCKLET')")
-    }
-  }
-
   override def afterAll: Unit = {
     // drop table
     dropTable
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/compaction/TestHybridCompaction.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/compaction/TestHybridCompaction.scala
index 4c37ebd..f65116c 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/compaction/TestHybridCompaction.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/compaction/TestHybridCompaction.scala
@@ -232,31 +232,4 @@
       Array("20", "23", "37", "39", "42", "44", "54", "54", "60", "61"))
   }
 
-
-  test("PREAGG") {
-    loadSortedData()
-    loadUnsortedData()
-    val datamapName = "d1"
-    val tableNameDatamapName = tableName + "_" + datamapName
-
-    sql(
-      s"""
-         | CREATE DATAMAP $datamapName
-         | ON TABLE $tableName
-         | USING 'preaggregate'
-         | AS
-         |   SELECT AVG(age), state
-         |   FROM $tableName
-         |   GROUP BY state
-      """.stripMargin)
-
-    loadSortedData()
-    loadUnsortedData()
-
-    sql(s"ALTER TABLE $tableName COMPACT 'major'")
-    val out = sql(s"SELECT * FROM $tableNameDatamapName").collect()
-    out.map(_.get(2).toString) should equal(
-      Array("AL", "CT", "IA", "KS", "ME", "MT", "ND", "WA"))
-  }
-
 }
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestRenameTableWithDataMap.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestRenameTableWithDataMap.scala
index 78cf44b..0844b90 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestRenameTableWithDataMap.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestRenameTableWithDataMap.scala
@@ -108,54 +108,6 @@
        """.stripMargin).show(false)
   }
 
-  test("Creating a preaggregate datamap,then table rename") {
-    sql(
-      """
-        | CREATE TABLE fact_table1 (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'
-      """.stripMargin)
-    sql(s"""LOAD DATA local inpath '$resourcesPath/data_big.csv' INTO TABLE fact_table1 OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
-    sql(s"""LOAD DATA local inpath '$resourcesPath/data_big.csv' INTO TABLE fact_table1 OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
-
-    sql(
-      s"""
-         | CREATE DATAMAP dm_agg_age ON TABLE fact_table1
-         | USING 'preaggregate'
-         | AS
-         |  select deptno,deptname,sum(salary) from fact_table1
-         |  group by deptno,deptname
-            """.stripMargin)
-
-    sql(
-      s"""
-         | show datamap on table fact_table1
-       """.stripMargin).show(false)
-    sql(
-      s"""
-         | select deptno,deptname,sum(salary) from fact_table1
-         | group by deptno,deptname
-       """.stripMargin).show(false)
-
-    sql(
-      s"""
-         | explain select deptno,deptname,sum(salary) from fact_table1
-         | group by deptno,deptname
-       """.stripMargin).show(false)
-
-    val exception_tb_rename: Exception = intercept[Exception] {
-      sql(
-        s"""
-           | alter TABLE fact_table1 rename to fact_tb
-       """.stripMargin)
-    }
-    assert(exception_tb_rename.getMessage
-      .contains("Rename operation is not supported for table"
-                + " with pre-aggregate tables"))
-  }
-
   /*
    * mv datamap does not support running here, now must run in mv project.
   test("Creating a mv datamap,then table rename") {
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
index 7c3e107..9802306 100644
--- 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
@@ -415,24 +415,6 @@
     sql("DROP TABLE IF EXISTS partitionTable")
   }
 
-  test("Verify index merge for pre-aggregate table") {
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.CARBON_MERGE_INDEX_IN_SEGMENT, "true")
-    sql("DROP TABLE IF EXISTS preAggTable")
-    sql(
-      """
-        | CREATE TABLE preAggTable(id INT, name STRING, city STRING, age INT)
-        | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('SORT_COLUMNS'='city,name')
-      """.stripMargin)
-    sql(s"LOAD DATA LOCAL INPATH '$file2' INTO TABLE preAggTable OPTIONS('header'='false')")
-    assert(getIndexFileCount("default_preAggTable", "0") == 0)
-    sql("create datamap preAggSum on table preAggTable using 'preaggregate' as " +
-        "select city,sum(age) as sum from preAggTable group by city")
-    assert(getIndexFileCount("default_preAggTable_preAggSum", "0") == 0)
-    sql("DROP TABLE IF EXISTS partitionTable")
-  }
-
   test("Verify index merge for streaming table") {
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.CARBON_MERGE_INDEX_IN_SEGMENT, "true")
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/TestDataMapCommand.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/TestDataMapCommand.scala
index 0da9428..8b00943 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/TestDataMapCommand.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/TestDataMapCommand.scala
@@ -66,144 +66,12 @@
     }
   }
 
-  test("test datamap create with preagg") {
-    sql("drop datamap if exists datamap3 on table datamaptest")
-    sql(
-      "create datamap datamap3 on table datamaptest using 'preaggregate' as select count(a) from datamaptest")
-    val table = CarbonMetadata.getInstance().getCarbonTable("default", "datamaptest")
-    assert(table != null)
-    val dataMapSchemaList = table.getTableInfo.getDataMapSchemaList
-    assert(dataMapSchemaList.size() == 1)
-    assert(dataMapSchemaList.get(0).getDataMapName.equals("datamap3"))
-    assert(dataMapSchemaList.get(0).getChildSchema.getTableName.equals("datamaptest_datamap3"))
-  }
-
-  test("check hivemetastore after drop datamap") {
-    try {
-      CarbonProperties.getInstance()
-        .addProperty(CarbonCommonConstants.ENABLE_HIVE_SCHEMA_META_STORE,
-          "true")
-      sql("drop table if exists hiveMetaStoreTable")
-      sql("create table hiveMetaStoreTable (a string, b string, c string) stored by 'carbondata'")
-
-      sql(
-        "create datamap datamap_hiveMetaStoreTable on table hiveMetaStoreTable using 'preaggregate' as select count(a) from hiveMetaStoreTable")
-      checkExistence(sql("show datamap on table hiveMetaStoreTable"), true, "datamap_hiveMetaStoreTable")
-
-      sql("drop datamap datamap_hiveMetaStoreTable on table hiveMetaStoreTable")
-      checkExistence(sql("show datamap on table hiveMetaStoreTable"), false, "datamap_hiveMetaStoreTable")
-
-    } finally {
-      sql("drop table hiveMetaStoreTable")
-      CarbonProperties.getInstance()
-        .addProperty(CarbonCommonConstants.ENABLE_HIVE_SCHEMA_META_STORE,
-          CarbonCommonConstants.ENABLE_HIVE_SCHEMA_META_STORE_DEFAULT)
-    }
-  }
-
-  test("drop the table having pre-aggregate") {
-    try {
-      CarbonProperties.getInstance()
-        .addProperty(CarbonCommonConstants.ENABLE_HIVE_SCHEMA_META_STORE,
-          "true")
-      sql("drop table if exists hiveMetaStoreTable_1")
-      sql("create table hiveMetaStoreTable_1 (a string, b string, c string) stored by 'carbondata'")
-
-      sql(
-        "create datamap datamap_hiveMetaStoreTable_1 on table hiveMetaStoreTable_1 using 'preaggregate' as select count(a) from hiveMetaStoreTable_1")
-
-      checkExistence(sql("show datamap on table hiveMetaStoreTable_1"),
-        true,
-        "datamap_hiveMetaStoreTable_1")
-
-      sql("drop datamap datamap_hiveMetaStoreTable_1 on table hiveMetaStoreTable_1")
-      checkExistence(sql("show datamap on table hiveMetaStoreTable_1"),
-        false,
-        "datamap_hiveMetaStoreTable_1")
-      assert(sql("show datamap on table hiveMetaStoreTable_1").collect().length == 0)
-      sql("drop table hiveMetaStoreTable_1")
-
-      checkExistence(sql("show tables"), false, "datamap_hiveMetaStoreTable_1")
-    }
-    finally {
-      CarbonProperties.getInstance()
-        .addProperty(CarbonCommonConstants.ENABLE_HIVE_SCHEMA_META_STORE,
-          CarbonCommonConstants.ENABLE_HIVE_SCHEMA_META_STORE_DEFAULT)
-    }
-  }
-
-  test("test datamap create with preagg with duplicate name") {
-    sql(
-      s"""
-         | CREATE DATAMAP datamap10 ON TABLE datamaptest
-         | USING 'preaggregate'
-         | AS SELECT COUNT(a) FROM datamaptest
-         """.stripMargin)
-    intercept[MalformedDataMapCommandException] {
-      sql(
-        s"""
-           | CREATE DATAMAP datamap10 ON TABLE datamaptest
-           | USING 'preaggregate'
-           | AS SELECT COUNT(a) FROM datamaptest
-         """.stripMargin)
-    }
-    val table = CarbonMetadata.getInstance().getCarbonTable("default", "datamaptest")
-    assert(table != null)
-    val dataMapSchemaList = table.getTableInfo.getDataMapSchemaList
-    assert(dataMapSchemaList.size() == 2)
-  }
-
-  test("test drop non-exist datamap") {
-    intercept[NoSuchDataMapException] {
-      sql("drop datamap nonexist on table datamaptest")
-    }
-    val table = CarbonMetadata.getInstance().getCarbonTable("default", "datamaptest")
-    assert(table != null)
-    val dataMapSchemaList = table.getTableInfo.getDataMapSchemaList
-    assert(dataMapSchemaList.size() == 2)
-  }
-
-  test("test show datamap without preaggregate: don't support using non-exist class") {
-    intercept[MetadataProcessException] {
-      sql("drop table if exists datamapshowtest")
-      sql("create table datamapshowtest (a string, b string, c string) stored by 'carbondata'")
-      sql(s"CREATE DATAMAP datamap1 ON TABLE datamapshowtest USING '$newClass' ")
-      sql(s"CREATE DATAMAP datamap2 ON TABLE datamapshowtest USING '$newClass' ")
-      checkExistence(sql("SHOW DATAMAP ON TABLE datamapshowtest"), true, "datamap1", "datamap2", "(NA)", newClass)
-    }
-  }
-
-  test("test show datamap with preaggregate: don't support using non-exist class") {
-    intercept[MetadataProcessException] {
-      sql("drop table if exists datamapshowtest")
-      sql("create table datamapshowtest (a string, b string, c string) stored by 'carbondata'")
-      sql("create datamap datamap1 on table datamapshowtest using 'preaggregate' as select count(a) from datamapshowtest")
-      sql(s"CREATE DATAMAP datamap2 ON TABLE datamapshowtest USING '$newClass' ")
-      val frame = sql("show datamap on table datamapshowtest")
-      assert(frame.collect().length == 2)
-      checkExistence(frame, true, "datamap1", "datamap2", "(NA)", newClass, "default.datamapshowtest_datamap1")
-    }
-  }
-
   test("test show datamap with no datamap") {
     sql("drop table if exists datamapshowtest")
     sql("create table datamapshowtest (a string, b string, c string) stored by 'carbondata'")
     assert(sql("show datamap on table datamapshowtest").collect().length == 0)
   }
 
-  test("test show datamap after dropping datamap: don't support using non-exist class") {
-    intercept[MetadataProcessException] {
-      sql("drop table if exists datamapshowtest")
-      sql("create table datamapshowtest (a string, b string, c string) stored by 'carbondata'")
-      sql("create datamap datamap1 on table datamapshowtest using 'preaggregate' as select count(a) from datamapshowtest")
-      sql(s"CREATE DATAMAP datamap2 ON TABLE datamapshowtest USING '$newClass' ")
-      sql("drop datamap datamap1 on table datamapshowtest")
-      val frame = sql("show datamap on table datamapshowtest")
-      assert(frame.collect().length == 1)
-      checkExistence(frame, true, "datamap2", "(NA)", newClass)
-    }
-  }
-
   test("test show datamap: show datamap property related information") {
     val tableName = "datamapshowtest"
     val datamapName = "bloomdatamap"
@@ -235,161 +103,8 @@
     result.unpersist()
     sql(s"drop table if exists $tableName")
 
-    // for timeseries datamap
-    sql(s"CREATE TABLE $tableName(mytime timestamp, name string, age int) STORED BY 'org.apache.carbondata.format'")
-    sql(
-      s"""
-         | CREATE DATAMAP agg0_hour ON TABLE $tableName
-         | USING 'timeSeries'
-         | DMPROPERTIES (
-         | 'EVENT_TIME'='mytime',
-         | 'HOUR_GRANULARITY'='1')
-         | AS SELECT mytime, SUM(age) FROM $tableName
-         | GROUP BY mytime
-       """.stripMargin)
-    checkAnswer(sql(s"show datamap on table $tableName"),
-      Seq(Row("agg0_hour", "timeSeries", s"default.${tableName}_agg0_hour", "'event_time'='mytime', 'hour_granularity'='1'","NA", "NA")))
-    sql(s"drop table if exists $tableName")
-
-    // for preaggreate datamap, the property is empty
-    sql(s"CREATE TABLE $tableName(id int, name string, city string, age string)" +
-        s" STORED BY 'org.apache.carbondata.format'")
-    sql (
-      s"""
-         | CREATE DATAMAP agg0 ON TABLE $tableName USING 'preaggregate' AS
-         | SELECT name,
-         | count(age)
-         | FROM $tableName GROUP BY name
-         | """.stripMargin)
-    checkAnswer(sql(s"show datamap on table $tableName"),
-      Seq(Row("agg0", "preaggregate", s"default.${tableName}_agg0", "","NA","NA")))
-    sql(s"drop table if exists $tableName")
   }
 
-    
-
-    test("test don't support timeseries on binary data type") {
-        val tableName = "datamapshowtest"
-        sql(s"drop table if exists $tableName")
-
-        // for timeseries datamap
-        sql(s"CREATE TABLE $tableName(mytime timestamp, name string, age int, image binary) STORED BY 'org.apache.carbondata.format'")
-        val e = intercept[MalformedCarbonCommandException] {
-            sql(
-                s"""
-                   | CREATE DATAMAP agg0_hour ON TABLE $tableName
-                   | USING 'timeSeries'
-                   | DMPROPERTIES (
-                   | 'EVENT_TIME'='image',
-                   | 'HOUR_GRANULARITY'='1')
-                   | AS SELECT image, SUM(age) FROM $tableName
-                   | GROUP BY image
-                """.stripMargin)
-        }
-        assert(e.getMessage.contains("Timeseries event time is only supported on Timestamp column"))
-    }
-
-    test("test support preaggregate on binary data type") {
-        val tableName = "datamapshowtest"
-        sql(s"drop table if exists $tableName")
-
-        // for preaggreate datamap, the property is empty
-        sql(s"CREATE TABLE $tableName(id int, name string, city string, age string, image binary)" +
-                s" STORED BY 'org.apache.carbondata.format'")
-
-        sql(s"insert into $tableName  values(1,'a3','b3','c1','image2')")
-        sql(s"insert into $tableName  values(2,'a3','b2','c2','image2')")
-        sql(s"insert into $tableName  values(3,'a1','b2','c1','image3')")
-        sql(
-            s"""
-               | CREATE DATAMAP agg0 ON TABLE $tableName USING 'preaggregate' AS
-               | SELECT name, image,
-               | count(age)
-               | FROM $tableName GROUP BY name,image
-               | """.stripMargin)
-        checkAnswer(sql(s"show datamap on table $tableName"),
-            Seq(Row("agg0", "preaggregate", s"default.${tableName}_agg0", "", "NA", "NA")))
-        val pre = sql(
-            s"""
-               | select name, image, count(age)
-               | from $tableName
-               | where name = 'a3' and image=cast('image2' as binary)
-               | GROUP BY name,image
-             """.stripMargin)
-        assert(1 == pre.collect().length)
-        pre.collect().foreach { each =>
-            assert(3 == each.length)
-            assert("a3".equals(each.get(0)))
-            assert("image2".equals(new String(each.getAs[Array[Byte]](1))))
-            assert(2 == each.get(2))
-        }
-
-        val preExplain = sql(
-            s"""
-               | explain extended select name, image, count(age)
-               | from $tableName
-               | where name = 'a3' and image=cast('image2' as binary)
-               | GROUP BY name,image
-             """.stripMargin)
-        assert(preExplain.collect()(0).getString(0).contains("datamapshowtest_agg0"))
-        sql(s"drop table if exists $tableName")
-    }
-
-    test("Create table and preaggregate and load data with binary column for hive: test encode with base64") {
-        val tableName = "carbontable"
-        sql(s"drop table if exists $tableName")
-
-        sql(
-            s"""
-               | CREATE TABLE IF NOT EXISTS $tableName (
-               |    id int,
-               |    label boolean,
-               |    name string,
-               |    binaryField binary,
-               |    image boolean)
-               | STORED BY 'carbondata'
-             """.stripMargin)
-        sql(
-            s"""
-               | LOAD DATA LOCAL INPATH '$resourcesPath/binaryDataBase64.csv'
-               | INTO TABLE $tableName
-               | OPTIONS('header'='false','DELIMITER'=',','binary_decoder'='baSe64')
-             """.stripMargin)
-
-        sql(
-            s"""
-               | CREATE DATAMAP agg0 ON TABLE $tableName USING 'preaggregate' AS
-               | SELECT name, binaryField,
-               | count(id)
-               | FROM $tableName GROUP BY name,binaryField
-               | """.stripMargin)
-        checkAnswer(sql(s"show datamap on table $tableName"),
-            Seq(Row("agg0", "preaggregate", s"default.${tableName}_agg0", "", "NA", "NA")))
-        val pre = sql(
-            s"""
-               | select name, binaryField, count(id)
-               | from $tableName
-               | where name = '2.png'
-               | GROUP BY name,binaryField
-             """.stripMargin)
-        assert(1 == pre.collect().length)
-        pre.collect().foreach { each =>
-            assert(3 == each.length)
-            assert("2.png".equals(each.get(0)))
-            assert((new String(each.getAs[Array[Byte]](1))).startsWith("�PNG"))
-            assert(1 == each.get(2))
-        }
-
-        val preExplain = sql(
-            s"""
-               | explain extended select name, binaryField, count(id)
-               | from $tableName
-               | where name = '2.png'
-               | GROUP BY name,binaryField
-             """.stripMargin)
-        assert(preExplain.collect()(0).getString(0).contains("carbontable_agg0"))
-    }
-
     test("test don't support lucene on binary data type") {
         val tableName = "datamapshowtest20"
         sql(s"drop table if exists $tableName")
@@ -438,89 +153,6 @@
         sql(s"drop table if exists $tableName")
     }
 
-    test("test if preaggregate load is successfull for hivemetastore") {
-        try {
-            CarbonProperties.getInstance()
-                    .addProperty(CarbonCommonConstants.ENABLE_HIVE_SCHEMA_META_STORE, "true")
-            sql("DROP TABLE IF EXISTS maintable")
-            sql(
-                """
-                  | CREATE TABLE maintable(id int, name string, city string, age int)
-                  | STORED BY 'org.apache.carbondata.format'
-                """.stripMargin)
-            sql(
-                s"""create datamap preagg_sum on table maintable using 'preaggregate' as select id,sum(age) from maintable group by id"""
-
-          .stripMargin)
-      sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-      checkAnswer(sql(s"select * from maintable_preagg_sum"),
-        Seq(Row(1, 31), Row(2, 27), Row(3, 70), Row(4, 55)))
-    } finally {
-      CarbonProperties.getInstance()
-        .addProperty(CarbonCommonConstants.ENABLE_HIVE_SCHEMA_META_STORE,
-          CarbonCommonConstants.ENABLE_HIVE_SCHEMA_META_STORE_DEFAULT)
-    }
-  }
-
-  test("test preaggregate load for decimal column for hivemetastore") {
-    CarbonProperties.getInstance().addProperty(CarbonCommonConstants.ENABLE_HIVE_SCHEMA_META_STORE, "true")
-    sql("CREATE TABLE uniqdata(CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string,DOB timestamp,DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10),DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format'")
-    sql("insert into uniqdata select 9000,'CUST_NAME_00000','ACTIVE_EMUI_VERSION_00000','1970-01-01 01:00:03','1970-01-01 02:00:03',123372036854,-223372036854,12345678901.1234000000,22345678901.1234000000,11234567489.7976000000,-11234567489.7976000000,1")
-    sql("create datamap uniqdata_agg on table uniqdata using 'preaggregate' as select min(DECIMAL_COLUMN1) from uniqdata group by DECIMAL_COLUMN1")
-    checkAnswer(sql("select * from uniqdata_uniqdata_agg"), Seq(Row(12345678901.1234000000, 12345678901.1234000000)))
-    sql("drop datamap if exists uniqdata_agg on table uniqdata")
-  }
-
-  test("create pre-agg table with path") {
-    sql("drop table if exists main_preagg")
-    sql("drop table if exists main ")
-    val warehouse = s"$metaStoreDB/warehouse"
-    val path = warehouse + "/" + System.nanoTime + "_preAggTestPath"
-    sql(
-      s"""
-         | create table main(
-         |     year int,
-         |     month int,
-         |     name string,
-         |     salary int)
-         | stored by 'carbondata'
-         | tblproperties('sort_columns'='month,year,name')
-      """.stripMargin)
-    sql("insert into main select 10,11,'amy',12")
-    sql("insert into main select 10,11,'amy',14")
-    sql(
-      s"""
-         | create datamap preagg
-         | on table main
-         | using 'preaggregate'
-         | dmproperties ('path'='$path')
-         | as select name,avg(salary)
-         |    from main
-         |    group by name
-       """.stripMargin)
-    assertResult(true)(new File(path).exists())
-    if (FileFactory.isFileExist(CarbonTablePath.getSegmentPath(path, "0"))) {
-      assertResult(true)(new File(s"${CarbonTablePath.getSegmentPath(path, "0")}")
-         .list(new FilenameFilter {
-           override def accept(dir: File, name: String): Boolean = {
-             name.contains(CarbonCommonConstants.FACT_FILE_EXT)
-           }
-         }).length > 0)
-    } else {
-      val segment = Segment.getSegment("0", path)
-      val store = new SegmentFileStore(path, segment.getSegmentFileName)
-      store.readIndexFiles(new Configuration(false))
-      val size = store.getIndexFilesMap.asScala.map(f => f._2.size()).sum
-      assertResult(true)(size > 0)
-    }
-
-    checkAnswer(sql("select name,avg(salary) from main group by name"), Row("amy", 13.0))
-    checkAnswer(sql("select * from main_preagg"), Row("amy", 26, 2))
-    sql("drop datamap preagg on table main")
-    assertResult(false)(new File(path).exists())
-    sql("drop table main")
-  }
-
   override def afterAll {
     sql("DROP TABLE IF EXISTS maintable")
     sql("drop table if exists uniqdata")
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/flatfolder/FlatFolderTableLoadingTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/flatfolder/FlatFolderTableLoadingTestCase.scala
index 97bcb5f..d049249 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/flatfolder/FlatFolderTableLoadingTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/flatfolder/FlatFolderTableLoadingTestCase.scala
@@ -86,26 +86,6 @@
 
   }
 
-  test("data loading for flat folder pre-agg") {
-    sql(
-      """
-        | CREATE TABLE flatfolder_preagg (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,empno int)
-        | STORED BY 'org.apache.carbondata.format' tblproperties('flat_folder'='true')
-      """.stripMargin)
-    sql("create datamap p2 on table flatfolder_preagg using 'preaggregate' as select empname, designation, min(salary) from flatfolder_preagg group by empname, designation ")
-    sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE flatfolder_preagg OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
-
-    validateDataFiles("default_flatfolder_preagg", "0")
-    validateDataFiles("default_flatfolder_preagg_p2", "0")
-
-    checkAnswer(sql("select empname, designation, min(salary) from flatfolder_preagg group by empname, designation"),
-      sql("select empname, designation, min(salary) from originTable group by empname, designation"))
-
-  }
-
   test("merge index flat folder issue") {
     sql("drop table if exists t1")
     sql("create table t1(c1 int,c2 string,c3 float,c4 date) stored by 'carbondata' TBLPROPERTIES('flat_folder'='true')")
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/DeleteCarbonTableTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/DeleteCarbonTableTestCase.scala
index 3036ef1..cd1bacd 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/DeleteCarbonTableTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/DeleteCarbonTableTestCase.scala
@@ -164,21 +164,6 @@
     sql("DROP TABLE IF EXISTS carbon2")
   }
 
-  test("test if delete is unsupported for pre-aggregate tables") {
-    sql("drop table if exists preaggMain")
-    sql("drop table if exists preaggmain_preagg1")
-    sql("create table preaggMain (a string, b string, c string) stored by 'carbondata'")
-    sql("create datamap preagg1 on table PreAggMain USING 'preaggregate' as select a,sum(b) from PreAggMain group by a")
-    intercept[RuntimeException] {
-      sql("delete from preaggmain where a = 'abc'").show()
-    }.getMessage.contains("Delete operation is not supported for tables")
-    intercept[RuntimeException] {
-      sql("delete from preaggmain_preagg1 where preaggmain_a = 'abc'").show()
-    }.getMessage.contains("Delete operation is not supported for pre-aggregate table")
-    sql("drop table if exists preaggMain")
-    sql("drop table if exists preaggmain_preagg1")
-  }
-
   test("test select query after compaction, delete and clean files") {
     sql("drop table if exists select_after_clean")
     sql("create table select_after_clean(id int, name string) stored by 'carbondata'")
@@ -302,30 +287,6 @@
 
   }
 
-  test("block deleting records from table which has preaggregate datamap") {
-    sql("drop table if exists test_dm_main")
-    sql("drop table if exists test_dm_main_preagg1")
-
-    sql("create table test_dm_main (a string, b string, c string) stored by 'carbondata'")
-    sql("insert into test_dm_main select 'aaa','bbb','ccc'")
-    sql("insert into test_dm_main select 'bbb','bbb','ccc'")
-    sql("insert into test_dm_main select 'ccc','bbb','ccc'")
-
-    sql(
-      "create datamap preagg1 on table test_dm_main using 'preaggregate' as select" +
-      " a,sum(b) from test_dm_main group by a")
-
-    assert(intercept[UnsupportedOperationException] {
-      sql("delete from test_dm_main_preagg1 where test_dm_main_a = 'bbb'")
-    }.getMessage.contains("Delete operation is not supported for pre-aggregate table"))
-    assert(intercept[UnsupportedOperationException] {
-      sql("delete from test_dm_main where a = 'ccc'")
-    }.getMessage.contains("Delete operation is not supported for tables which have a pre-aggregate table"))
-
-    sql("drop table if exists test_dm_main")
-    sql("drop table if exists test_dm_main_preagg1")
-  }
-
   test("block deleting records from table which has index datamap") {
     sql("drop table if exists test_dm_index")
 
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/UpdateCarbonTableTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/UpdateCarbonTableTestCase.scala
index ef18035..0e7f41a 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/UpdateCarbonTableTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/UpdateCarbonTableTestCase.scala
@@ -521,21 +521,6 @@
     sql("DROP TABLE IF EXISTS iud.rand")
   }
 
-  test("test if update is unsupported for pre-aggregate tables") {
-    sql("drop table if exists preaggMain")
-    sql("drop table if exists preaggMain_preagg1")
-    sql("create table preaggMain (a string, b string, c string) stored by 'carbondata'")
-    sql("create datamap preagg1 on table PreAggMain using 'preaggregate' as select a,sum(b) from PreAggMain group by a")
-    intercept[RuntimeException] {
-      sql("update preaggmain set (a)=('a')").show
-    }.getMessage.contains("Update operation is not supported for tables")
-    intercept[RuntimeException] {
-      sql("update preaggMain_preagg1 set (a)=('a')").show
-    }.getMessage.contains("Update operation is not supported for pre-aggregate table")
-    sql("drop table if exists preaggMain")
-    sql("drop table if exists preaggMain_preagg1")
-  }
-
   test("Update operation on carbon table with singlepass") {
     sql(s"""set ${ CarbonLoadOptionConstants.CARBON_OPTIONS_SINGLE_PASS }=true""")
     sql("drop database if exists carbon1 cascade")
@@ -714,31 +699,6 @@
     sql("drop table if exists senten")
   }
 
-  test("block updating table which has preaggreate datamap") {
-    sql("use iud")
-    sql("drop table if exists test_dm_main")
-    sql("drop table if exists test_dm_main_preagg1")
-
-    sql("create table test_dm_main (a string, b string, c string) stored by 'carbondata'")
-    sql("insert into test_dm_main select 'aaa','bbb','ccc'")
-    sql("insert into test_dm_main select 'bbb','bbb','ccc'")
-    sql("insert into test_dm_main select 'ccc','bbb','ccc'")
-
-    sql(
-      "create datamap preagg1 on table test_dm_main using 'preaggregate' as select" +
-      " a,sum(b) from test_dm_main group by a")
-
-    assert(intercept[UnsupportedOperationException] {
-      sql("update test_dm_main_preagg1 set(test_dm_main_a) = ('aaa') where test_dm_main_a = 'bbb'")
-    }.getMessage.contains("Update operation is not supported for pre-aggregate table"))
-    assert(intercept[UnsupportedOperationException] {
-      sql("update test_dm_main set(a) = ('aaa') where a = 'ccc'")
-    }.getMessage.contains("Update operation is not supported for tables which have a pre-aggregate table"))
-
-    sql("drop table if exists test_dm_main")
-    sql("drop table if exists test_dm_main_preagg1")
-  }
-
   test("block updating table which has index datamap") {
     sql("use iud")
     sql("drop table if exists test_dm_index")
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/longstring/VarcharDataTypesBasicTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/longstring/VarcharDataTypesBasicTestCase.scala
index 96f1095..fb8dd5f 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/longstring/VarcharDataTypesBasicTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/longstring/VarcharDataTypesBasicTestCase.scala
@@ -304,66 +304,6 @@
       CarbonCommonConstants.ENABLE_UNSAFE_COLUMN_PAGE_DEFAULT)
   }
 
-  test("Create datamap with long string column selected") {
-    val datamapName = "pre_agg_dm"
-    prepareTable()
-    sql(
-      s"""
-         | CREATE DATAMAP $datamapName ON TABLE $longStringTable
-         | USING 'preaggregate'
-         | DMPROPERTIES('LONG_STRING_COLUMNS'='description, note')
-         | AS SELECT id,description,note,count(*) FROM $longStringTable
-         | GROUP BY id,description,note
-         |""".
-        stripMargin)
-
-    val parentTable = CarbonMetadata.getInstance().getCarbonTable("default", longStringTable)
-    assert(null != parentTable)
-    val dmSchemaList = parentTable.getTableInfo.getDataMapSchemaList
-    assert(dmSchemaList.size() == 1)
-    assert(dmSchemaList.get(0).getDataMapName.equalsIgnoreCase(datamapName))
-
-    val dmTableName = longStringTable + "_" + datamapName
-    val dmTable = CarbonMetadata.getInstance().getCarbonTable("default", dmTableName)
-    assert(null != dmTable)
-    assert(dmTable.getColumnByName(longStringTable + "_description").getDataType
-      == DataTypes.VARCHAR)
-    assert(dmTable.getColumnByName(longStringTable + "_note").getDataType
-      == DataTypes.VARCHAR)
-    sql(s"DROP DATAMAP IF EXISTS $datamapName ON TABLE $longStringTable")
-  }
-
-  test("creating datamap with long string column selected and loading data should be success") {
-
-    sql(s"drop table if exists $longStringTable")
-    val datamapName = "pre_agg_dm"
-    sql(
-      s"""
-         | CREATE TABLE if not exists $longStringTable(
-         | id INT, name STRING, description STRING, address STRING, note STRING
-         | ) STORED BY 'carbondata'
-         | TBLPROPERTIES('LONG_STRING_COLUMNS'='description, note', 'SORT_COLUMNS'='name')
-         |""".stripMargin)
-
-    sql(
-      s"""
-         | CREATE DATAMAP $datamapName ON TABLE $longStringTable
-         | USING 'preaggregate'
-         | AS SELECT id,description,note,count(*) FROM $longStringTable
-         | GROUP BY id,description,note
-         |""".
-        stripMargin)
-
-    sql(
-      s"""
-         | LOAD DATA LOCAL INPATH '$inputFile' INTO TABLE $longStringTable
-         | OPTIONS('header'='false')
-       """.stripMargin)
-
-    checkAnswer(sql(s"select count(*) from $longStringTable"), Row(1000))
-    sql(s"drop table if exists $longStringTable")
-  }
-
   test("create table with varchar column and complex column") {
     sql("DROP TABLE IF EXISTS varchar_complex_table")
     sql("""
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableQueryTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableQueryTestCase.scala
index e151547..e897047 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableQueryTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableQueryTestCase.scala
@@ -472,21 +472,6 @@
     }
   }
 
-  test("drop partition on preAggregate table should fail"){
-    sql("drop table if exists partitionTable")
-    sql("create table partitionTable (id int,city string,age int) partitioned by(name string) stored by 'carbondata'".stripMargin)
-    sql(
-    s"""create datamap preaggTable on table partitionTable using 'preaggregate' as select id,sum(age) from partitionTable group by id"""
-      .stripMargin)
-    sql("insert into partitionTable select 1,'Bangalore',30,'John'")
-    sql("insert into partitionTable select 2,'Chennai',20,'Huawei'")
-    checkAnswer(sql("show partitions partitionTable"), Seq(Row("name=John"),Row("name=Huawei")))
-    intercept[Exception]{
-      sql("alter table partitionTable drop PARTITION(name='John')")
-    }
-    sql("drop datamap if exists preaggTable on table partitionTable")
-  }
-
   test("validate data in partition table after dropping and adding a column") {
     sql("drop table if exists par")
     sql("create table par(name string, add string) partitioned by (age double) stored by " +
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionWithPreaggregateTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionWithPreaggregateTestCase.scala
deleted file mode 100644
index b55fec7..0000000
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionWithPreaggregateTestCase.scala
+++ /dev/null
@@ -1,677 +0,0 @@
-/*
-* Licensed to the Apache Software Foundation (ASF) under one or more
-* contributor license agreements.  See the NOTICE file distributed with
-* this work for additional information regarding copyright ownership.
-* The ASF licenses this file to You under the Apache License, Version 2.0
-* (the "License"); you may not use this file except in compliance with
-* the License.  You may obtain id 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.standardpartition
-
-import scala.collection.JavaConverters._
-
-import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
-import org.apache.spark.sql.execution.datasources.LogicalRelation
-import org.apache.spark.sql.hive.CarbonRelation
-import org.apache.spark.sql.{CarbonDatasourceHadoopRelation, CarbonEnv, Row}
-import org.apache.spark.sql.test.util.QueryTest
-import org.scalatest.BeforeAndAfterAll
-
-import org.apache.carbondata.core.datastore.impl.FileFactory
-
-class StandardPartitionWithPreaggregateTestCase extends QueryTest with BeforeAndAfterAll {
-
-  val testData = s"$resourcesPath/sample.csv"
-
-  override def beforeAll(): Unit = {
-    sql("drop database if exists partition_preaggregate cascade")
-    sql("create database partition_preaggregate")
-    sql("use partition_preaggregate")
-    sql(
-      """
-        | CREATE TABLE par(id INT, name STRING, age INT) PARTITIONED BY(city STRING)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    sql(
-      """
-        | CREATE TABLE maintable(id int, name string, city string) partitioned by (age int)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-  }
-
-  override def afterAll(): Unit = {
-    sql("drop database if exists partition_preaggregate cascade")
-    sql("use default")
-  }
-
-  // Create aggregate table on partition with partition column in aggregation only.
-  test("test preaggregate table creation on partition table with partition col as aggregation") {
-    sql("create datamap p1 on table par using 'preaggregate' as select id, sum(city) from par group by id")
-    assert(!CarbonEnv.getCarbonTable(Some("partition_preaggregate"), "par_p1")(sqlContext.sparkSession).isHivePartitionTable)
-  }
-
-  // Create aggregate table on partition with partition column in projection and aggregation only.
-  test("test preaggregate table creation on partition table with partition col as projection") {
-    sql("create datamap p2 on table par using 'preaggregate' as select id, city, min(city) from par group by id,city ")
-    assert(CarbonEnv.getCarbonTable(Some("partition_preaggregate"), "par_p2")(sqlContext.sparkSession).isHivePartitionTable)
-  }
-
-  // Create aggregate table on partition with partition column as group by.
-  test("test preaggregate table creation on partition table with partition col as group by") {
-    sql("create datamap p3 on table par using 'preaggregate' as select id, max(city) from par group by id,city ")
-    assert(CarbonEnv.getCarbonTable(Some("partition_preaggregate"), "par_p3")(sqlContext.sparkSession).isHivePartitionTable)
-  }
-
-  // Create aggregate table on partition without partition column.
-  test("test preaggregate table creation on partition table without partition column") {
-    sql("create datamap p4 on table par using 'preaggregate' as select name, count(id) from par group by name ")
-    assert(!CarbonEnv.getCarbonTable(Some("partition_preaggregate"), "par_p4")(sqlContext.sparkSession).isHivePartitionTable)
-  }
-  
-  test("test data correction in aggregate table when partition column is used") {
-    sql("create datamap p1 on table maintable using 'preaggregate' as select id, sum(age) from maintable group by id, age")
-    checkAnswer(sql("select * from maintable_p1"),
-      Seq(Row(1,31,31),
-        Row(2,27,27),
-        Row(3,70,35),
-        Row(4,26,26),
-        Row(4,29,29)))
-    preAggTableValidator(sql("select id, sum(age) from maintable group by id, age").queryExecution.analyzed, "maintable_p1")
-    sql("drop datamap p1 on table maintable")
-  }
-
-  test("test data correction in aggregate table when partition column is not used") {
-    sql("create datamap p2 on table maintable using 'preaggregate' as select id, max(age) from maintable group by id")
-    checkAnswer(sql("select * from maintable_p2"),
-      Seq(Row(1,31),
-        Row(2,27),
-        Row(3,35),
-        Row(4,29)))
-    preAggTableValidator(sql("select id, max(age) from maintable group by id").queryExecution.analyzed, "maintable_p2")
-    sql("drop datamap p2 on table maintable")
-  }
-
-  test("test data correction with insert overwrite") {
-    sql("drop table if exists partitionone")
-    sql(
-      """
-        | CREATE TABLE if not exists partitionone (empname String)
-        | PARTITIONED BY (year int, month int,day int)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    sql("create datamap p1 on table partitionone using 'preaggregate' as select empname, sum(year) from partitionone group by empname, year, month,day")
-    sql("insert into partitionone values('k',2014,1,1)")
-    sql("insert overwrite table partitionone values('v',2014,1,1)")
-    checkAnswer(sql("select * from partitionone"), Seq(Row("v",2014,1,1)))
-    checkAnswer(sql("select * from partitionone_p1"), Seq(Row("v",2014,2014,1,1)))
-  }
-
-  test("test data correction with insert overwrite on different value") {
-    sql("drop table if exists partitionone")
-    sql(
-      """
-        | CREATE TABLE if not exists partitionone (empname String)
-        | PARTITIONED BY (year int, month int,day int)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    sql("create datamap p1 on table partitionone using 'preaggregate' as select empname, sum(year) from partitionone group by empname, year, month,day")
-    sql("insert into partitionone values('k',2014,1,1)")
-    sql("insert overwrite table partitionone values('v',2015,1,1)")
-    checkAnswer(sql("select * from partitionone"), Seq(Row("k",2014,1,1), Row("v",2015,1,1)))
-    checkAnswer(sql("select * from partitionone_p1"), Seq(Row("k",2014,2014,1,1), Row("v",2015,2015,1,1)))
-  }
-
-  test("test to check column ordering in parent and child table") {
-    sql("drop table if exists partitionone")
-    sql(
-      """
-        | CREATE TABLE if not exists partitionone (empname String)
-        | PARTITIONED BY (year int, month int,day int)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    sql("create datamap p1 on table partitionone using 'preaggregate' as select empname, sum(year) from partitionone group by empname, month, year,day")
-    val parentTable = CarbonEnv.getCarbonTable(Some("partition_preaggregate"), "partitionone")(sqlContext.sparkSession)
-    val childTable = CarbonEnv.getCarbonTable(Some("partition_preaggregate"), "partitionone_p1")(sqlContext.sparkSession)
-    val parentPartitionColumns = parentTable.getPartitionInfo.getColumnSchemaList
-    val childPartitionColumns = childTable.getPartitionInfo.getColumnSchemaList
-    assert(parentPartitionColumns.asScala.zip(childPartitionColumns.asScala).forall {
-      case (a,b) =>
-        a.getColumnName.equalsIgnoreCase(b.getParentColumnTableRelations.get(0).getColumnName)
-    })
-  }
-  
-  test("test data after minor compaction on partition table with pre-aggregate") {
-    sql("drop table if exists partitionone")
-    sql(
-      """
-        | CREATE TABLE if not exists partitionone (empname String)
-        | PARTITIONED BY (year int, month int,day int)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    sql("create datamap p1 on table partitionone using 'preaggregate' as select empname, sum(year) from partitionone group by empname, year, month,day")
-    sql("insert into partitionone values('k',2014,1,1)")
-    sql("insert into partitionone values('k',2014,1,2)")
-    sql("insert into partitionone values('k',2014,1,1)")
-    sql("insert into partitionone values('k',2015,2,1)")
-    sql("alter table partitionone compact 'minor'")
-    val showSegments = sql("show segments for table partitionone").collect().map{a=> (a.get(0), a.get(1))}
-    assert(showSegments.count(_._2 == "Success") == 1)
-    assert(showSegments.count(_._2 == "Compacted") == 4)
-  }
-
-  test("test data after major compaction on partition table with pre-aggregate") {
-    sql("drop table if exists partitionone")
-    sql(
-      """
-        | CREATE TABLE if not exists partitionone (empname String)
-        | PARTITIONED BY (year int, month int,day int)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    sql("create datamap p1 on table partitionone using 'preaggregate' as select empname, sum(year) from partitionone group by empname, year, month,day")
-    sql("insert into partitionone values('k',2014,1,1)")
-    sql("insert into partitionone values('k',2014,1,2)")
-    sql("insert into partitionone values('k',2014,1,1)")
-    sql("insert into partitionone values('k',2015,2,1)")
-    sql("insert into partitionone values('k',2015,2,1)")
-    sql("alter table partitionone compact 'major'")
-    val showSegments = sql("show segments for table partitionone").collect().map{a=> (a.get(0), a.get(1))}
-    assert(showSegments.count(_._2 == "Success") == 1)
-    assert(showSegments.count(_._2 == "Compacted") == 5)
-  }
-
-  test("test drop partition 1") {
-    sql("drop table if exists partitionone")
-    sql(
-      """
-        | CREATE TABLE if not exists partitionone (empname String)
-        | PARTITIONED BY (year int, month int,day int)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    sql("create datamap p1 on table partitionone using 'preaggregate' as select empname, sum(year) from partitionone group by empname, year, month, day")
-    sql("insert into partitionone values('k',2014,1,1)")
-    sql("insert into partitionone values('k',2014,1,2)")
-    sql("insert into partitionone values('k',2014,1,1)")
-    sql("insert into partitionone values('k',2015,2,1)")
-    sql("insert into partitionone values('k',2015,2,1)")
-    sql("alter table partitionone drop partition(day=1)")
-    checkAnswer(sql("select * from partitionone"), Seq(Row("k",2014,1,2)))
-    checkAnswer(sql("select * from partitionone_p1"), Seq(Row("k",2014,2014,1,2)))
-  }
-
-  test("test drop partition 2") {
-    sql("drop table if exists partitionone")
-    sql(
-      """
-        | CREATE TABLE if not exists partitionone (empname String)
-        | PARTITIONED BY (year int, month int,day int)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    sql("create datamap p1 on table partitionone using 'preaggregate' as select empname, sum(year) from partitionone group by empname, year, month, day")
-    sql("insert into partitionone values('k',2014,1,1)")
-    sql("insert into partitionone values('k',2014,1,2)")
-    sql("insert into partitionone values('k',2014,1,1)")
-    sql("insert into partitionone values('k',2015,2,3)")
-    sql("insert into partitionone values('k',2015,2,1)")
-    sql("alter table partitionone drop partition(day=1)")
-    checkAnswer(sql("select * from partitionone"), Seq(Row("k",2014,1,2), Row("k",2015,2,3)))
-    checkAnswer(sql("select * from partitionone_p1"), Seq(Row("k",2014,2014,1,2), Row("k",2015,2015,2,3)))
-  }
-
-  test("test drop partition directory") {
-    sql("drop table if exists droppartition")
-    sql(
-      """
-        | CREATE TABLE if not exists droppartition (empname String)
-        | PARTITIONED BY (year int, month int,day int)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    sql("insert into droppartition values('k',2014,1,1)")
-    sql("insert into droppartition values('k',2015,2,3)")
-    sql("alter table droppartition drop partition(year=2015,month=2,day=3)")
-    sql("clean files for table droppartition")
-    val table = CarbonEnv.getCarbonTable(Option("partition_preaggregate"), "droppartition")(sqlContext.sparkSession)
-    val tablePath = table.getTablePath
-    val carbonFiles = FileFactory.getCarbonFile(tablePath).listFiles().filter{
-      file => file.getName.equalsIgnoreCase("year=2015")
-    }
-    assert(carbonFiles.length == 0)
-  }
-
-  test("test data with filter query") {
-    sql("drop table if exists partitionone")
-    sql(
-      """
-        | CREATE TABLE if not exists partitionone (empname String)
-        | PARTITIONED BY (year int, month int,day int)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    sql("create datamap p1 on table partitionone using 'preaggregate' as select empname, sum(year) from partitionone group by empname, year, month, day")
-    sql("insert into partitionone values('k',2014,1,1)")
-    sql("insert into partitionone values('k',2014,1,2)")
-    sql("insert into partitionone values('k',2014,1,1)")
-    sql("insert into partitionone values('k',2015,2,3)")
-    sql("insert into partitionone values('k',2015,2,1)")
-    sql("alter table partitionone drop partition(day=1)")
-    checkAnswer(sql("select empname, sum(year) from partitionone where day=3 group by empname, year, month, day"), Seq(Row("k",2015)))
-    checkAnswer(sql("select * from partitionone_p1"), Seq(Row("k",2014,2014,1,2), Row("k",2015,2015,2,3)))
-  }
-
-  test("test drop partition 3") {
-    sql("drop table if exists partitionone")
-    sql(
-      """
-        | CREATE TABLE if not exists partitionone (empname String)
-        | PARTITIONED BY (year int, month int,day int)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    sql("create datamap p1 on table partitionone using 'preaggregate' as select empname, sum(year) from partitionone group by empname, year, month, day")
-    sql("insert into partitionone values('k',2014,1,1)")
-    sql("insert into partitionone values('k',2014,1,2)")
-    sql("insert into partitionone values('k',2014,1,1)")
-    sql("insert into partitionone values('k',2015,2,3)")
-    sql("insert into partitionone values('k',2015,2,1)")
-    sql("alter table partitionone drop partition(day=1,month=1)")
-    checkAnswer(sql("select * from partitionone"), Seq(Row("k",2014,1,2), Row("k",2015, 2,3), Row("k",2015, 2,1)))
-    checkAnswer(sql("select * from partitionone_p1"), Seq(Row("k",2014,2014,1,2), Row("k",2015,2015,2,3), Row("k",2015,2015,2,1)))
-  }
-
-  test("test drop partition 4") {
-    sql("drop table if exists partitionone")
-    sql(
-      """
-        | CREATE TABLE if not exists partitionone (empname String)
-        | PARTITIONED BY (year int, month int,day int)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    sql("create datamap p1 on table partitionone using 'preaggregate' as select empname, sum(year) from partitionone group by empname, year, month, day")
-    sql("insert into partitionone values('k',2014,1,1)")
-    sql("insert into partitionone values('k',2014,1,2)")
-    sql("insert into partitionone values('k',2014,1,1)")
-    sql("insert into partitionone values('k',2015,2,3)")
-    sql("insert into partitionone values('k',2015,2,1)")
-    sql("alter table partitionone drop partition(year=2014,day=1)")
-    checkAnswer(sql("select * from partitionone"), Seq(Row("k",2014,1,2), Row("k",2015, 2,3), Row("k",2015, 2,1)))
-    checkAnswer(sql("select * from partitionone_p1"), Seq(Row("k",2014,2014,1,2), Row("k",2015,2015, 2,3), Row("k",2015,2015, 2,1)))
-  }
-
-  test("test drop partition 5") {
-    sql("drop table if exists partitionone")
-    sql(
-      """
-        | CREATE TABLE if not exists partitionone (empname String)
-        | PARTITIONED BY (year int, month int,day int)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    sql("create datamap p1 on table partitionone using 'preaggregate' as select empname, sum(year) from partitionone group by empname, year, month, day")
-    sql("insert into partitionone values('k',2014,1,1)")
-    sql("insert into partitionone values('k',2014,1,2)")
-    sql("insert into partitionone values('k',2014,1,1)")
-    sql("insert into partitionone values('k',2015,2,3)")
-    sql("insert into partitionone values('k',2015,2,1)")
-    sql("alter table partitionone drop partition(year=2014,month=1, day=1)")
-
-    checkAnswer(sql("select * from partitionone"), Seq(Row("k",2014,1,2), Row("k",2015, 2,3), Row("k",2015, 2,1)))
-    checkAnswer(sql("select * from partitionone_p1"), Seq(Row("k",2014,2014,1,2), Row("k",2015,2015,2,3), Row("k",2015,2015,2,1)))
-    sql("show partitions partitionone_p1").show()
-  }
-
-  test("test drop partition 6") {
-    sql("drop table if exists partitionone")
-    sql(
-      """
-        | CREATE TABLE if not exists partitionone (empname String)
-        | PARTITIONED BY (year int, month int,day int)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    sql("create datamap p1 on table partitionone using 'preaggregate' as select empname, sum(year) from partitionone group by empname, year, month, day")
-    sql("insert into partitionone values('k',2014,1,1)")
-    sql("insert into partitionone values('k',2014,1,2)")
-    sql("insert into partitionone values('k',2014,1,1)")
-    sql("insert into partitionone values('k',2015,2,3)")
-    sql("insert into partitionone values('k',2015,2,1)")
-    sql("alter table partitionone drop partition(year=2014,month=1, day=1)")
-    checkAnswer(sql("select * from partitionone"), Seq(Row("k",2014,1,2), Row("k",2015, 2,3), Row("k",2015, 2,1)))
-    checkAnswer(sql("select * from partitionone_p1"), Seq(Row("k",2014,2014,1,2), Row("k",2015,2015,2,3), Row("k",2015,2015,2,1)))
-  }
-
-  test("test drop partition 7") {
-    sql("drop table if exists partitionone")
-    sql(
-      """
-        | CREATE TABLE if not exists partitionone (empname String)
-        | PARTITIONED BY (year int, month int,day int)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    sql(
-      "create datamap p1 on table partitionone using 'preaggregate' as select empname, sum(year) from partitionone group by empname, year, day")
-    sql(
-      "create datamap p2 on table partitionone using 'preaggregate' as select empname, sum(year) from partitionone group by empname, month")
-    sql("insert into partitionone values('k',2014,1,1)")
-    sql("insert into partitionone values('k',2014,1,2)")
-    val exceptionMessage = intercept[Exception] {
-      sql("alter table partitionone drop partition(year=2014,month=1, day=1)")
-    }.getMessage
-    assert(exceptionMessage.contains("Cannot drop partition as one of the partition"))
-    assert(exceptionMessage.contains("p2"))
-    assert(exceptionMessage.contains("p1"))
-  }
-
-  test("test drop partition 8") {
-    sql("drop table if exists partitionone")
-    sql(
-      """
-        | CREATE TABLE if not exists partitionone (empname String)
-        | PARTITIONED BY (year int, month int,day int)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    sql(
-      "create datamap p1 on table partitionone using 'preaggregate' as select empname, sum(year) from partitionone group by empname, year, month, day")
-    sql(
-      "create datamap p2 on table partitionone using 'preaggregate' as select empname, sum(year) from partitionone group by empname, month, day")
-    sql(
-      "create datamap p3 on table partitionone using 'preaggregate' as select empname, sum(year) from partitionone group by empname, year, month")
-    sql("insert into partitionone values('k',2014,1,1)")
-    sql("insert into partitionone values('k',2014,1,2)")
-    val exceptionMessage = intercept[Exception] {
-      sql("alter table partitionone drop partition(year=2014,month=1, day=1)")
-    }.getMessage
-    assert(exceptionMessage.contains("Cannot drop partition as one of the partition"))
-    assert(exceptionMessage.contains("p2"))
-    assert(exceptionMessage.contains("p3"))
-    assert(!exceptionMessage.contains("p1"))
-  }
-
-  test("test drop partition 9") {
-    sql("drop table if exists partitionone")
-    sql(
-      """
-        | CREATE TABLE if not exists partitionone (empname String)
-        | PARTITIONED BY (year int, month int,day int)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    sql(
-      "create datamap p1 on table partitionone using 'preaggregate' as select empname, sum(year) from partitionone group by empname")
-    sql("insert into partitionone values('k',2014,1,1)")
-    sql("insert into partitionone values('k',2014,1,2)")
-    val exceptionMessage = intercept[Exception] {
-      sql("alter table partitionone drop partition(year=2014,month=1, day=1)")
-    }.getMessage
-    assert(exceptionMessage.contains("Cannot drop partition as one of the partition"))
-    assert(exceptionMessage.contains("p1"))
-  }
-
-  test("test drop partition 10") {
-    sql("drop table if exists partitionone")
-    sql(
-      """
-        | CREATE TABLE if not exists partitionone (empname String)
-        | PARTITIONED BY (year int, month int,day int)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    sql(
-      "create datamap p1 on table partitionone using 'preaggregate' as select empname, sum(year) from partitionone group by empname")
-    sql(
-      "create datamap p2 on table partitionone using 'preaggregate' as select empname, sum(year) from partitionone group by empname, year, month, day")
-    sql("insert into partitionone values('k',2014,1,1)")
-    sql("insert into partitionone values('k',2014,1,2)")
-    val exceptionMessage = intercept[Exception] {
-      sql("alter table partitionone drop partition(year=2014,month=1, day=1)")
-    }.getMessage
-    assert(exceptionMessage.contains("Cannot drop partition as one of the partition"))
-    assert(exceptionMessage.contains("p1"))
-    sql("drop datamap p1 on table partitionone")
-    sql("alter table partitionone drop partition(year=2014,month=1, day=1)")
-  }
-
-  test("test drop partition 11") {
-    sql("drop table if exists partitionone")
-    sql(
-      """
-        | CREATE TABLE if not exists partitionone (empname String)
-        | PARTITIONED BY (year int, month int,day int)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    sql(
-      "create datamap p1 on table partitionone using 'preaggregate' as select empname, sum(year) from partitionone group by empname, year")
-    sql("insert into partitionone values('k',2014,1,1)")
-    val exceptionMessage = intercept[Exception] {
-      sql("alter table partitionone_p1 drop partition(partitionone_year=1)")
-    }.getMessage
-    assert(exceptionMessage.contains("Cannot drop partition directly on aggregate table"))
-  }
-
-  test("test drop partition 12") {
-    sql("drop table if exists partitionone")
-    sql(
-      """
-        | CREATE TABLE if not exists partitionone (empname String)
-        | PARTITIONED BY (year int, month int,day int)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    sql(
-      "create datamap p1 on table partitionone using 'preaggregate' as select empname, sum(year) from partitionone group by empname")
-    sql("insert into partitionone values('k',2014,1,1)")
-    val exceptionMessage = intercept[Exception] {
-      sql("alter table partitionone_p1 drop partition(year=2014)")
-    }.getMessage
-    assert(exceptionMessage.contains("operation failed for partition_preaggregate.partitionone_p1: Not a partitioned table"))
-  }
-
-  test("test add partition on aggregate table") {
-    sql("drop table if exists partitionone")
-    sql(
-      """
-        | CREATE TABLE if not exists partitionone (empname String)
-        | PARTITIONED BY (year int, month int,day int)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    sql(
-      "create datamap p1 on table partitionone using 'preaggregate' as select empname, sum(year) from partitionone group by empname")
-    assert(intercept[Exception] {
-      sql("alter table partitionone_p1 add partition(c=1)")
-    }.getMessage.equals("Cannot add partition directly on non partitioned table"))
-  }
-
-  test("test if alter rename is blocked on partition table with preaggregate") {
-    sql("drop table if exists partitionone")
-    sql(
-      """
-        | CREATE TABLE if not exists partitionone (empname String, id int)
-        | PARTITIONED BY (year int, month int,day int)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    sql(
-      "create datamap p1 on table partitionone using 'preaggregate' as select empname, sum(year) from partitionone group by empname")
-    intercept[Exception] {
-      sql("alter table partitionone rename to p")
-    }
-  }
-
-  test("test dropping partition which has already been deleted") {
-    sql("drop table if exists partitiontable")
-    sql("create table partitiontable(id int,name string) partitioned by (email string) " +
-      "stored by 'carbondata' tblproperties('sort_scope'='global_sort')")
-    sql("insert into table partitiontable select 1,'huawei','abc'")
-    sql("create datamap ag1 on table partitiontable using 'preaggregate' as select count(email),id" +
-      " from partitiontable group by id")
-    sql("create datamap ag2 on table partitiontable using 'preaggregate' as select sum(email),name" +
-      " from partitiontable group by name")
-    sql("create datamap ag3 on table partitiontable using 'preaggregate' as select max(email),name" +
-      " from partitiontable group by name")
-    sql("create datamap ag4 on table partitiontable using 'preaggregate' as select min(email),name" +
-      " from partitiontable group by name")
-    sql("create datamap ag5 on table partitiontable using 'preaggregate' as select avg(email),name" +
-      " from partitiontable group by name")
-    sql("alter table partitiontable add partition (email='def')")
-    sql("insert into table partitiontable select 1,'huawei','def'")
-    sql("drop datamap ag1 on table partitiontable")
-    sql("drop datamap ag2 on table partitiontable")
-    sql("drop datamap ag3 on table partitiontable")
-    sql("drop datamap ag4 on table partitiontable")
-    sql("drop datamap ag5 on table partitiontable")
-    sql("alter table partitiontable drop partition(email='def')")
-    assert(intercept[Exception] {
-      sql("alter table partitiontable drop partition(email='def')")
-    }.getMessage.contains("No partition is dropped. One partition spec 'Map(email -> def)' does not exist in table 'partitiontable' database 'partition_preaggregate'"))
-  }
-
-  test("test Pre-Aggregate table creation with count(*) on Partition table") {
-    sql("drop table if exists partitiontable")
-    sql("create table partitiontable(id int,name string) partitioned by (email string) " +
-      "stored by 'carbondata' tblproperties('sort_scope'='global_sort')")
-    sql("insert into table partitiontable select 1,'huawei','abc'")
-    sql("create datamap ag1 on table partitiontable using 'preaggregate' as select count(*),id" +
-      " from partitiontable group by id")
-    sql("insert into table partitiontable select 1,'huawei','def'")
-    assert(sql("show datamap on table partitiontable").collect().head.get(0).toString.equalsIgnoreCase("ag1"))
-    sql("drop datamap ag1 on table partitiontable")
-  }
-  
-  test("test blocking partitioning of Pre-Aggregate table") {
-    sql("drop table if exists updatetime_8")
-    sql("create table updatetime_8" +
-      "(countryid smallint,hs_len smallint,minstartdate string,startdate string,newdate string,minnewdate string) partitioned by (imex smallint) stored by 'carbondata' tblproperties('sort_scope'='global_sort','sort_columns'='countryid,imex,hs_len,minstartdate,startdate,newdate,minnewdate','table_blocksize'='256')")
-    sql("create datamap ag on table updatetime_8 using 'preaggregate' dmproperties('partitioning'='false') as select imex,sum(hs_len) from updatetime_8 group by imex")
-    val carbonTable = CarbonEnv.getCarbonTable(Some("partition_preaggregate"), "updatetime_8_ag")(sqlContext.sparkSession)
-    assert(!carbonTable.isHivePartitionTable)
-    sql("drop table if exists updatetime_8")
-  }
-
-  test("Test data updation after compaction on Partition with Pre-Aggregate tables") {
-    sql("drop table if exists partitionallcompaction")
-    sql(
-      "create table partitionallcompaction(empno int,empname String,designation String," +
-      "workgroupcategory int,workgroupcategoryname String,deptno int,projectjoindate timestamp," +
-      "projectenddate date,attendance int,utilization int,salary int) partitioned by (deptname " +
-      "String,doj timestamp,projectcode int) stored  by 'carbondata' tblproperties" +
-      "('sort_scope'='global_sort')")
-    sql(
-      "create datamap sensor_1 on table partitionallcompaction using 'preaggregate' as select " +
-      "sum(salary),doj, deptname,projectcode from partitionallcompaction group by doj," +
-      "deptname,projectcode")
-    sql(
-      s"""LOAD DATA local inpath '$resourcesPath/data.csv' OVERWRITE INTO TABLE
-         |partitionallcompaction OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""".stripMargin)
-    sql(
-      s"""LOAD DATA local inpath '$resourcesPath/data.csv' OVERWRITE INTO TABLE
-         |partitionallcompaction OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""".stripMargin)
-    sql(
-      s"""LOAD DATA local inpath '$resourcesPath/data.csv' OVERWRITE INTO TABLE
-         |partitionallcompaction PARTITION(deptname='Learning', doj, projectcode) OPTIONS
-         |('DELIMITER'= ',', 'QUOTECHAR'= '"') """.stripMargin)
-    sql(
-      s"""LOAD DATA local inpath '$resourcesPath/data.csv' OVERWRITE INTO TABLE
-         |partitionallcompaction PARTITION(deptname='configManagement', doj, projectcode) OPTIONS
-         |('DELIMITER'= ',', 'QUOTECHAR'= '"')""".stripMargin)
-    sql(
-      s"""LOAD DATA local inpath '$resourcesPath/data.csv' OVERWRITE INTO TABLE
-         |partitionallcompaction PARTITION(deptname='network', doj, projectcode) OPTIONS
-         |('DELIMITER'= ',', 'QUOTECHAR'= '"')""".stripMargin)
-    sql(
-      s"""LOAD DATA local inpath '$resourcesPath/data.csv' OVERWRITE INTO TABLE
-         |partitionallcompaction PARTITION(deptname='protocol', doj, projectcode) OPTIONS
-         |('DELIMITER'= ',', 'QUOTECHAR'= '"')""".stripMargin)
-    sql(
-      s"""LOAD DATA local inpath '$resourcesPath/data.csv' OVERWRITE INTO TABLE
-         |partitionallcompaction PARTITION(deptname='security', doj, projectcode) OPTIONS
-         |('DELIMITER'= ',', 'QUOTECHAR'= '"')""".stripMargin)
-    sql("ALTER TABLE partitionallcompaction COMPACT 'MINOR'").collect()
-    checkAnswer(sql("select count(empno) from partitionallcompaction where empno=14"),
-      Seq(Row(5)))
-  }
-
-  test("Test data updation in Aggregate query after compaction on Partitioned table with Pre-Aggregate table") {
-    sql("drop table if exists updatetime_8")
-    sql("create table updatetime_8" +
-      "(countryid smallint,hs_len smallint,minstartdate string,startdate string,newdate string,minnewdate string) partitioned by (imex smallint) stored by 'carbondata' tblproperties('sort_scope'='global_sort','sort_columns'='countryid,imex,hs_len,minstartdate,startdate,newdate,minnewdate','table_blocksize'='256')")
-    sql("create datamap ag on table updatetime_8 using 'preaggregate' as select sum(hs_len) from updatetime_8 group by imex")
-    sql("insert into updatetime_8 select 21,20,'fbv','gbv','wvsw','vwr',23")
-    sql("insert into updatetime_8 select 21,20,'fbv','gbv','wvsw','vwr',24")
-    sql("insert into updatetime_8 select 21,20,'fbv','gbv','wvsw','vwr',23")
-    sql("insert into updatetime_8 select 21,21,'fbv','gbv','wvsw','vwr',24")
-    sql("insert into updatetime_8 select 21,21,'fbv','gbv','wvsw','vwr',24")
-    sql("insert into updatetime_8 select 21,21,'fbv','gbv','wvsw','vwr',24")
-    sql("insert into updatetime_8 select 21,21,'fbv','gbv','wvsw','vwr',25")
-    sql("insert into updatetime_8 select 21,21,'fbv','gbv','wvsw','vwr',25")
-    sql("alter table updatetime_8 compact 'minor'")
-    sql("alter table updatetime_8 compact 'minor'")
-    checkAnswer(sql("select sum(hs_len) from updatetime_8 group by imex"),Seq(Row(40),Row(42),Row(83)))
-  }
-
-  test("check partitioning for child tables with various combinations") {
-    sql("drop table if exists partitionone")
-    sql(
-      """
-        | CREATE TABLE if not exists partitionone (empname String, id int)
-        | PARTITIONED BY (year int, month int,day int)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    sql(
-      "create datamap p7 on table partitionone using 'preaggregate' as select empname, sum(year), sum(day) from partitionone group by empname, year, day")
-    sql(
-      "create datamap p1 on table partitionone using 'preaggregate' as select empname, sum(year) from partitionone group by empname")
-    sql(
-      "create datamap p2 on table partitionone using 'preaggregate' as select empname, sum(year) from partitionone group by empname, year")
-    sql(
-      "create datamap p3 on table partitionone using 'preaggregate' as select empname, sum(year), sum(month) from partitionone group by empname, year, month")
-    sql(
-      "create datamap p4 on table partitionone using 'preaggregate' as select empname, sum(year) from partitionone group by empname, year, month, day")
-    sql(
-      "create datamap p5 on table partitionone using 'preaggregate' as select empname, sum(year) from partitionone group by empname, month")
-    sql(
-      "create datamap p6 on table partitionone using 'preaggregate' as select empname, sum(year), sum(month) from partitionone group by empname, month, day")
-    assert(!CarbonEnv.getCarbonTable(Some("partition_preaggregate"),"partitionone_p1")(sqlContext.sparkSession).isHivePartitionTable)
-    assert(CarbonEnv.getCarbonTable(Some("partition_preaggregate"),"partitionone_p2")(sqlContext.sparkSession).getPartitionInfo.getColumnSchemaList.size() == 1)
-    assert(CarbonEnv.getCarbonTable(Some("partition_preaggregate"),"partitionone_p3")(sqlContext.sparkSession).getPartitionInfo.getColumnSchemaList.size == 2)
-    assert(CarbonEnv.getCarbonTable(Some("partition_preaggregate"),"partitionone_p4")(sqlContext.sparkSession).getPartitionInfo.getColumnSchemaList.size == 3)
-    assert(!CarbonEnv.getCarbonTable(Some("partition_preaggregate"),"partitionone_p5")(sqlContext.sparkSession).isHivePartitionTable)
-    assert(!CarbonEnv.getCarbonTable(Some("partition_preaggregate"),"partitionone_p6")(sqlContext.sparkSession).isHivePartitionTable)
-    assert(!CarbonEnv.getCarbonTable(Some("partition_preaggregate"),"partitionone_p7")(sqlContext.sparkSession).isHivePartitionTable)
-  }
-
-  test("test partition at last column") {
-    sql("drop table if exists partitionone")
-    sql("create table partitionone(a int,b int) partitioned by (c int) stored by 'carbondata'")
-    sql("insert into partitionone values(1,2,3)")
-    sql("drop datamap if exists dm1")
-    sql("create datamap dm1 on table partitionone using 'preaggregate' as select c,sum(b) from partitionone group by c")
-    checkAnswer(sql("select c,sum(b) from partitionone group by c"), Seq(Row(3,2)))
-    sql("drop table if exists partitionone")
-  }
-
-  def preAggTableValidator(plan: LogicalPlan, actualTableName: String) : Unit = {
-    var isValidPlan = false
-    plan.transform {
-      // first check if any preaTable1 scala function is applied it is present is in plan
-      // then call is from create preaTable1regate table class so no need to transform the query plan
-      case ca:CarbonRelation =>
-        if (ca.isInstanceOf[CarbonDatasourceHadoopRelation]) {
-          val relation = ca.asInstanceOf[CarbonDatasourceHadoopRelation]
-          if(relation.carbonTable.getTableName.equalsIgnoreCase(actualTableName)) {
-            isValidPlan = true
-          }
-        }
-        ca
-      case logicalRelation:LogicalRelation =>
-        if(logicalRelation.relation.isInstanceOf[CarbonDatasourceHadoopRelation]) {
-          val relation = logicalRelation.relation.asInstanceOf[CarbonDatasourceHadoopRelation]
-          if(relation.carbonTable.getTableName.equalsIgnoreCase(actualTableName)) {
-            isValidPlan = true
-          }
-        }
-        logicalRelation
-    }
-    assert(isValidPlan)
-  }
-
-}
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/util/SparkQueryTest.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/util/SparkQueryTest.scala
deleted file mode 100644
index bad300b..0000000
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/util/SparkQueryTest.scala
+++ /dev/null
@@ -1,50 +0,0 @@
-package org.apache.carbondata.spark.util
-
-import org.apache.spark.sql.{CarbonDatasourceHadoopRelation, DataFrame}
-import org.apache.spark.sql.execution.datasources.LogicalRelation
-import org.apache.spark.sql.hive.CarbonRelation
-import org.apache.spark.sql.test.util.QueryTest
-
-class SparkQueryTest extends QueryTest {
-
-  /**
-   * check whether the pre-aggregate tables are in DataFrame
-   *
-   * @param df DataFrame
-   * @param exists whether the preAggTableNames exists
-   * @param preAggTableNames preAggTableNames
-   */
-  def checkPreAggTable(df: DataFrame, exists: Boolean, preAggTableNames: String*): Unit = {
-    val plan = df.queryExecution.analyzed
-    for (preAggTableName <- preAggTableNames) {
-      var isValidPlan = false
-      plan.transform {
-        // first check if any preaTable1 scala function is applied it is present is in plan
-        // then call is from create preaTable1regate table class so no need to transform the query plan
-        case ca: CarbonRelation =>
-          if (ca.isInstanceOf[CarbonDatasourceHadoopRelation]) {
-            val relation = ca.asInstanceOf[CarbonDatasourceHadoopRelation]
-            if (relation.carbonTable.getTableName.equalsIgnoreCase(preAggTableName)) {
-              isValidPlan = true
-            }
-          }
-          ca
-        case logicalRelation: LogicalRelation =>
-          if (logicalRelation.relation.isInstanceOf[CarbonDatasourceHadoopRelation]) {
-            val relation = logicalRelation.relation.asInstanceOf[CarbonDatasourceHadoopRelation]
-            if (relation.carbonTable.getTableName.equalsIgnoreCase(preAggTableName)) {
-              isValidPlan = true
-            }
-          }
-          logicalRelation
-      }
-
-      if (exists != isValidPlan) {
-        assert(false)
-      } else {
-        assert(true)
-      }
-    }
-  }
-
-}
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/sql/commands/TestCarbonDropCacheCommand.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/sql/commands/TestCarbonDropCacheCommand.scala
index 982ec76..9f2de4b 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/sql/commands/TestCarbonDropCacheCommand.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/sql/commands/TestCarbonDropCacheCommand.scala
@@ -87,50 +87,6 @@
     }
   }
 
-
-  test("Test preaggregate datamap") {
-    val tableName = "t2"
-
-    sql(s"CREATE TABLE $tableName(empno int, empname String, designation String, " +
-        s"doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, " +
-        s"deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp," +
-        s"attendance int, utilization int, salary int) stored by 'carbondata'")
-    sql(s"CREATE DATAMAP dpagg ON TABLE $tableName USING 'preaggregate' AS " +
-        s"SELECT AVG(salary), workgroupcategoryname from $tableName GROUP BY workgroupcategoryname")
-    sql(s"LOAD DATA INPATH '$resourcesPath/data.csv' INTO TABLE $tableName")
-    sql(s"SELECT * FROM $tableName").collect()
-    sql(s"SELECT AVG(salary), workgroupcategoryname from $tableName " +
-        s"GROUP BY workgroupcategoryname").collect()
-    val droppedCacheKeys = clone(CacheProvider.getInstance().getCarbonCache.getCacheMap.keySet())
-
-    sql(s"DROP METACACHE ON TABLE $tableName")
-
-    val cacheAfterDrop = clone(CacheProvider.getInstance().getCarbonCache.getCacheMap.keySet())
-    droppedCacheKeys.removeAll(cacheAfterDrop)
-
-    val tableIdentifier = new TableIdentifier(tableName, Some(dbName))
-    val carbonTable = CarbonEnv.getCarbonTable(tableIdentifier)(sqlContext.sparkSession)
-    val dbPath = CarbonEnv
-      .getDatabaseLocation(tableIdentifier.database.get, sqlContext.sparkSession)
-    val tablePath = carbonTable.getTablePath
-    val preaggPath = dbPath + CarbonCommonConstants.FILE_SEPARATOR + carbonTable.getTableName +
-                     "_" + carbonTable.getTableInfo.getDataMapSchemaList.get(0).getDataMapName +
-                     CarbonCommonConstants.FILE_SEPARATOR
-
-    // Check if table index entries are dropped
-    assert(droppedCacheKeys.asScala.exists(key => key.startsWith(tablePath)))
-
-    // check if cache does not have any more table index entries
-    assert(!cacheAfterDrop.asScala.exists(key => key.startsWith(tablePath)))
-
-    // Check if preaggregate index entries are dropped
-    assert(droppedCacheKeys.asScala.exists(key => key.startsWith(preaggPath)))
-
-    // check if cache does not have any more preaggregate index entries
-    assert(!cacheAfterDrop.asScala.exists(key => key.startsWith(preaggPath)))
-  }
-
-
   test("Test bloom filter") {
     val tableName = "t3"
 
@@ -170,28 +126,6 @@
     assert(!cacheAfterDrop.asScala.exists(key => key.contains(bloomPath)))
   }
 
-
-  test("Test preaggregate datamap fail") {
-    val tableName = "t4"
-
-    sql(s"CREATE TABLE $tableName(empno int, empname String, designation String, " +
-        s"doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, " +
-        s"deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp," +
-        s"attendance int, utilization int, salary int) stored by 'carbondata'")
-    sql(s"CREATE DATAMAP dpagg ON TABLE $tableName USING 'preaggregate' AS " +
-        s"SELECT AVG(salary), workgroupcategoryname from $tableName GROUP BY workgroupcategoryname")
-    sql(s"LOAD DATA INPATH '$resourcesPath/data.csv' INTO TABLE $tableName")
-    sql(s"SELECT * FROM $tableName").collect()
-    sql(s"SELECT AVG(salary), workgroupcategoryname from $tableName " +
-        s"GROUP BY workgroupcategoryname").collect()
-
-    val fail_message = intercept[UnsupportedOperationException] {
-      sql(s"DROP METACACHE ON TABLE ${tableName}_dpagg")
-    }.getMessage
-    assert(fail_message.contains("Operation not allowed on child table."))
-  }
-
-
   def clone(oldSet: util.Set[String]): util.HashSet[String] = {
     val newSet = new util.HashSet[String]
     newSet.addAll(oldSet)
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/sql/commands/TestCarbonShowCacheCommand.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/sql/commands/TestCarbonShowCacheCommand.scala
index 0cc2727..a219bfb 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/sql/commands/TestCarbonShowCacheCommand.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/sql/commands/TestCarbonShowCacheCommand.scala
@@ -101,10 +101,6 @@
       "workgroupcategoryname,deptname,projectcode,projectjoindate,projectenddate,attendance," +
       "utilization,salary,deptno from cache_4").collect()
 
-    // datamap
-    sql("create datamap cache_4_count on table cache_4 using 'preaggregate' as " +
-        "select workgroupcategoryname,count(empname) as count from cache_4 group by workgroupcategoryname")
-
     // count star to cache index
     sql("select max(deptname) from cache_db.cache_1").collect()
     sql("SELECT deptno FROM cache_db.cache_1 where deptno=10").collect()
@@ -217,13 +213,6 @@
       Seq(Row("Index", "0 B", "0/1 index files cached", "DRIVER"),
         Row("Dictionary", "0 B", "", "DRIVER")))
 
-    // Table with Index, Dictionary & PreAgg child table
-    val result4 = sql("show metacache on table default.cache_4").collect()
-    assertResult(3)(result4.length)
-    assertResult("1/1 index files cached")(result4(0).getString(2))
-    assertResult("0 B")(result4(1).getString(1))
-    assertResult("preaggregate")(result4(2).getString(2))
-
     sql("use default").collect()
 
     // Table with 5 index files
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/events/CreateTableEvents.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/events/CreateTableEvents.scala
index abb1ede..eddd22a 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/events/CreateTableEvents.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/events/CreateTableEvents.scala
@@ -33,7 +33,7 @@
 
 /**
  * Class for handling operations after data load completion and before final
- * commit of load operation. Example usage: For loading pre-aggregate tables
+ * commit of load operation.
  */
 case class CreateTablePostExecutionEvent(sparkSession: SparkSession,
     identifier: AbsoluteTableIdentifier) extends Event with TableEventInfo
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
index 34a82d8..a6dfc8a 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
@@ -39,18 +39,17 @@
 import org.apache.spark.sql.hive.DistributionUtil
 import org.apache.spark.sql.profiler.{GetPartition, Profiler}
 import org.apache.spark.sql.util.SparkSQLUtil.sessionState
-import org.apache.spark.util.{SparkUtil, TaskCompletionListener}
+import org.apache.spark.util.TaskCompletionListener
 
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.converter.SparkDataTypeConverterImpl
-import org.apache.carbondata.core.constants.{CarbonCommonConstants, CarbonCommonConstantsInternal}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datamap.DataMapFilter
 import org.apache.carbondata.core.datastore.block.Distributable
 import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.indexstore.PartitionSpec
-import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, ColumnarFormatVersion}
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
 import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, TableInfo}
-import org.apache.carbondata.core.scan.expression.Expression
 import org.apache.carbondata.core.scan.expression.conditional.ImplicitExpression
 import org.apache.carbondata.core.scan.filter.FilterUtil
 import org.apache.carbondata.core.scan.model.QueryModel
@@ -628,28 +627,12 @@
     val carbonSessionInfo = ThreadLocalSessionInfo.getCarbonSessionInfo
     if (carbonSessionInfo != null) {
       val tableUniqueKey = identifier.getDatabaseName + "." + identifier.getTableName
-      val validateInputSegmentsKey = CarbonCommonConstants.VALIDATE_CARBON_INPUT_SEGMENTS +
-                                     tableUniqueKey
-      CarbonInputFormat.setValidateSegmentsToAccess(conf, carbonSessionInfo.getThreadParams
-        .getProperty(validateInputSegmentsKey, "true").toBoolean)
-      val queryOnPreAggStreamingKey = CarbonCommonConstantsInternal.QUERY_ON_PRE_AGG_STREAMING +
-                                      tableUniqueKey
-      val queryOnPreAggStreaming = carbonSessionInfo.getThreadParams
-        .getProperty(queryOnPreAggStreamingKey, "false").toBoolean
       val inputSegmentsKey = CarbonCommonConstants.CARBON_INPUT_SEGMENTS + tableUniqueKey
-      CarbonInputFormat.setValidateSegmentsToAccess(conf, carbonSessionInfo.getThreadParams
-        .getProperty(validateInputSegmentsKey, "true").toBoolean)
       CarbonInputFormat
         .setQuerySegment(conf,
           carbonSessionInfo.getThreadParams
             .getProperty(inputSegmentsKey,
               CarbonProperties.getInstance().getProperty(inputSegmentsKey, "*")))
-      if (queryOnPreAggStreaming) {
-        CarbonInputFormat.setAccessStreamingSegments(conf, queryOnPreAggStreaming)
-        carbonSessionInfo.getThreadParams.removeProperty(queryOnPreAggStreamingKey)
-        carbonSessionInfo.getThreadParams.removeProperty(inputSegmentsKey)
-        carbonSessionInfo.getThreadParams.removeProperty(validateInputSegmentsKey)
-      }
     }
     format
   }
@@ -665,38 +648,13 @@
     // when validate segments is disabled in thread local update it to CarbonTableInputFormat
     if (carbonSessionInfo != null) {
       val tableUniqueKey = identifier.getDatabaseName + "." + identifier.getTableName
-      val validateInputSegmentsKey = CarbonCommonConstants.VALIDATE_CARBON_INPUT_SEGMENTS +
-                                     tableUniqueKey
-      CarbonInputFormat.setValidateSegmentsToAccess(conf, carbonSessionInfo.getThreadParams
-        .getProperty(validateInputSegmentsKey, "true").toBoolean)
-      val queryOnPreAggStreamingKey = CarbonCommonConstantsInternal.QUERY_ON_PRE_AGG_STREAMING +
-                                      tableUniqueKey
-      val queryOnPreAggStreaming = carbonSessionInfo.getThreadParams
-        .getProperty(queryOnPreAggStreamingKey, "false").toBoolean
       val inputSegmentsKey = CarbonCommonConstants.CARBON_INPUT_SEGMENTS + tableUniqueKey
-      CarbonInputFormat.setValidateSegmentsToAccess(conf, carbonSessionInfo.getThreadParams
-        .getProperty(validateInputSegmentsKey, "true").toBoolean)
       CarbonInputFormat
         .setQuerySegment(conf,
           carbonSessionInfo.getThreadParams
             .getProperty(inputSegmentsKey, carbonSessionInfo.getSessionParams
               .getProperty(inputSegmentsKey,
               CarbonProperties.getInstance().getProperty(inputSegmentsKey, "*"))))
-      if (queryOnPreAggStreaming) {
-        CarbonInputFormat.setAccessStreamingSegments(conf, queryOnPreAggStreaming)
-        // union for streaming preaggregate can happen concurrently from spark.
-        // Need to clean both maintable and aggregate table segments
-        var keyList = scala.collection.immutable.List[String]()
-        carbonSessionInfo.getThreadParams.getAll.asScala.foreach {
-          case (key, value) =>
-            if (key.contains(CarbonCommonConstants.VALIDATE_CARBON_INPUT_SEGMENTS) ||
-                key.contains(CarbonCommonConstantsInternal.QUERY_ON_PRE_AGG_STREAMING) ||
-                key.contains(CarbonCommonConstants.CARBON_INPUT_SEGMENTS)) {
-              keyList ::= key
-            }
-        }
-        keyList.foreach(key => carbonSessionInfo.getThreadParams.removeProperty(key))
-      }
     }
     format
   }
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 0656e49..51c8cfd 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
@@ -290,7 +290,6 @@
       tableProperties: Map[String, String],
       bucketFields: Option[BucketFields],
       isAlterFlow: Boolean = false,
-      isPreAggFlow: Boolean = false,
       tableComment: Option[String] = None): TableModel = {
 
     // do not allow below key words as column name
@@ -349,11 +348,9 @@
 
     // validate the local dictionary columns defined, this we will validated if the local dictionary
     // is enabled, else it is not validated
-    // if it is preaggregate flow no need to validate anything, as all the properties will be
-    // inherited from parent table
     if ((tableProperties.get(CarbonCommonConstants.LOCAL_DICTIONARY_ENABLE).isDefined &&
           tableProperties(CarbonCommonConstants.LOCAL_DICTIONARY_ENABLE).trim
-            .equalsIgnoreCase("true")) && !isPreAggFlow) {
+            .equalsIgnoreCase("true"))) {
       var localDictIncludeColumns: Seq[String] = Seq[String]()
       var localDictExcludeColumns: Seq[String] = Seq[String]()
       val isLocalDictIncludeDefined = tableProperties
@@ -1297,16 +1294,10 @@
         Field(e1, e2.dataType, Some(e1), e2.children, null, e3)
     }
 
-  lazy val addPreAgg: Parser[String] =
+  lazy val addMVSkipUDF: Parser[String] =
     SELECT ~> restInput <~ opt(";") ^^ {
       case query =>
-        "select preAGG() as preAgg, " + query
-    }
-
-  lazy val addPreAggLoad: Parser[String] =
-    SELECT ~> restInput <~ opt(";") ^^ {
-      case query =>
-        "select preAggLoad() as preAggLoad, " + query
+        "select mv() as mv, " + query
     }
 
   protected lazy val primitiveFieldType: Parser[Field] =
diff --git a/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/CarbonAnalyzer.scala b/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/CarbonAnalyzer.scala
index dfb89fd..8f4d45e 100644
--- a/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/CarbonAnalyzer.scala
+++ b/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/CarbonAnalyzer.scala
@@ -14,6 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.spark.sql.hive
 
 import org.apache.spark.sql.SparkSession
@@ -39,9 +40,7 @@
   }
 
   override def execute(plan: LogicalPlan): LogicalPlan = {
-    var logicalPlan = analyzer.execute(plan)
-    logicalPlan = CarbonPreAggregateDataLoadingRules(sparkSession).apply(logicalPlan)
-    logicalPlan = CarbonPreAggregateQueryRules(sparkSession).apply(logicalPlan)
+    val logicalPlan = analyzer.execute(plan)
     if (mvPlan != null) {
       mvPlan.apply(logicalPlan)
     } else {
diff --git a/integration/spark2/src/main/java/org/apache/carbondata/datamap/DataMapManager.java b/integration/spark2/src/main/java/org/apache/carbondata/datamap/DataMapManager.java
index 1d4d862..59e0ac7 100644
--- a/integration/spark2/src/main/java/org/apache/carbondata/datamap/DataMapManager.java
+++ b/integration/spark2/src/main/java/org/apache/carbondata/datamap/DataMapManager.java
@@ -24,8 +24,6 @@
 import org.apache.carbondata.spark.util.CarbonScalaUtil;
 
 import static org.apache.carbondata.core.metadata.schema.datamap.DataMapClassProvider.MV;
-import static org.apache.carbondata.core.metadata.schema.datamap.DataMapClassProvider.PREAGGREGATE;
-import static org.apache.carbondata.core.metadata.schema.datamap.DataMapClassProvider.TIMESERIES;
 
 import org.apache.spark.sql.SparkSession;
 
@@ -48,11 +46,7 @@
   public DataMapProvider getDataMapProvider(CarbonTable mainTable, DataMapSchema dataMapSchema,
       SparkSession sparkSession) throws MalformedDataMapCommandException {
     DataMapProvider provider;
-    if (dataMapSchema.getProviderName().equalsIgnoreCase(PREAGGREGATE.toString())) {
-      provider = new PreAggregateDataMapProvider(mainTable, dataMapSchema, sparkSession);
-    } else if (dataMapSchema.getProviderName().equalsIgnoreCase(TIMESERIES.toString())) {
-      provider = new TimeseriesDataMapProvider(mainTable, dataMapSchema, sparkSession);
-    } else if (dataMapSchema.getProviderName().equalsIgnoreCase(MV.toString())) {
+    if (dataMapSchema.getProviderName().equalsIgnoreCase(MV.toString())) {
       provider = (DataMapProvider) CarbonScalaUtil.createDataMapProvider(
           "org.apache.carbondata.mv.datamap.MVDataMapProvider",
               sparkSession,
diff --git a/integration/spark2/src/main/java/org/apache/carbondata/datamap/PreAggregateDataMapProvider.java b/integration/spark2/src/main/java/org/apache/carbondata/datamap/PreAggregateDataMapProvider.java
deleted file mode 100644
index 46c1180..0000000
--- a/integration/spark2/src/main/java/org/apache/carbondata/datamap/PreAggregateDataMapProvider.java
+++ /dev/null
@@ -1,120 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.datamap;
-
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.carbondata.common.annotations.InterfaceAudience;
-import org.apache.carbondata.common.exceptions.sql.MalformedDataMapCommandException;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datamap.DataMapProvider;
-import org.apache.carbondata.core.datamap.dev.DataMapFactory;
-import org.apache.carbondata.core.metadata.schema.datamap.DataMapProperty;
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
-import org.apache.carbondata.core.metadata.schema.table.DataMapSchema;
-
-import org.apache.spark.sql.SparkSession;
-import org.apache.spark.sql.execution.command.preaaggregate.PreAggregateTableHelper;
-import org.apache.spark.sql.execution.command.table.CarbonDropTableCommand;
-import scala.Some;
-
-@InterfaceAudience.Internal
-public class PreAggregateDataMapProvider extends DataMapProvider {
-  protected PreAggregateTableHelper helper;
-  protected CarbonDropTableCommand dropTableCommand;
-  protected SparkSession sparkSession;
-  private String dbName;
-  private String tableName;
-
-  PreAggregateDataMapProvider(CarbonTable table, DataMapSchema schema,
-      SparkSession sparkSession) {
-    super(table, schema);
-    this.sparkSession = sparkSession;
-    this.dbName = table.getDatabaseName();
-    this.tableName = table.getTableName() + '_' + schema.getDataMapName();
-  }
-
-  @Override
-  public void initMeta(String ctasSqlStatement) throws MalformedDataMapCommandException {
-    DataMapSchema dataMapSchema = getDataMapSchema();
-    validateDmProperty(dataMapSchema);
-    helper = new PreAggregateTableHelper(
-        getMainTable(), dataMapSchema.getDataMapName(), dataMapSchema.getProviderName(),
-        dataMapSchema.getProperties(), ctasSqlStatement, null, false);
-    helper.initMeta(sparkSession);
-  }
-
-  private void validateDmProperty(DataMapSchema dataMapSchema)
-      throws MalformedDataMapCommandException {
-    if (!dataMapSchema.getProperties().isEmpty()) {
-      Map<String, String> properties = new HashMap<>(dataMapSchema.getProperties());
-      properties.remove(DataMapProperty.DEFERRED_REBUILD);
-      properties.remove(DataMapProperty.PATH);
-      properties.remove(DataMapProperty.PARTITIONING);
-      properties.remove(CarbonCommonConstants.LONG_STRING_COLUMNS);
-      if (properties.size() > 0) {
-        throw new MalformedDataMapCommandException(
-                "Only 'path', 'partitioning' and 'long_string_columns' dmproperties " +
-                "are allowed for this datamap");
-      }
-    }
-  }
-
-  @Override
-  public void cleanMeta() {
-    dropTableCommand = new CarbonDropTableCommand(
-        true,
-        new Some<>(dbName),
-        tableName,
-        true);
-    dropTableCommand.processMetadata(sparkSession);
-  }
-
-  @Override
-  public void cleanData() {
-    if (dropTableCommand != null) {
-      dropTableCommand.processData(sparkSession);
-    }
-  }
-
-  @Override
-  public boolean rebuild() {
-    if (helper != null) {
-      helper.initData(sparkSession);
-    }
-    return true;
-  }
-
-  @Override
-  public DataMapFactory getDataMapFactory() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public boolean supportRebuild() {
-    return false;
-  }
-
-  @Override
-  public boolean rebuildInternal(String newLoadName, Map<String, List<String>> segmentMap,
-      CarbonTable carbonTable) {
-    return false;
-  }
-}
diff --git a/integration/spark2/src/main/java/org/apache/carbondata/datamap/TimeseriesDataMapProvider.java b/integration/spark2/src/main/java/org/apache/carbondata/datamap/TimeseriesDataMapProvider.java
deleted file mode 100644
index 4b93f59..0000000
--- a/integration/spark2/src/main/java/org/apache/carbondata/datamap/TimeseriesDataMapProvider.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.datamap;
-
-import java.util.Map;
-
-import org.apache.carbondata.common.annotations.InterfaceAudience;
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
-import org.apache.carbondata.core.metadata.schema.table.DataMapSchema;
-
-import org.apache.spark.sql.SparkSession;
-import org.apache.spark.sql.execution.command.preaaggregate.PreAggregateTableHelper;
-import org.apache.spark.sql.execution.command.timeseries.TimeSeriesUtil;
-import scala.Some;
-import scala.Tuple2;
-
-@InterfaceAudience.Internal
-public class TimeseriesDataMapProvider extends PreAggregateDataMapProvider {
-
-  TimeseriesDataMapProvider(CarbonTable mainTable, DataMapSchema dataMapSchema,
-      SparkSession sparkSession) {
-    super(mainTable, dataMapSchema, sparkSession);
-  }
-
-  @Override
-  public void initMeta(String ctasSqlStatement) {
-    DataMapSchema dataMapSchema = getDataMapSchema();
-    CarbonTable mainTable = getMainTable();
-    Map<String, String> dmProperties = dataMapSchema.getProperties();
-    String dmProviderName = dataMapSchema.getProviderName();
-    TimeSeriesUtil.validateTimeSeriesGranularity(dmProperties, dmProviderName);
-    Tuple2<String, String> details =
-        TimeSeriesUtil.getTimeSeriesGranularityDetails(dmProperties, dmProviderName);
-    dmProperties.remove(details._1());
-    helper = new PreAggregateTableHelper(
-        mainTable, dataMapSchema.getDataMapName(), dataMapSchema.getProviderName(),
-        dmProperties, ctasSqlStatement, new Some(details._1()), false);
-    helper.initMeta(sparkSession);
-  }
-
-}
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/datamap/IndexDataMapRebuildRDD.scala b/integration/spark2/src/main/scala/org/apache/carbondata/datamap/IndexDataMapRebuildRDD.scala
index e339b34..f42cc8f 100644
--- a/integration/spark2/src/main/scala/org/apache/carbondata/datamap/IndexDataMapRebuildRDD.scala
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/datamap/IndexDataMapRebuildRDD.scala
@@ -80,7 +80,7 @@
     val tableIdentifier = carbonTable.getAbsoluteTableIdentifier
     val segmentStatusManager = new SegmentStatusManager(tableIdentifier)
     val validAndInvalidSegments = segmentStatusManager
-      .getValidAndInvalidSegments(carbonTable.isChildTable)
+      .getValidAndInvalidSegments(carbonTable.isChildTableForMV)
     val validSegments = validAndInvalidSegments.getValidSegments
     val indexedCarbonColumns = carbonTable.getIndexedColumns(schema)
     val operationContext = new OperationContext()
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/AggregateDataMapCompactor.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/AggregateDataMapCompactor.scala
deleted file mode 100644
index 6d28295..0000000
--- a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/AggregateDataMapCompactor.scala
+++ /dev/null
@@ -1,167 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.spark.rdd
-
-import java.util.concurrent.ExecutorService
-
-import scala.collection.JavaConverters._
-
-import org.apache.spark.sql.{CarbonSession, SQLContext}
-import org.apache.spark.sql.execution.command.CompactionModel
-import org.apache.spark.sql.execution.command.management.CarbonLoadDataCommand
-import org.apache.spark.sql.execution.command.preaaggregate.{CommitPreAggregateListener, PreAggregateUtil}
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.datamap.Segment
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable
-import org.apache.carbondata.core.mutate.CarbonUpdateUtil
-import org.apache.carbondata.core.statusmanager.{SegmentStatus, SegmentStatusManager}
-import org.apache.carbondata.core.util.path.CarbonTablePath
-import org.apache.carbondata.events.OperationContext
-import org.apache.carbondata.processing.loading.model.CarbonLoadModel
-import org.apache.carbondata.processing.merger.{CarbonDataMergerUtil, CompactionType}
-
-/**
- * Used to perform compaction on Aggregate data map.
- */
-class AggregateDataMapCompactor(carbonLoadModel: CarbonLoadModel,
-    compactionModel: CompactionModel,
-    executor: ExecutorService,
-    sqlContext: SQLContext,
-    storeLocation: String,
-    operationContext: OperationContext)
-  extends Compactor(carbonLoadModel, compactionModel, executor, sqlContext, storeLocation) {
-
-  override def executeCompaction(): Unit = {
-    val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
-    val loadMetaDataDetails = identifySegmentsToBeMerged()
-    // If segmentFile name is specified in load details then segment is for partition table
-    // therefore the segment file name should be loadName#segmentFileName.segment
-    val segments = loadMetaDataDetails.asScala.map {
-      loadDetail =>
-        new Segment(loadDetail.getLoadName, loadDetail.getSegmentFile, null).toString
-    }
-
-    if (segments.nonEmpty) {
-      val mergedLoadName = CarbonDataMergerUtil.getMergedLoadName(loadMetaDataDetails).split("_")(1)
-      CarbonSession.threadSet(
-        CarbonCommonConstants.CARBON_INPUT_SEGMENTS +
-        carbonLoadModel.getDatabaseName + "." +
-        carbonLoadModel.getTableName,
-        segments.mkString(","))
-      CarbonSession.threadSet(
-        CarbonCommonConstants.VALIDATE_CARBON_INPUT_SEGMENTS +
-        carbonLoadModel.getDatabaseName + "." +
-        carbonLoadModel.getTableName, "false")
-      val loadCommand = operationContext.getProperty(carbonTable.getTableName + "_Compaction")
-        .asInstanceOf[CarbonLoadDataCommand]
-      val uuid = Option(loadCommand.operationContext.getProperty("uuid")).getOrElse("").toString
-      try {
-        val newInternalOptions = loadCommand.internalOptions ++
-                                 Map("mergedSegmentName" -> mergedLoadName)
-        loadCommand.internalOptions = newInternalOptions
-        loadCommand.dataFrame =
-                  Some(PreAggregateUtil.getDataFrame(
-                    sqlContext.sparkSession, loadCommand.logicalPlan.get))
-        CarbonSession.threadSet(CarbonCommonConstants.SUPPORT_DIRECT_QUERY_ON_DATAMAP,
-          "true")
-        loadCommand.processData(sqlContext.sparkSession)
-        // After load is completed for child table the UUID table status will have 0.1 as success
-        // and the table status file will have 0,1,2,3 as Success and 0.1 as In Progress.
-        // Therefore we will read the table status and write 0,1,2,3 as Compacted as the commit
-        // listener will take care of merging the UUID and the table status files.
-        val newMetadataDetails = SegmentStatusManager.readLoadMetadata(
-          carbonTable.getMetadataPath, uuid)
-        val mergedContent = loadMetaDataDetails.asScala.map {
-          segment => segment.setSegmentStatus(SegmentStatus.COMPACTED)
-            segment.setMergedLoadName(mergedLoadName)
-            segment.setModificationOrdeletionTimesStamp(CarbonUpdateUtil.readCurrentTime)
-            segment
-        } ++ newMetadataDetails
-        SegmentStatusManager.writeLoadDetailsIntoFile(
-          CarbonTablePath.getTableStatusFilePathWithUUID(carbonTable.getTablePath, uuid),
-          mergedContent.toArray)
-        carbonLoadModel.setLoadMetadataDetails((carbonLoadModel.getLoadMetadataDetails.asScala ++
-        newMetadataDetails).asJava)
-        // If isCompaction is true then it means that the compaction on aggregate table was
-        // triggered by the maintable thus no need to commit the tablestatus file but if the
-        // compaction was triggered directly for aggregate table then commit has to be fired as
-        // the commit listener would not be called.
-        val directAggregateCompactionCall = Option(operationContext
-          .getProperty("isCompaction")).getOrElse("false").toString.toBoolean
-        if (!directAggregateCompactionCall) {
-          commitAggregateTableStatus(carbonTable, uuid)
-        }
-      } finally {
-        // check if any other segments needs compaction on in case of MINOR_COMPACTION.
-        // For example: after 8.1 creation 0.1, 4.1, 8.1 have to be merged to 0.2 if threshhold
-        // allows it.
-        // Also as the load which will be fired for 2nd level compaction will read the
-        // tablestatus file and not the tablestatus_UUID therefore we have to commit the
-        // intermediate tablestatus file for 2nd level compaction to be successful.
-        // This is required because:
-        //  1. after doing 12 loads and a compaction after every 4 loads the table status file will
-        //     have 0.1, 4.1, 8, 9, 10, 11 as Success segments. While tablestatus_UUID will have
-        //     0.1, 4.1, 8.1.
-        //  2. Now for 2nd level compaction 0.1, 8.1, 4.1 have to be merged to 0.2. therefore we
-        //     need to read the tablestatus_UUID. But load flow should always read tablestatus file
-        //     because it contains the actual In-Process status for the segments.
-        //  3. If we read the tablestatus then 8, 9, 10, 11 will keep getting compacted into 8.1.
-        //  4. Therefore tablestatus file will be committed in between multiple commits.
-        if (!compactionModel.compactionType.equals(CompactionType.MAJOR) &&
-          !compactionModel.compactionType.equals(CompactionType.CUSTOM)) {
-          if (!identifySegmentsToBeMerged().isEmpty) {
-            commitAggregateTableStatus(carbonTable, uuid)
-            executeCompaction()
-          }
-        }
-        CarbonSession
-          .threadUnset(CarbonCommonConstants.CARBON_INPUT_SEGMENTS +
-                       carbonLoadModel.getDatabaseName + "." +
-                       carbonLoadModel.getTableName)
-        CarbonSession.threadUnset(CarbonCommonConstants.VALIDATE_CARBON_INPUT_SEGMENTS +
-                                  carbonLoadModel.getDatabaseName + "." +
-                                  carbonLoadModel.getTableName)
-        LOGGER
-          .info(s"Compaction request for datamap ${ carbonTable.getTableUniqueName } is successful")
-      }
-    }
-  }
-
-  /**
-   * Used to merge the contents of tablestatus and tablestatus_uuid files and write the new
-   * details to tablestatus file. For Example:-
-   * tablestatus contents are = 0(Success), 1(Success),2(Success),3(Success), 0.1(In Progress)
-   * tablestatus_uuid contents are = 0(Compacted), 1(Compacted),2(Compacted),3(Compacted), 0.1
-   * (Success).
-   *
-   * So after merging the tablestatus file will have: 0(Compacted), 1(Compacted),2(Compacted),
-   * 3(Compacted), 0.1(Success).
-   *
-   * NOTE: This method will be called when direct compaction is fired on child aggregate table or
-   * when there are anymore segments to be compacted and the intermediate state of the
-   * tablestatus has to be committed for further compaction to pick other segments.
-   */
-  private def commitAggregateTableStatus(carbonTable: CarbonTable, uuid: String) {
-    if (!CommitPreAggregateListener.mergeTableStatusContents(carbonTable, CarbonTablePath
-      .getTableStatusFilePathWithUUID(carbonTable.getTablePath, uuid), CarbonTablePath
-      .getTableStatusFilePathWithUUID(carbonTable.getTablePath, ""))) {
-      throw new RuntimeException("Unable to acquire lock for table status updation")
-    }
-  }
-
-}
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 71c5a89..93101e9 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
@@ -510,8 +510,7 @@
       // as no record loaded in new segment, new segment should be deleted
       val newEntryLoadStatus =
         if (carbonLoadModel.isCarbonTransactionalTable &&
-            !carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable.isChildDataMap &&
-            !carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable.isChildTable &&
+            !carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable.isChildTableForMV &&
             !CarbonLoaderUtil.isValidSegment(carbonLoadModel, carbonLoadModel.getSegmentId.toInt)) {
           LOGGER.warn("Cannot write load metadata file as there is no data to load")
           SegmentStatus.MARKED_FOR_DELETE
@@ -843,8 +842,7 @@
       operationContext: OperationContext): Unit = {
     LOGGER.info(s"compaction need status is" +
                 s" ${ CarbonDataMergerUtil.checkIfAutoLoadMergingRequired(carbonTable) }")
-    if (!carbonTable.isChildDataMap &&
-        CarbonDataMergerUtil.checkIfAutoLoadMergingRequired(carbonTable)) {
+    if (CarbonDataMergerUtil.checkIfAutoLoadMergingRequired(carbonTable)) {
       val compactionSize = 0
       val isCompactionTriggerByDDl = false
       val compactionModel = CompactionModel(
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CompactionFactory.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CompactionFactory.scala
index c22e3f0..12f0f98 100644
--- a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CompactionFactory.scala
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CompactionFactory.scala
@@ -37,23 +37,13 @@
       storeLocation: String,
       mergedLoads: java.util.List[String],
       operationContext: OperationContext): Compactor = {
-    if (carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable.isChildDataMap) {
-      new AggregateDataMapCompactor(
-        carbonLoadModel,
-        compactionModel,
-        executor,
-        sqlContext,
-        storeLocation,
-        operationContext)
-    } else {
-      new CarbonTableCompactor(
-        carbonLoadModel,
-        compactionModel,
-        executor,
-        sqlContext,
-        storeLocation,
-        mergedLoads,
-        operationContext)
-    }
+    new CarbonTableCompactor(
+      carbonLoadModel,
+      compactionModel,
+      executor,
+      sqlContext,
+      storeLocation,
+      mergedLoads,
+      operationContext)
   }
 }
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/util/CarbonSparkUtil.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/util/CarbonSparkUtil.scala
index 867566b..0aa310a 100644
--- a/integration/spark2/src/main/scala/org/apache/carbondata/spark/util/CarbonSparkUtil.scala
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/spark/util/CarbonSparkUtil.scala
@@ -45,11 +45,11 @@
           f.hasEncoding(Encoding.DICTIONARY) && !f.hasEncoding(Encoding.DIRECT_DICTIONARY) &&
             !f.getDataType.isComplexType)
       }
-    CarbonMetaData(dimensionsAttr,
+    CarbonMetaData(
+      dimensionsAttr,
       measureAttr,
       carbonTable,
-      DictionaryMap(dictionary.toMap),
-      CarbonUtil.hasAggregationDataMap(carbonTable))
+      DictionaryMap(dictionary.toMap))
   }
 
   def createCarbonRelation(tableInfo: TableInfo, tablePath: String): CarbonRelation = {
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDatasourceHadoopRelation.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDatasourceHadoopRelation.scala
index d1b948e..0a1c0bd 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDatasourceHadoopRelation.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDatasourceHadoopRelation.scala
@@ -17,7 +17,6 @@
 
 package org.apache.spark.sql
 
-import scala.collection.JavaConverters._
 import scala.collection.mutable.{ArrayBuffer, ListBuffer}
 import scala.util.control.Breaks._
 
@@ -37,7 +36,6 @@
 import org.apache.carbondata.core.indexstore.PartitionSpec
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension
 import org.apache.carbondata.core.scan.expression.Expression
 import org.apache.carbondata.core.scan.expression.logical.AndExpression
 import org.apache.carbondata.hadoop.CarbonProjection
@@ -80,16 +78,6 @@
 
     val projection = new CarbonProjection
 
-    if (carbonTable.isChildDataMap) {
-      val parentTableIdentifier = carbonTable.getTableInfo.getParentRelationIdentifiers.get(0)
-      val path = CarbonEnv.getCarbonTable(Some(parentTableIdentifier.getDatabaseName),
-        parentTableIdentifier.getTableName)(sparkSession).getTablePath
-      for (carbonDimension: CarbonDimension <- carbonTable.getAllDimensions.asScala) {
-        carbonDimension.getColumnSchema.getParentColumnTableRelations.get(0)
-          .getRelationIdentifier.setTablePath(path)
-      }
-    }
-
     // As Filter pushdown for Complex datatype is not supported, if filter is applied on complex
     // column, then Projection pushdown on Complex Columns will not take effect. Hence, check if
     // filter contains Struct Complex Column.
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonEnv.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonEnv.scala
index 37db135..0a97f96 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonEnv.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonEnv.scala
@@ -27,11 +27,9 @@
 import org.apache.spark.sql.catalyst.catalog.SessionCatalog
 import org.apache.spark.sql.events.{MergeBloomIndexEventListener, MergeIndexEventListener}
 import org.apache.spark.sql.execution.command.cache._
-import org.apache.spark.sql.execution.command.indexserver.PrePrimingEventListener
-import org.apache.spark.sql.execution.command.mv._
-import org.apache.spark.sql.execution.command.preaaggregate._
 import org.apache.spark.sql.execution.command.timeseries.TimeSeriesFunction
 import org.apache.spark.sql.hive._
+import org.apache.spark.sql.listeners.{AlterDataMaptableCompactionPostListener, DataMapAddColumnsPreListener, DataMapAlterTableDropPartitionMetaListener, DataMapAlterTableDropPartitionPreStatusListener, DataMapChangeDataTypeorRenameColumnPreListener, DataMapDeleteSegmentPreListener, DataMapDropColumnPreListener, DropCacheBloomEventListener, DropCacheDataMapEventListener, LoadMVTablePreListener, LoadPostDataMapListener, PrePrimingEventListener, ShowCacheDataMapEventListener, ShowCachePreMVEventListener}
 import org.apache.spark.sql.profiler.Profiler
 
 import org.apache.carbondata.common.logging.LogServiceFactory
@@ -75,13 +73,9 @@
     LOGGER.info(s"Initializing CarbonEnv, store location: $storePath")
 
     sparkSession.udf.register("getTupleId", () => "")
-    // added for handling preaggregate table creation. when user will fire create ddl for
+    // added for handling MV table creation. when user will fire create ddl for
     // create table we are adding a udf so no need to apply PreAggregate rules.
-    sparkSession.udf.register("preAgg", () => "")
-    // added to apply proper rules for loading data into pre-agg table. If this UDF is present
-    // only then the CarbonPreAggregateDataLoadingRules would be applied to split the average
-    // column to sum and count.
-    sparkSession.udf.register("preAggLoad", () => "")
+    sparkSession.udf.register(CarbonEnv.MV_SKIP_RULE_UDF, () => "")
 
     // register for lucene datamap
     // TODO: move it to proper place, it should be registered by datamap implementation
@@ -166,6 +160,8 @@
 
 object CarbonEnv {
 
+  lazy val MV_SKIP_RULE_UDF = "mv"
+
   val carbonEnvMap = new ConcurrentHashMap[SparkSession, CarbonEnv]
 
   val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
@@ -200,32 +196,10 @@
    */
   def initListeners(): Unit = {
     OperationListenerBus.getInstance()
-      .addListener(classOf[LoadTablePreStatusUpdateEvent], LoadPostAggregateListener)
-      .addListener(classOf[DeleteSegmentByIdPreEvent], PreAggregateDeleteSegmentByIdPreListener)
-      .addListener(classOf[DeleteSegmentByDatePreEvent], PreAggregateDeleteSegmentByDatePreListener)
-      .addListener(classOf[UpdateTablePreEvent], UpdatePreAggregatePreListener)
-      .addListener(classOf[DeleteFromTablePreEvent], DeletePreAggregatePreListener)
-      .addListener(classOf[DeleteFromTablePreEvent], DeletePreAggregatePreListener)
       .addListener(classOf[IndexServerLoadEvent], PrePrimingEventListener)
-      .addListener(classOf[AlterTableDropColumnPreEvent], PreAggregateDropColumnPreListener)
-      .addListener(classOf[AlterTableRenamePreEvent], RenameTablePreListener)
-      .addListener(classOf[AlterTableColRenameAndDataTypeChangePreEvent],
-        PreAggregateDataTypeChangePreListener)
-      .addListener(classOf[AlterTableAddColumnPreEvent], PreAggregateAddColumnsPreListener)
-      .addListener(classOf[LoadTablePreExecutionEvent], LoadPreAggregateTablePreListener)
-      .addListener(classOf[AlterTableCompactionPreStatusUpdateEvent],
-        AlterPreAggregateTableCompactionPostListener)
+      .addListener(classOf[LoadTablePreExecutionEvent], LoadMVTablePreListener)
       .addListener(classOf[AlterTableCompactionPreStatusUpdateEvent],
         AlterDataMaptableCompactionPostListener)
-      .addListener(classOf[LoadMetadataEvent], LoadProcessMetaListener)
-      .addListener(classOf[LoadMetadataEvent], CompactionProcessMetaListener)
-      .addListener(classOf[LoadTablePostStatusUpdateEvent], CommitPreAggregateListener)
-      .addListener(classOf[AlterTableCompactionPostStatusUpdateEvent], CommitPreAggregateListener)
-      .addListener(classOf[AlterTableDropPartitionPreStatusEvent],
-        AlterTableDropPartitionPreStatusListener)
-      .addListener(classOf[AlterTableDropPartitionPostStatusEvent],
-        AlterTableDropPartitionPostStatusListener)
-      .addListener(classOf[AlterTableDropPartitionMetaEvent], AlterTableDropPartitionMetaListener)
       .addListener(classOf[LoadTablePreStatusUpdateEvent], new MergeIndexEventListener)
       .addListener(classOf[LoadTablePostExecutionEvent], LoadPostDataMapListener)
       .addListener(classOf[UpdateTablePostEvent], LoadPostDataMapListener )
@@ -234,7 +208,7 @@
       .addListener(classOf[BuildDataMapPostExecutionEvent], new MergeBloomIndexEventListener)
       .addListener(classOf[DropTableCacheEvent], DropCacheDataMapEventListener)
       .addListener(classOf[DropTableCacheEvent], DropCacheBloomEventListener)
-      .addListener(classOf[ShowTableCacheEvent], ShowCachePreAggEventListener)
+      .addListener(classOf[ShowTableCacheEvent], ShowCachePreMVEventListener)
       .addListener(classOf[ShowTableCacheEvent], ShowCacheDataMapEventListener)
       .addListener(classOf[DeleteSegmentByIdPreEvent], DataMapDeleteSegmentPreListener)
       .addListener(classOf[DeleteSegmentByDatePreEvent], DataMapDeleteSegmentPreListener)
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/events/MergeIndexEventListener.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/events/MergeIndexEventListener.scala
index fcee9f1..995c7fd 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/events/MergeIndexEventListener.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/events/MergeIndexEventListener.scala
@@ -86,9 +86,8 @@
                           }")
               val segmentsToMerge =
                 if (alterTableMergeIndexEvent.alterTableModel.customSegmentIds.isEmpty) {
-                  val validSegments = CarbonDataMergerUtil.getValidSegmentList(
-                    carbonMainTable.getAbsoluteTableIdentifier,
-                    carbonMainTable.isChildTable).asScala
+                  val validSegments =
+                    CarbonDataMergerUtil.getValidSegmentList(carbonMainTable).asScala
                   val validSegmentIds: mutable.Buffer[String] = mutable.Buffer[String]()
                   validSegments.foreach { segment =>
                     validSegmentIds += segment.getSegmentNo
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/cache/CacheUtil.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/cache/CacheUtil.scala
index 0565e20..5e369e7 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/cache/CacheUtil.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/cache/CacheUtil.scala
@@ -51,7 +51,7 @@
     if (carbonTable.isTransactionalTable) {
       val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
       val validAndInvalidSegmentsInfo = new SegmentStatusManager(absoluteTableIdentifier)
-        .getValidAndInvalidSegments(carbonTable.isChildTable)
+        .getValidAndInvalidSegments(carbonTable.isChildTableForMV)
       // Fire a job to clear the invalid segments cached in the executors.
       if (CarbonProperties.getInstance().isDistributedPruningEnabled(carbonTable.getDatabaseName,
         carbonTable.getTableName)) {
@@ -110,8 +110,7 @@
   }
 
   def getBloomCacheKeys(carbonTable: CarbonTable, datamap: DataMapSchema): List[String] = {
-    val segments = CarbonDataMergerUtil
-      .getValidSegmentList(carbonTable.getAbsoluteTableIdentifier, carbonTable.isChildTable).asScala
+    val segments = CarbonDataMergerUtil.getValidSegmentList(carbonTable).asScala
 
     // Generate shard Path for the datamap
     val shardPaths = segments.flatMap {
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/cache/CarbonShowCacheCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/cache/CarbonShowCacheCommand.scala
index f293f10..a922c98 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/cache/CarbonShowCacheCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/cache/CarbonShowCacheCommand.scala
@@ -123,7 +123,7 @@
       tableIdent =>
         try {
           val carbonTable = CarbonEnv.getCarbonTable(tableIdent)(sparkSession)
-          if (!carbonTable.isChildDataMap && !carbonTable.isChildTable) {
+          if (!carbonTable.isChildTableForMV) {
             carbonTables += carbonTable
           }
         } catch {
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonCreateDataMapCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonCreateDataMapCommand.scala
index d4524d0..be9dafc 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonCreateDataMapCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonCreateDataMapCommand.scala
@@ -103,21 +103,18 @@
     }
 
     if (null != mainTable) {
-      if (mainTable.isChildTable || mainTable.isChildDataMap) {
+      if (mainTable.isChildTableForMV) {
         throw new MalformedDataMapCommandException(
           "Cannot create DataMap on child table " + mainTable.getTableUniqueName)
       }
     }
-    if (!dataMapSchema.isIndexDataMap && !dataMapSchema.getProviderName
-      .equalsIgnoreCase(DataMapClassProvider.PREAGGREGATE.getShortName) && !dataMapSchema
-      .getProviderName.equalsIgnoreCase(DataMapClassProvider.TIMESERIES.getShortName)) {
+    if (!dataMapSchema.isIndexDataMap) {
       if (DataMapStoreManager.getInstance().getAllDataMapSchemas.asScala
         .exists(_.getDataMapName.equalsIgnoreCase(dataMapSchema.getDataMapName))) {
         if (!ifNotExistsSet) {
           throw new MalformedDataMapCommandException(
             "DataMap with name " + dataMapSchema.getDataMapName + " already exists in storage")
-        }
-        else {
+        } else {
           return Seq.empty
         }
       }
@@ -189,11 +186,7 @@
     if (dataMapProvider != null) {
       dataMapProvider.initData()
       // TODO: remove these checks once the preaggregate and preaggregate timeseries are deprecated
-      if (mainTable != null && !deferredRebuild &&
-          (dataMapSchema.isIndexDataMap || dataMapSchema.getProviderName
-            .equalsIgnoreCase(DataMapClassProvider.PREAGGREGATE.getShortName) ||
-           dataMapSchema.getProviderName
-             .equalsIgnoreCase(DataMapClassProvider.TIMESERIES.getShortName))) {
+      if (mainTable != null && !deferredRebuild && dataMapSchema.isIndexDataMap) {
         dataMapProvider.rebuild()
         if (dataMapSchema.isIndexDataMap) {
           val operationContext: OperationContext = new OperationContext()
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonDataMapRebuildCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonDataMapRebuildCommand.scala
index 345c5a5..ae1290b 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonDataMapRebuildCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonDataMapRebuildCommand.scala
@@ -23,8 +23,6 @@
 
 import org.apache.carbondata.common.exceptions.sql.MalformedDataMapCommandException
 import org.apache.carbondata.core.datamap.status.DataMapStatusManager
-import org.apache.carbondata.core.metadata.schema.datamap.DataMapClassProvider.MV
-import org.apache.carbondata.core.metadata.schema.table.AggregationDataMapSchema
 import org.apache.carbondata.core.util.CarbonProperties
 import org.apache.carbondata.datamap.DataMapManager
 import org.apache.carbondata.events.{UpdateDataMapPostExecutionEvent, _}
@@ -52,8 +50,7 @@
       }
     }
     val schema = schemaOption.get
-    if (!schema.isLazy &&
-        (schema.isIndexDataMap || schema.isInstanceOf[AggregationDataMapSchema])) {
+    if (!schema.isLazy && schema.isIndexDataMap) {
       throw new MalformedDataMapCommandException(
         s"Non-lazy datamap $dataMapName does not support rebuild")
     }
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonDataMapShowCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonDataMapShowCommand.scala
index 85f5068..b23974a 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonDataMapShowCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonDataMapShowCommand.scala
@@ -88,10 +88,7 @@
           val relationIdentifier = s.getRelationIdentifier
           val table = relationIdentifier.getDatabaseName + "." + relationIdentifier.getTableName
           // preaggregate datamap does not support user specified property, therefor we return empty
-          val dmPropertieStr = if (s.getProviderName.equalsIgnoreCase(
-              DataMapClassProvider.PREAGGREGATE.getShortName)) {
-            ""
-          } else {
+          val dmPropertieStr =
             s.getProperties.asScala
               // ignore internal used property
               .filter(p => !p._1.equalsIgnoreCase(DataMapProperty.DEFERRED_REBUILD) &&
@@ -99,54 +96,50 @@
                            !p._1.equalsIgnoreCase(DataMapProperty.QUERY_TYPE))
               .map(p => s"'${ p._1 }'='${ p._2 }'").toSeq
               .sorted.mkString(", ")
-          }
+
           // Get datamap status and sync information details
           var dataMapStatus = "NA"
           var syncInfo: String = "NA"
-          if (!s.getProviderName.equalsIgnoreCase(
-            DataMapClassProvider.PREAGGREGATE.getShortName) && !s.getProviderName.equalsIgnoreCase(
-            DataMapClassProvider.TIMESERIES.getShortName)) {
-            if (DataMapStatusManager.getEnabledDataMapStatusDetails
-              .exists(_.getDataMapName.equalsIgnoreCase(s.getDataMapName))) {
-              dataMapStatus = DataMapStatus.ENABLED.name()
-            } else {
-              dataMapStatus = DataMapStatus.DISABLED.name()
-            }
-            val loadMetadataDetails = SegmentStatusManager
-              .readLoadMetadata(CarbonTablePath
-                .getMetadataPath(s.getRelationIdentifier.getTablePath))
-            if (!s.isIndexDataMap && loadMetadataDetails.nonEmpty) {
-              breakable({
-                for (i <- loadMetadataDetails.length - 1 to 0 by -1) {
-                  if (loadMetadataDetails(i).getSegmentStatus.equals(SegmentStatus.SUCCESS)) {
-                    val segmentMaps =
-                      DataMapSegmentStatusUtil.getSegmentMap(loadMetadataDetails(i).getExtraInfo)
-                    val syncInfoMap = new util.HashMap[String, String]()
-                    val iterator = segmentMaps.entrySet().iterator()
-                    while (iterator.hasNext) {
-                      val entry = iterator.next()
-                      // when in join scenario, one table is loaded and one more is not loaded,
-                      // then put value as NA
-                      if (entry.getValue.isEmpty) {
-                        syncInfoMap.put(entry.getKey, "NA")
-                      } else {
-                        syncInfoMap.put(entry.getKey, DataMapUtil.getMaxSegmentID(entry.getValue))
-                      }
+          if (DataMapStatusManager.getEnabledDataMapStatusDetails
+            .exists(_.getDataMapName.equalsIgnoreCase(s.getDataMapName))) {
+            dataMapStatus = DataMapStatus.ENABLED.name()
+          } else {
+            dataMapStatus = DataMapStatus.DISABLED.name()
+          }
+          val loadMetadataDetails = SegmentStatusManager
+            .readLoadMetadata(CarbonTablePath
+              .getMetadataPath(s.getRelationIdentifier.getTablePath))
+          if (!s.isIndexDataMap && loadMetadataDetails.nonEmpty) {
+            breakable({
+              for (i <- loadMetadataDetails.length - 1 to 0 by -1) {
+                if (loadMetadataDetails(i).getSegmentStatus.equals(SegmentStatus.SUCCESS)) {
+                  val segmentMaps =
+                    DataMapSegmentStatusUtil.getSegmentMap(loadMetadataDetails(i).getExtraInfo)
+                  val syncInfoMap = new util.HashMap[String, String]()
+                  val iterator = segmentMaps.entrySet().iterator()
+                  while (iterator.hasNext) {
+                    val entry = iterator.next()
+                    // when in join scenario, one table is loaded and one more is not loaded,
+                    // then put value as NA
+                    if (entry.getValue.isEmpty) {
+                      syncInfoMap.put(entry.getKey, "NA")
+                    } else {
+                      syncInfoMap.put(entry.getKey, DataMapUtil.getMaxSegmentID(entry.getValue))
                     }
-                    val loadEndTime =
-                      if (loadMetadataDetails(i).getLoadEndTime ==
-                          CarbonCommonConstants.SEGMENT_LOAD_TIME_DEFAULT) {
-                        "NA"
-                      } else {
-                        new java.sql.Timestamp(loadMetadataDetails(i).getLoadEndTime).toString
-                      }
-                    syncInfoMap.put(CarbonCommonConstants.LOAD_SYNC_TIME, loadEndTime)
-                    syncInfo = new Gson().toJson(syncInfoMap)
-                    break()
                   }
+                  val loadEndTime =
+                    if (loadMetadataDetails(i).getLoadEndTime ==
+                        CarbonCommonConstants.SEGMENT_LOAD_TIME_DEFAULT) {
+                      "NA"
+                    } else {
+                      new java.sql.Timestamp(loadMetadataDetails(i).getLoadEndTime).toString
+                    }
+                  syncInfoMap.put(CarbonCommonConstants.LOAD_SYNC_TIME, loadEndTime)
+                  syncInfo = new Gson().toJson(syncInfoMap)
+                  break()
                 }
-              })
-            }
+              }
+            })
           }
           Row(s.getDataMapName, s.getProviderName, table, dmPropertieStr, dataMapStatus, syncInfo)
       }
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonDropDataMapCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonDropDataMapCommand.scala
index 1fa1337..ded87b9 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonDropDataMapCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonDropDataMapCommand.scala
@@ -24,7 +24,6 @@
 import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.catalyst.analysis.NoSuchTableException
 import org.apache.spark.sql.execution.command.AtomicRunnableCommand
-import org.apache.spark.sql.execution.command.preaaggregate.PreAggregateUtil
 import org.apache.spark.sql.execution.command.table.CarbonDropTableCommand
 
 import org.apache.carbondata.common.exceptions.sql.NoSuchDataMapException
@@ -33,7 +32,6 @@
 import org.apache.carbondata.core.datamap.status.DataMapStatusManager
 import org.apache.carbondata.core.locks.{CarbonLockUtil, ICarbonLock, LockUsage}
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
-import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl
 import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, DataMapSchema}
 import org.apache.carbondata.core.util.CarbonProperties
 import org.apache.carbondata.datamap.{DataMapManager, IndexDataMapProvider}
@@ -132,48 +130,7 @@
           }
         }
 
-        // drop preaggregate datamap.
-        // If datamap to be dropped in parent table then drop the datamap from metastore and remove
-        // entry from parent table.
-        // If force drop is true then remove the datamap from hivemetastore. No need to remove from
-        // parent as the first condition would have taken care of it.
-        if (mainTable != null && mainTable.getTableInfo.getDataMapSchemaList.size() > 0) {
-          val dataMapSchemaOp = mainTable.getTableInfo.getDataMapSchemaList.asScala.zipWithIndex.
-            find(_._1.getDataMapName.equalsIgnoreCase(dataMapName))
-          if (dataMapSchemaOp.isDefined) {
-            dataMapSchema = dataMapSchemaOp.get._1
-            val operationContext = new OperationContext
-            val dropDataMapPreEvent =
-              DropDataMapPreEvent(
-                Some(dataMapSchema),
-                ifExistsSet,
-                sparkSession)
-            OperationListenerBus.getInstance.fireEvent(dropDataMapPreEvent, operationContext)
-            mainTable.getTableInfo.getDataMapSchemaList.remove(dataMapSchemaOp.get._2)
-            val schemaConverter = new ThriftWrapperSchemaConverterImpl
-            PreAggregateUtil.updateSchemaInfo(
-              mainTable,
-              schemaConverter.fromWrapperToExternalTableInfo(
-                mainTable.getTableInfo,
-                dbName,
-                tableName))(sparkSession)
-            if (dataMapProvider == null) {
-              dataMapProvider =
-                DataMapManager.get.getDataMapProvider(mainTable, dataMapSchema, sparkSession)
-            }
-            dataMapProvider.cleanMeta()
-
-            // fires the event after dropping datamap from main table schema
-            val dropDataMapPostEvent =
-              DropDataMapPostEvent(
-                Some(dataMapSchema),
-                ifExistsSet,
-                sparkSession)
-            OperationListenerBus.getInstance.fireEvent(dropDataMapPostEvent, operationContext)
-          } else if (!ifExistsSet) {
-            throw new NoSuchDataMapException(dataMapName, tableName)
-          }
-        } else if (!ifExistsSet) {
+        if (!ifExistsSet) {
           throw new NoSuchDataMapException(dataMapName)
         }
       } catch {
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAddLoadCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAddLoadCommand.scala
index 7a40566..f90c303 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAddLoadCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAddLoadCommand.scala
@@ -85,8 +85,8 @@
       throw new MalformedCarbonCommandException(
         "Unsupported operation on global dictionary columns table")
     }
-    if (carbonTable.isChildTable || carbonTable.isChildDataMap) {
-      throw new MalformedCarbonCommandException("Unsupported operation on MV/Pre-aggrergated table")
+    if (carbonTable.isChildTableForMV) {
+      throw new MalformedCarbonCommandException("Unsupported operation on MV table")
     }
     // if insert overwrite in progress, do not allow add segment
     if (SegmentStatusManager.isOverwriteInProgressInTable(carbonTable)) {
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala
index 0a0ce19..5a55570 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala
@@ -89,19 +89,6 @@
     if (!table.getTableInfo.isTransactionalTable) {
       throw new MalformedCarbonCommandException("Unsupported operation on non transactional table")
     }
-    if (CarbonUtil.hasAggregationDataMap(table) ||
-        (table.isChildDataMap && null == operationContext.getProperty(table.getTableName))) {
-      // If the compaction request is of 'streaming' type then we need to generate loadCommands
-      // for all the child datamaps in the LoadMetadataEvent. Therefore setting isCompaction=false.
-      // If set to true then only loadCommands for compaction will be created.
-      val loadMetadataEvent =
-        if (alterTableModel.compactionType.equalsIgnoreCase(CompactionType.STREAMING.name())) {
-          new LoadMetadataEvent(table, false, Map.empty[String, String].asJava)
-        } else {
-          new LoadMetadataEvent(table, true, Map.empty[String, String].asJava)
-        }
-      OperationListenerBus.getInstance().fireEvent(loadMetadataEvent, operationContext)
-    }
     Seq.empty
   }
 
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonCleanFilesCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonCleanFilesCommand.scala
index a2a9d3c..019d760 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonCleanFilesCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonCleanFilesCommand.scala
@@ -63,16 +63,7 @@
     setAuditInfo(Map(
       "force" -> forceTableClean.toString,
       "internal" -> isInternalCleanCall.toString))
-    if (carbonTable.hasAggregationDataMap) {
-      cleanFileCommands = carbonTable.getTableInfo.getDataMapSchemaList.asScala.map {
-        dataMapSchema =>
-          val relationIdentifier = dataMapSchema.getRelationIdentifier
-          CarbonCleanFilesCommand(
-            Some(relationIdentifier.getDatabaseName), Some(relationIdentifier.getTableName),
-            isInternalCleanCall = true)
-      }.toList
-      cleanFileCommands.foreach(_.processMetadata(sparkSession))
-    } else if (DataMapUtil.hasMVDataMap(carbonTable)) {
+    if (DataMapUtil.hasMVDataMap(carbonTable)) {
       val allDataMapSchemas = DataMapStoreManager.getInstance
         .getDataMapSchemasOfTable(carbonTable).asScala
         .filter(dataMapSchema => null != dataMapSchema.getRelationIdentifier &&
@@ -85,10 +76,6 @@
             isInternalCleanCall = true)
       }.toList
       cleanFileCommands.foreach(_.processMetadata(sparkSession))
-    } else if (carbonTable.isChildDataMap && !isInternalCleanCall) {
-      throwMetadataException(
-        carbonTable.getDatabaseName, carbonTable.getTableName,
-        "Cannot clean files directly for aggregate table.")
     }
     Seq.empty
   }
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonInsertFromStageCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonInsertFromStageCommand.scala
index 0c665a8..5d7359d 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonInsertFromStageCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonInsertFromStageCommand.scala
@@ -70,8 +70,8 @@
     if (!table.getTableInfo.isTransactionalTable) {
       throw new MalformedCarbonCommandException("Unsupported operation on non transactional table")
     }
-    if (table.isChildDataMap || table.isChildTable) {
-      throw new MalformedCarbonCommandException("Unsupported operation on child table")
+    if (table.isChildTableForMV) {
+      throw new MalformedCarbonCommandException("Unsupported operation on MV table")
     }
 
     val tablePath = table.getTablePath
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
index bb691f7..382ed12 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
@@ -135,17 +135,6 @@
         }.head
       sizeInBytes = logicalPartitionRelation.relation.sizeInBytes
     }
-    if (table.isChildDataMap) {
-      val parentTableIdentifier = table.getTableInfo.getParentRelationIdentifiers.get(0)
-      parentTablePath = CarbonEnv
-        .getCarbonTable(Some(parentTableIdentifier.getDatabaseName),
-          parentTableIdentifier.getTableName)(sparkSession).getTablePath
-    }
-    operationContext.setProperty("isOverwrite", isOverwriteTable)
-    if(CarbonUtil.hasAggregationDataMap(table)) {
-      val loadMetadataEvent = new LoadMetadataEvent(table, false, options.asJava)
-      OperationListenerBus.getInstance().fireEvent(loadMetadataEvent, operationContext)
-    }
     Seq.empty
   }
 
@@ -270,16 +259,7 @@
     LOGGER.info(s"Deleting stale folders if present for table $dbName.$tableName")
     TableProcessingOperations.deletePartialLoadDataIfExist(table, false)
     var isUpdateTableStatusRequired = false
-    // if the table is child then extract the uuid from the operation context and the parent would
-    // already generated UUID.
-    // if parent table then generate a new UUID else use empty.
-    val uuid = if (table.isChildDataMap) {
-      Option(operationContext.getProperty("uuid")).getOrElse("").toString
-    } else if (table.hasAggregationDataMap) {
-      UUID.randomUUID().toString
-    } else {
-      ""
-    }
+    val uuid = ""
     try {
       operationContext.setProperty("uuid", uuid)
       val loadTablePreExecutionEvent: LoadTablePreExecutionEvent =
@@ -390,7 +370,7 @@
       case ex: Exception =>
         LOGGER.error(ex)
         // update the load entry in table status file for changing the status to marked for delete
-        if (isUpdateTableStatusRequired && !table.isChildDataMap) {
+        if (isUpdateTableStatusRequired) {
           CarbonLoaderUtil.updateTableStatusForFailure(carbonLoadModel, uuid)
         }
         throw ex
@@ -963,8 +943,7 @@
     }
     // Rearrange the partition column at the end of output list
     if (catalogTable.partitionColumnNames.nonEmpty &&
-        (loadModel.getCarbonDataLoadSchema.getCarbonTable.isChildTable ||
-         loadModel.getCarbonDataLoadSchema.getCarbonTable.isChildDataMap) && output.nonEmpty) {
+        (loadModel.getCarbonDataLoadSchema.getCarbonTable.isChildTableForMV) && output.nonEmpty) {
       val partitionOutPut =
         catalogTable.partitionColumnNames.map(col => output.find(_.name.equalsIgnoreCase(col)).get)
       output = output.filterNot(partitionOutPut.contains(_)) ++ partitionOutPut
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/HorizontalCompaction.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/HorizontalCompaction.scala
index 62a3486..931f065 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/HorizontalCompaction.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/HorizontalCompaction.scala
@@ -63,8 +63,7 @@
     val deleteTimeStamp = updateTimeStamp + 1
 
     // get the valid segments
-    var segLists = CarbonDataMergerUtil
-      .getValidSegmentList(absTableIdentifier, carbonTable.isChildTable)
+    var segLists = CarbonDataMergerUtil.getValidSegmentList(carbonTable)
 
     if (segLists == null || segLists.size() == 0) {
       return
@@ -92,8 +91,7 @@
 
     // After Update Compaction perform delete compaction
     compactionTypeIUD = CompactionType.IUD_DELETE_DELTA
-    segLists = CarbonDataMergerUtil
-      .getValidSegmentList(absTableIdentifier, carbonTable.isChildTable)
+    segLists = CarbonDataMergerUtil.getValidSegmentList(carbonTable)
     if (segLists == null || segLists.size() == 0) {
       return
     }
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableAddHivePartitionCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableAddHivePartitionCommand.scala
index f29b7b2..806d0f9 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableAddHivePartitionCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableAddHivePartitionCommand.scala
@@ -58,7 +58,7 @@
     setAuditTable(table)
     setAuditInfo(Map("partition" -> partitionSpecsAndLocs.mkString(", ")))
     if (table.isHivePartitionTable) {
-      if (table.isChildDataMap || table.isChildTable) {
+      if (table.isChildTableForMV) {
         throw new UnsupportedOperationException("Cannot add partition directly on child tables")
       }
       val partitionWithLoc = partitionSpecsAndLocs.filter(_._2.isDefined)
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableDropHivePartitionCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableDropHivePartitionCommand.scala
index d6abf86..302a52e 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableDropHivePartitionCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableDropHivePartitionCommand.scala
@@ -154,26 +154,10 @@
         table.getDatabaseName,
         table.getTableName,
         locksToBeAcquired)(sparkSession)
-      // If flow is for child table then get the uuid from operation context.
-      // If flow is for parent table then generate uuid for child flows and set the uuid to ""
-      // for parent table
-      // If normal table then set uuid to "".
-      val uuid = if (table.isChildDataMap) {
-        val uuid = operationContext.getProperty("uuid")
-        if (uuid != null) {
-          uuid.toString
-        } else {
-          LOGGER.warn(s"UUID not set for table ${table.getTableUniqueName} in operation context.")
-          ""
-        }
-      } else if (table.hasAggregationDataMap) {
-        operationContext.setProperty("uuid", UUID.randomUUID().toString)
-        ""
-      } else {
-        ""
-      }
+      // If normal table then set uuid to ""
+      val uuid = "";
       val segments = new SegmentStatusManager(table.getAbsoluteTableIdentifier)
-        .getValidAndInvalidSegments(table.isChildTable).getValidSegments
+        .getValidAndInvalidSegments(table.isChildTableForMV).getValidSegments
       // First drop the partitions from partition mapper files of each segment
       val tuples = new CarbonDropPartitionRDD(sparkSession,
         table.getTablePath,
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateListeners.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateListeners.scala
deleted file mode 100644
index adf89cd..0000000
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateListeners.scala
+++ /dev/null
@@ -1,932 +0,0 @@
-  /*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.execution.command.preaaggregate
-
-import java.util.UUID
-
-import scala.collection.JavaConverters._
-import scala.collection.mutable
-
-import org.apache.spark.sql.SparkSession
-import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.execution.command.AlterTableModel
-import org.apache.spark.sql.execution.command.management.{CarbonAlterTableCompactionCommand, CarbonLoadDataCommand}
-import org.apache.spark.sql.execution.command.partition.CarbonAlterTableDropHivePartitionCommand
-import org.apache.spark.sql.parser.CarbonSpark2SqlParser
-
-import org.apache.carbondata.common.exceptions.MetadataProcessException
-import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.datamap.Segment
-import org.apache.carbondata.core.datastore.filesystem.{CarbonFile, CarbonFileFilter}
-import org.apache.carbondata.core.datastore.impl.FileFactory
-import org.apache.carbondata.core.locks.{CarbonLockUtil, ICarbonLock}
-import org.apache.carbondata.core.metadata.schema.table.{AggregationDataMapSchema, CarbonTable}
-import org.apache.carbondata.core.statusmanager.{SegmentStatus, SegmentStatusManager}
-import org.apache.carbondata.core.util.CarbonUtil
-import org.apache.carbondata.core.util.path.CarbonTablePath
-import org.apache.carbondata.events._
-import org.apache.carbondata.processing.loading.events.LoadEvents.{LoadMetadataEvent, LoadTablePostExecutionEvent, LoadTablePostStatusUpdateEvent, LoadTablePreExecutionEvent, LoadTablePreStatusUpdateEvent}
-import org.apache.carbondata.processing.merger.CompactionType
-
-object AlterTableDropPartitionPreStatusListener extends OperationEventListener {
-  /**
-   * Called on a specified event occurrence
-   *
-   * @param event
-   * @param operationContext
-   */
-  override protected def onEvent(event: Event,
-      operationContext: OperationContext) = {
-    val preStatusListener = event.asInstanceOf[AlterTableDropPartitionPreStatusEvent]
-    val carbonTable = preStatusListener.carbonTable
-    val childDropPartitionCommands = operationContext.getProperty("dropPartitionCommands")
-    if (childDropPartitionCommands != null && carbonTable.hasAggregationDataMap) {
-      val childCommands =
-        childDropPartitionCommands.asInstanceOf[Seq[CarbonAlterTableDropHivePartitionCommand]]
-      childCommands.foreach(_.processData(SparkSession.getActiveSession.get))
-    }
-  }
-}
-
-trait CommitHelper {
-
-  val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
-
-  protected def markInProgressSegmentAsDeleted(tableStatusFile: String,
-      operationContext: OperationContext,
-      carbonTable: CarbonTable): Unit = {
-    val lockFile: ICarbonLock = new SegmentStatusManager(carbonTable
-      .getAbsoluteTableIdentifier).getTableStatusLock
-    val retryCount = CarbonLockUtil
-      .getLockProperty(CarbonCommonConstants.NUMBER_OF_TRIES_FOR_CONCURRENT_LOCK,
-        CarbonCommonConstants.NUMBER_OF_TRIES_FOR_CONCURRENT_LOCK_DEFAULT)
-    val maxTimeout = CarbonLockUtil
-      .getLockProperty(CarbonCommonConstants.MAX_TIMEOUT_FOR_CONCURRENT_LOCK,
-        CarbonCommonConstants.MAX_TIMEOUT_FOR_CONCURRENT_LOCK_DEFAULT)
-    try {
-      if (lockFile.lockWithRetries(retryCount, maxTimeout)) {
-        val loadMetaDataDetails = SegmentStatusManager.readTableStatusFile(tableStatusFile)
-        val segmentBeingLoaded =
-          operationContext.getProperty(carbonTable.getTableUniqueName + "_Segment").toString
-        val newDetails = loadMetaDataDetails.collect {
-          case detail if detail.getLoadName.equalsIgnoreCase(segmentBeingLoaded) =>
-            detail.setSegmentStatus(SegmentStatus.MARKED_FOR_DELETE)
-            detail
-          case others => others
-        }
-        SegmentStatusManager.writeLoadDetailsIntoFile(tableStatusFile, newDetails)
-      } else {
-        throw new RuntimeException("Uable to update table status file")
-      }
-    } finally {
-      lockFile.unlock()
-    }
-  }
-
-  /**
-   *  Used to rename table status files for commit operation.
-   */
-  protected def renameDataMapTableStatusFiles(sourceFileName: String,
-      destinationFileName: String, uuid: String): Boolean = {
-    val oldCarbonFile = FileFactory.getCarbonFile(sourceFileName)
-    val newCarbonFile = FileFactory.getCarbonFile(destinationFileName)
-    if (oldCarbonFile.exists() && newCarbonFile.exists()) {
-      val backUpPostFix = if (uuid.nonEmpty) {
-        "_backup_" + uuid
-      } else {
-        ""
-      }
-      LOGGER.info(s"Renaming $newCarbonFile to ${destinationFileName + backUpPostFix}")
-      if (newCarbonFile.renameForce(destinationFileName + backUpPostFix)) {
-        LOGGER.info(s"Renaming $oldCarbonFile to $destinationFileName")
-        oldCarbonFile.renameForce(destinationFileName)
-      } else {
-        LOGGER.info(s"Renaming $newCarbonFile to ${destinationFileName + backUpPostFix} failed")
-        false
-      }
-    } else {
-     /**
-      * Tablestatus_uuid will fail when Pre-Aggregate table is not valid for compaction.
-      * Hence this should return true
-      */
-      true
-    }
-  }
-
-  def mergeTableStatusContents(carbonTable: CarbonTable, uuidTableStatusPath: String,
-      tableStatusPath: String): Boolean = {
-    val lockFile: ICarbonLock = new SegmentStatusManager(carbonTable
-      .getAbsoluteTableIdentifier).getTableStatusLock
-    try {
-      val retryCount = CarbonLockUtil
-        .getLockProperty(CarbonCommonConstants.NUMBER_OF_TRIES_FOR_CONCURRENT_LOCK,
-          CarbonCommonConstants.NUMBER_OF_TRIES_FOR_CONCURRENT_LOCK_DEFAULT)
-      val maxTimeout = CarbonLockUtil
-        .getLockProperty(CarbonCommonConstants.MAX_TIMEOUT_FOR_CONCURRENT_LOCK,
-          CarbonCommonConstants.MAX_TIMEOUT_FOR_CONCURRENT_LOCK_DEFAULT)
-      if (lockFile.lockWithRetries(retryCount, maxTimeout)) {
-        val tableStatusContents = SegmentStatusManager.readTableStatusFile(tableStatusPath)
-        val newLoadContent = SegmentStatusManager.readTableStatusFile(uuidTableStatusPath)
-        val mergedContent = tableStatusContents.collect {
-          case content =>
-            val contentIndex = newLoadContent.indexOf(content)
-            if (contentIndex == -1) {
-              content
-            } else {
-              newLoadContent(contentIndex)
-            }
-        }
-        SegmentStatusManager.writeLoadDetailsIntoFile(tableStatusPath, mergedContent)
-        true
-      } else {
-        false
-      }
-    } catch {
-      case ex: Exception =>
-        LOGGER.error("Exception occurred while merging files", ex)
-        false
-    } finally {
-      lockFile.unlock()
-    }
-  }
-
-  /**
-   * Used to remove table status files with UUID and segment folders.
-   */
-  protected def cleanUpStaleTableStatusFiles(
-      childTables: Seq[CarbonTable],
-      operationContext: OperationContext,
-      uuid: String): Unit = {
-    childTables.foreach { childTable =>
-      val metaDataDir = FileFactory.getCarbonFile(
-        CarbonTablePath.getMetadataPath(childTable.getTablePath))
-      val tableStatusFiles = metaDataDir.listFiles(new CarbonFileFilter {
-        override def accept(file: CarbonFile): Boolean = {
-          file.getName.contains(uuid) || file.getName.contains("backup")
-        }
-      })
-      tableStatusFiles.foreach(_.delete())
-    }
-  }
-}
-
-object AlterTableDropPartitionPostStatusListener extends OperationEventListener with CommitHelper {
-  /**
-   * Called on a specified event occurrence
-   *
-   * @param event
-   * @param operationContext
-   */
-  override protected def onEvent(event: Event,
-      operationContext: OperationContext) = {
-    val postStatusListener = event.asInstanceOf[AlterTableDropPartitionPostStatusEvent]
-    val carbonTable = postStatusListener.carbonTable
-    val childDropPartitionCommands = operationContext.getProperty("dropPartitionCommands")
-    val uuid = Option(operationContext.getProperty("uuid")).getOrElse("").toString
-    if (childDropPartitionCommands != null && carbonTable.hasAggregationDataMap) {
-      val childCommands =
-        childDropPartitionCommands.asInstanceOf[Seq[CarbonAlterTableDropHivePartitionCommand]]
-      val updateFailed = try {
-        val renamedDataMaps = childCommands.takeWhile {
-          childCommand =>
-            val childCarbonTable = childCommand.table
-            val oldTableSchemaPath = CarbonTablePath.getTableStatusFilePathWithUUID(
-              childCarbonTable.getTablePath, uuid)
-            // Generate table status file name without UUID, forExample: tablestatus
-            val newTableSchemaPath = CarbonTablePath.getTableStatusFilePath(
-              childCarbonTable.getTablePath)
-            mergeTableStatusContents(childCarbonTable, oldTableSchemaPath, newTableSchemaPath)
-        }
-        // if true then the commit for one of the child tables has failed
-        val commitFailed = renamedDataMaps.lengthCompare(childCommands.length) != 0
-        if (commitFailed) {
-          LOGGER.warn("Reverting table status file to original state")
-          childCommands.foreach {
-            childDropCommand =>
-              val tableStatusPath = CarbonTablePath.getTableStatusFilePath(
-                childDropCommand.table.getTablePath)
-              markInProgressSegmentAsDeleted(tableStatusPath,
-                operationContext,
-                childDropCommand.table)
-          }
-        }
-        commitFailed
-      } finally {
-        // after success/failure of commit delete all tablestatus files with UUID in their names.
-        // if commit failed then remove the segment directory
-        // TODO Need to handle deletion on tablestatus files with UUID in cleanup command.
-        cleanUpStaleTableStatusFiles(childCommands.map(_.table),
-          operationContext,
-          uuid)
-      }
-      if (updateFailed) {
-        sys.error("Failed to update table status for pre-aggregate table")
-      }
-    }
-  }
-}
-
-object AlterTableDropPartitionMetaListener extends OperationEventListener{
-  /**
-   * Called on a specified event occurrence
-   *
-   * @param event
-   * @param operationContext
-   */
-  override protected def onEvent(event: Event,
-      operationContext: OperationContext) = {
-    val dropPartitionEvent = event.asInstanceOf[AlterTableDropPartitionMetaEvent]
-    val parentCarbonTable = dropPartitionEvent.parentCarbonTable
-    val partitionsToBeDropped = dropPartitionEvent.specs.flatMap(_.keys)
-    if (parentCarbonTable.hasAggregationDataMap) {
-      // used as a flag to block direct drop partition on aggregate tables fired by the user
-      operationContext.setProperty("isInternalDropCall", "true")
-      // Filter out all the tables which don't have the partition being dropped.
-      val childTablesWithoutPartitionColumns =
-        parentCarbonTable.getTableInfo.getDataMapSchemaList.asScala.filter { dataMapSchema =>
-          val childColumns = dataMapSchema.getChildSchema.getListOfColumns.asScala
-          val partitionColExists = partitionsToBeDropped.forall {
-            partition =>
-              childColumns.exists { childColumn =>
-                  childColumn.getAggFunction.isEmpty &&
-                    childColumn.getParentColumnTableRelations != null &&
-                  childColumn.getParentColumnTableRelations.asScala.head.getColumnName.
-                    equals(partition)
-              }
-          }
-          !partitionColExists
-      }
-      if (childTablesWithoutPartitionColumns.nonEmpty) {
-        throw new MetadataProcessException(s"Cannot drop partition as one of the partition is not" +
-                                           s" participating in the following datamaps ${
-          childTablesWithoutPartitionColumns.toList.map(_.getChildSchema.getTableName)
-        }. Please drop the specified aggregate tables to continue")
-      } else {
-        val childDropPartitionCommands =
-          parentCarbonTable.getTableInfo.getDataMapSchemaList.asScala.map { dataMapSchema =>
-            val tableIdentifier = TableIdentifier(dataMapSchema.getChildSchema.getTableName,
-              Some(parentCarbonTable.getDatabaseName))
-            // as the aggregate table columns start with parent table name therefore the
-            // partition column also has to be updated with parent table name to generate
-            // partitionSpecs for the child table.
-            val childSpecs = dropPartitionEvent.specs.map {
-               spec => spec.map {
-                case (key, value) => (s"${parentCarbonTable.getTableName}_$key", value)
-              }
-            }
-            CarbonAlterTableDropHivePartitionCommand(
-              tableIdentifier,
-              childSpecs,
-              dropPartitionEvent.ifExists,
-              dropPartitionEvent.purge,
-              dropPartitionEvent.retainData,
-              operationContext)
-        }
-        operationContext.setProperty("dropPartitionCommands", childDropPartitionCommands)
-        childDropPartitionCommands.foreach(_.processMetadata(SparkSession.getActiveSession.get))
-
-      }
-    } else if (parentCarbonTable.isChildDataMap) {
-      if (operationContext.getProperty("isInternalDropCall") == null) {
-        throw new UnsupportedOperationException("Cannot drop partition directly on aggregate table")
-      }
-    }
-  }
-}
-
-/**
- * below class will be used to create load command for compaction
- * for all the pre agregate child data map
- */
-object CompactionProcessMetaListener extends OperationEventListener {
-  /**
-   * Called on a specified event occurrence
-   *
-   * @param event
-   * @param operationContext
-   */
-  override protected def onEvent(event: Event,
-      operationContext: OperationContext): Unit = {
-    val sparkSession = SparkSession.getActiveSession.get
-    val tableEvent = event.asInstanceOf[LoadMetadataEvent]
-    val table = tableEvent.getCarbonTable
-    if (!table.isChildDataMap && CarbonUtil.hasAggregationDataMap(table)) {
-      val aggregationDataMapList = table.getTableInfo.getDataMapSchemaList.asScala
-        .filter(_.isInstanceOf[AggregationDataMapSchema])
-        .asInstanceOf[mutable.ArrayBuffer[AggregationDataMapSchema]]
-      for (dataMapSchema: AggregationDataMapSchema <- aggregationDataMapList) {
-        val childTableName = dataMapSchema.getRelationIdentifier.getTableName
-        val childDatabaseName = dataMapSchema.getRelationIdentifier.getDatabaseName
-        // Creating a new query string to insert data into pre-aggregate table from that same table.
-        // For example: To compact preaggtable1 we can fire a query like insert into preaggtable1
-        // select * from preaggtable1
-        // The following code will generate the select query with a load UDF that will be used to
-        // apply DataLoadingRules
-        val childDataFrame = sparkSession.sql(new CarbonSpark2SqlParser()
-          // adding the aggregation load UDF
-          .addPreAggLoadFunction(
-          // creating the select query on the bases on table schema
-          PreAggregateUtil.createChildSelectQuery(
-            dataMapSchema.getChildSchema, table.getDatabaseName))).drop("preAggLoad")
-        val loadCommand = PreAggregateUtil.createLoadCommandForChild(
-          dataMapSchema.getChildSchema.getListOfColumns,
-          TableIdentifier(childTableName, Some(childDatabaseName)),
-          childDataFrame,
-          false,
-          sparkSession,
-          mutable.Map.empty[String, String])
-        val uuid = Option(operationContext.getProperty("uuid")).
-          getOrElse(UUID.randomUUID()).toString
-        operationContext.setProperty("uuid", uuid)
-        loadCommand.processMetadata(sparkSession)
-        operationContext
-          .setProperty(dataMapSchema.getChildSchema.getTableName + "_Compaction", loadCommand)
-        loadCommand.operationContext = operationContext
-      }
-    } else if (table.isChildDataMap) {
-      val childTableName = table.getTableName
-      val childDatabaseName = table.getDatabaseName
-      // Creating a new query string to insert data into pre-aggregate table from that same table.
-      // For example: To compact preaggtable1 we can fire a query like insert into preaggtable1
-      // select * from preaggtable1
-      // The following code will generate the select query with a load UDF that will be used to
-      // apply DataLoadingRules
-      val childDataFrame = sparkSession.sql(new CarbonSpark2SqlParser()
-        // adding the aggregation load UDF
-        .addPreAggLoadFunction(
-        // creating the select query on the bases on table schema
-        PreAggregateUtil.createChildSelectQuery(
-          table.getTableInfo.getFactTable, table.getDatabaseName))).drop("preAggLoad")
-      val loadCommand = PreAggregateUtil.createLoadCommandForChild(
-        table.getTableInfo.getFactTable.getListOfColumns,
-        TableIdentifier(childTableName, Some(childDatabaseName)),
-        childDataFrame,
-        false,
-        sparkSession,
-        mutable.Map.empty[String, String])
-      val uuid = Option(operationContext.getProperty("uuid")).getOrElse(UUID.randomUUID()).toString
-      loadCommand.processMetadata(sparkSession)
-      operationContext.setProperty(table.getTableName + "_Compaction", loadCommand)
-      operationContext.setProperty("uuid", uuid)
-      loadCommand.operationContext = operationContext
-    }
-
-  }
-}
-
-/**
- * Below class to is to create LoadCommand for loading the
- * the data of pre aggregate data map
- */
-object LoadProcessMetaListener extends OperationEventListener {
-  /**
-   * Called on a specified event occurrence
-   *
-   * @param event
-   * @param operationContext
-   */
-  override protected def onEvent(event: Event,
-      operationContext: OperationContext): Unit = {
-    val sparkSession = SparkSession.getActiveSession.get
-    val tableEvent = event.asInstanceOf[LoadMetadataEvent]
-    if (!tableEvent.isCompaction) {
-      val table = tableEvent.getCarbonTable
-      if (CarbonUtil.hasAggregationDataMap(table)) {
-        // getting all the aggergate datamap schema
-        val aggregationDataMapList = table.getTableInfo.getDataMapSchemaList.asScala
-          .filter(_.isInstanceOf[AggregationDataMapSchema])
-          .asInstanceOf[mutable.ArrayBuffer[AggregationDataMapSchema]]
-        // sorting the datamap for timeseries rollup
-        val sortedList = aggregationDataMapList.sortBy(_.getOrdinal)
-        val parentTableName = table.getTableName
-        val databaseName = table.getDatabaseName
-        // if the table is child then extract the uuid from the operation context and the parent
-        // would already generated UUID.
-        // if parent table then generate a new UUID else use empty.
-        val uuid =
-          Option(operationContext.getProperty("uuid")).getOrElse(UUID.randomUUID()).toString
-        val list = scala.collection.mutable.ListBuffer.empty[AggregationDataMapSchema]
-        for (dataMapSchema: AggregationDataMapSchema <- sortedList) {
-          val childTableName = dataMapSchema.getRelationIdentifier.getTableName
-          val childDatabaseName = dataMapSchema.getRelationIdentifier.getDatabaseName
-          val childSelectQuery = if (!dataMapSchema.isTimeseriesDataMap) {
-            (PreAggregateUtil.getChildQuery(dataMapSchema), "")
-          } else {
-            // for timeseries rollup policy
-            val tableSelectedForRollup = PreAggregateUtil.getRollupDataMapNameForTimeSeries(list,
-              dataMapSchema)
-            list += dataMapSchema
-            // if non of the rollup data map is selected hit the maintable and prepare query
-            if (tableSelectedForRollup.isEmpty) {
-              (PreAggregateUtil.createTimeSeriesSelectQueryFromMain(dataMapSchema.getChildSchema,
-                parentTableName,
-                databaseName), "")
-            } else {
-              // otherwise hit the select rollup datamap schema
-              (PreAggregateUtil.createTimeseriesSelectQueryForRollup(dataMapSchema.getChildSchema,
-                tableSelectedForRollup.get,
-                databaseName),
-                s"$databaseName.${tableSelectedForRollup.get.getChildSchema.getTableName}")
-            }
-          }
-          val childDataFrame = sparkSession.sql(new CarbonSpark2SqlParser().addPreAggLoadFunction(
-            childSelectQuery._1)).drop("preAggLoad")
-          val isOverwrite =
-            operationContext.getProperty("isOverwrite").asInstanceOf[Boolean]
-          val loadCommand = PreAggregateUtil.createLoadCommandForChild(
-            dataMapSchema.getChildSchema.getListOfColumns,
-            TableIdentifier(childTableName, Some(childDatabaseName)),
-            childDataFrame,
-            isOverwrite,
-            sparkSession,
-            tableEvent.getOptions.asScala,
-            timeseriesParentTableName = childSelectQuery._2)
-          operationContext.setProperty("uuid", uuid)
-          loadCommand.operationContext.setProperty("uuid", uuid)
-          loadCommand.processMetadata(sparkSession)
-          operationContext.setProperty(dataMapSchema.getChildSchema.getTableName, loadCommand)
-        }
-      }
-    }
-  }
-}
-object LoadPostAggregateListener extends OperationEventListener {
-  /**
-   * Called on a specified event occurrence
-   *
-   * @param event
-   */
-  override def onEvent(event: Event, operationContext: OperationContext): Unit = {
-    val carbonLoadModelOption =
-      event match {
-        case e: LoadTablePreStatusUpdateEvent => Some(e.getCarbonLoadModel)
-        case e: LoadTablePostExecutionEvent => Some(e.getCarbonLoadModel)
-        case _ => None
-      }
-    val sparkSession = SparkSession.getActiveSession.get
-    if (carbonLoadModelOption.isDefined) {
-      val carbonLoadModel = carbonLoadModelOption.get
-      val table = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
-      if (CarbonUtil.hasAggregationDataMap(table)) {
-        val isOverwrite =
-          operationContext.getProperty("isOverwrite").asInstanceOf[Boolean]
-        if (isOverwrite && table.isHivePartitionTable) {
-          val parentPartitionColumns = table.getPartitionInfo.getColumnSchemaList.asScala
-            .map(_.getColumnName)
-          val childTablesWithoutPartitionColumns =
-            table.getTableInfo.getDataMapSchemaList.asScala.filter { dataMapSchema =>
-              val childColumns = dataMapSchema.getChildSchema.getListOfColumns.asScala
-              val partitionColExists = parentPartitionColumns.forall {
-                partition =>
-                  childColumns.exists { childColumn =>
-                    childColumn.getAggFunction.isEmpty &&
-                    childColumn.getParentColumnTableRelations.asScala.head.getColumnName.
-                      equals(partition)
-                  }
-              }
-              !partitionColExists
-            }
-          if (childTablesWithoutPartitionColumns.nonEmpty) {
-            throw new MetadataProcessException(
-              "Cannot execute load overwrite or insert overwrite as the following aggregate tables"
-              + s" ${
-                childTablesWithoutPartitionColumns.toList.map(_.getChildSchema.getTableName)
-              } are not partitioned on all the partition column. Drop these to continue")
-          }
-        }
-        // getting all the aggergate datamap schema
-        val aggregationDataMapList = table.getTableInfo.getDataMapSchemaList.asScala
-          .filter(_.isInstanceOf[AggregationDataMapSchema])
-          .asInstanceOf[mutable.ArrayBuffer[AggregationDataMapSchema]]
-        // sorting the datamap for timeseries rollup
-        val sortedList = aggregationDataMapList.sortBy(_.getOrdinal)
-        val successDataMaps = sortedList.takeWhile { dataMapSchema =>
-          val childLoadCommand = operationContext
-            .getProperty(dataMapSchema.getChildSchema.getTableName)
-            .asInstanceOf[CarbonLoadDataCommand]
-          childLoadCommand.dataFrame = Some(PreAggregateUtil
-            .getDataFrame(sparkSession, childLoadCommand.logicalPlan.get))
-          childLoadCommand.operationContext = operationContext
-          val timeseriesParent = childLoadCommand.internalOptions.get("timeseriesParent")
-          val (parentTableIdentifier, segmentToLoad) =
-            if (timeseriesParent.isDefined && timeseriesParent.get.nonEmpty) {
-              val (parentTableDatabase, parentTableName) =
-                (timeseriesParent.get.split('.')(0), timeseriesParent.get.split('.')(1))
-              (TableIdentifier(parentTableName, Some(parentTableDatabase)),
-                operationContext.getProperty(
-                  s"${ parentTableDatabase }_${ parentTableName }_Segment").toString)
-            } else {
-              val currentSegmentFile = operationContext.getProperty("current.segmentfile")
-              val segment = if (currentSegmentFile != null) {
-                new Segment(carbonLoadModel.getSegmentId, currentSegmentFile.toString)
-              } else {
-                Segment.toSegment(carbonLoadModel.getSegmentId, null)
-              }
-              (TableIdentifier(table.getTableName, Some(table.getDatabaseName)), segment.toString)
-            }
-
-        PreAggregateUtil.startDataLoadForDataMap(
-        parentTableIdentifier,
-            segmentToLoad,
-            validateSegments = false,
-            childLoadCommand,
-            isOverwrite,
-            sparkSession)
-        }
-        val loadFailed = successDataMaps.lengthCompare(sortedList.length) != 0
-        if (loadFailed) {
-          successDataMaps.foreach(dataMapSchema => markSuccessSegmentsAsFailed(operationContext
-            .getProperty(dataMapSchema.getChildSchema.getTableName)
-            .asInstanceOf[CarbonLoadDataCommand]))
-          throw new RuntimeException(
-            "Data Load failed for DataMap. Please check logs for the failure")
-        }
-      }
-    }
-  }
-
-  private def markSuccessSegmentsAsFailed(childLoadCommand: CarbonLoadDataCommand) {
-    val segmentToRevert = childLoadCommand.operationContext
-      .getProperty(childLoadCommand.table.getTableUniqueName + "_Segment")
-    val tableStatusPath = CarbonTablePath.getTableStatusFilePath(
-      childLoadCommand.table.getTablePath)
-    val tableStatusContents = SegmentStatusManager.readTableStatusFile(tableStatusPath)
-    val updatedLoadDetails = tableStatusContents.collect {
-      case content if content.getLoadName == segmentToRevert =>
-        content.setSegmentStatus(SegmentStatus.MARKED_FOR_DELETE)
-        content
-      case others => others
-    }
-    SegmentStatusManager.writeLoadDetailsIntoFile(tableStatusPath, updatedLoadDetails)
-  }
-}
-
-/**
- * This listener is used to commit all the child data aggregate tables in one transaction. If one
- * failes all will be reverted to original state.
- */
-object CommitPreAggregateListener extends OperationEventListener with CommitHelper {
-
-  override protected def onEvent(event: Event,
-      operationContext: OperationContext): Unit = {
-    // The same listener is called for both compaction and load therefore getting the
-    // carbonLoadModel from the appropriate event.
-    val carbonLoadModel = event match {
-      case loadEvent: LoadTablePostStatusUpdateEvent =>
-        loadEvent.getCarbonLoadModel
-      case compactionEvent: AlterTableCompactionPostStatusUpdateEvent =>
-        // Once main table compaction is done and 0.1, 4.1, 8.1 is created commit will happen for
-        // all the tables. The commit listener will compact the child tables until no more segments
-        // are left. But 2nd level compaction is yet to happen on the main table therefore again the
-        // compaction flow will try to commit the child tables which is wrong. This check tell the
-        // 2nd level compaction flow that the commit for datamaps is already done.
-        if (null != operationContext.getProperty("commitComplete")) {
-          return
-        }
-        compactionEvent.carbonLoadModel
-    }
-    val isCompactionFlow = Option(
-      operationContext.getProperty("isCompaction")).getOrElse("false").toString.toBoolean
-    val dataMapSchemas =
-      carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable.getTableInfo.getDataMapSchemaList
-      .asScala.filter(_.getChildSchema != null)
-    // extract all child LoadCommands
-    val childLoadCommands = if (!isCompactionFlow) {
-      // If not compaction flow then the key for load commands will be tableName
-        dataMapSchemas.map { dataMapSchema =>
-          operationContext.getProperty(dataMapSchema.getChildSchema.getTableName)
-            .asInstanceOf[CarbonLoadDataCommand]
-        }
-      } else {
-      // If not compaction flow then the key for load commands will be tableName_Compaction
-        dataMapSchemas.map { dataMapSchema =>
-          operationContext.getProperty(dataMapSchema.getChildSchema.getTableName + "_Compaction")
-            .asInstanceOf[CarbonLoadDataCommand]
-        }
-    }
-    var commitFailed = false
-    try {
-      if (dataMapSchemas.nonEmpty) {
-        val uuid = operationContext.getProperty("uuid").toString
-        // keep committing until one fails
-        val renamedDataMaps = childLoadCommands.takeWhile { childLoadCommand =>
-          val childCarbonTable = childLoadCommand.table
-          // Generate table status file name with UUID, forExample: tablestatus_1
-          val oldTableSchemaPath = CarbonTablePath.getTableStatusFilePathWithUUID(
-            childCarbonTable.getTablePath, uuid)
-          // Generate table status file name without UUID, forExample: tablestatus
-          val newTableSchemaPath = CarbonTablePath.getTableStatusFilePath(
-            childCarbonTable.getTablePath)
-          mergeTableStatusContents(childCarbonTable, oldTableSchemaPath, newTableSchemaPath)
-        }
-        // if true then the commit for one of the child tables has failed
-        commitFailed = renamedDataMaps.lengthCompare(dataMapSchemas.length) != 0
-        if (commitFailed) {
-          LOGGER.warn("Reverting table status file to original state")
-          childLoadCommands.foreach {
-            childLoadCommand =>
-              val tableStatusPath = CarbonTablePath.getTableStatusFilePath(
-                childLoadCommand.table.getTablePath)
-              markInProgressSegmentAsDeleted(tableStatusPath,
-                operationContext,
-                childLoadCommand.table)
-          }
-        }
-        // after success/failure of commit delete all tablestatus files with UUID in their names.
-        // if commit failed then remove the segment directory
-        cleanUpStaleTableStatusFiles(childLoadCommands.map(_.table),
-          operationContext,
-          uuid)
-        operationContext.setProperty("commitComplete", !commitFailed)
-        if (commitFailed) {
-          sys.error("Failed to update table status for pre-aggregate table")
-        }
-      }
-    } catch {
-      case e: Exception =>
-        operationContext.setProperty("commitComplete", false)
-        LOGGER.error("Problem while committing data maps", e)
-    }
-  }
-}
-
-/**
- * Listener to handle the operations that have to be done after compaction for a table has finished.
- */
-object AlterPreAggregateTableCompactionPostListener extends OperationEventListener {
-  /**
-   * Called on a specified event occurrence
-   *
-   * @param event
-   * @param operationContext
-   */
-  override def onEvent(event: Event, operationContext: OperationContext): Unit = {
-    val compactionEvent = event.asInstanceOf[AlterTableCompactionPreStatusUpdateEvent]
-    val carbonTable = compactionEvent.carbonTable
-    val compactionType = compactionEvent.carbonMergerMapping.campactionType
-    if (compactionType == CompactionType.CUSTOM) {
-      return
-    }
-    val carbonLoadModel = compactionEvent.carbonLoadModel
-    val sparkSession = compactionEvent.sparkSession
-    if (CarbonUtil.hasAggregationDataMap(carbonTable)) {
-      carbonTable.getTableInfo.getDataMapSchemaList.asScala.foreach { dataMapSchema =>
-        val childRelationIdentifier = dataMapSchema.getRelationIdentifier
-        val alterTableModel = AlterTableModel(Some(childRelationIdentifier.getDatabaseName),
-          childRelationIdentifier.getTableName,
-          None,
-          compactionType.toString,
-          Some(System.currentTimeMillis()),
-          "")
-        operationContext.setProperty(
-          dataMapSchema.getRelationIdentifier.getDatabaseName + "_" +
-          dataMapSchema.getRelationIdentifier.getTableName + "_Segment",
-          carbonLoadModel.getSegmentId)
-        CarbonAlterTableCompactionCommand(alterTableModel, operationContext = operationContext)
-          .run(sparkSession)
-      }
-    }
-  }
-}
-
-object LoadPreAggregateTablePreListener extends OperationEventListener {
-  /**
-   * Called on a specified event occurrence
-   *
-   * @param event
-   * @param operationContext
-   */
-  override def onEvent(event: Event, operationContext: OperationContext): Unit = {
-    val loadEvent = event.asInstanceOf[LoadTablePreExecutionEvent]
-    val carbonLoadModel = loadEvent.getCarbonLoadModel
-    val table = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
-    val isInternalLoadCall = carbonLoadModel.isAggLoadRequest
-    if ((table.isChildDataMap || table.isChildTable) && !isInternalLoadCall) {
-      throw new UnsupportedOperationException(
-        "Cannot insert/load data directly into pre-aggregate/child table")
-    }
-  }
-}
-
-object PreAggregateDataTypeChangePreListener extends OperationEventListener {
-  /**
-   * Called on a specified event occurrence
-   *
-   * @param event
-   * @param operationContext
-   */
-  override def onEvent(event: Event, operationContext: OperationContext): Unit = {
-    val colRenameDataTypeChangePreListener = event
-      .asInstanceOf[AlterTableColRenameAndDataTypeChangePreEvent]
-    val carbonTable = colRenameDataTypeChangePreListener.carbonTable
-    val alterTableDataTypeChangeModel = colRenameDataTypeChangePreListener
-      .alterTableDataTypeChangeModel
-    val columnToBeAltered: String = alterTableDataTypeChangeModel.columnName
-    if (CarbonUtil.hasAggregationDataMap(carbonTable)) {
-      val dataMapSchemas = carbonTable.getTableInfo.getDataMapSchemaList
-      dataMapSchemas.asScala.foreach { dataMapSchema =>
-        val childColumns = dataMapSchema.getChildSchema.getListOfColumns
-        val parentColumnNames = childColumns.asScala
-          .flatMap(_.getParentColumnTableRelations.asScala.map(_.getColumnName))
-        if (parentColumnNames.contains(columnToBeAltered)) {
-          throw new UnsupportedOperationException(
-            s"Column $columnToBeAltered exists in a pre-aggregate table. Drop pre-aggregate table" +
-            "to continue")
-        }
-      }
-    }
-    if (carbonTable.isChildDataMap) {
-      throw new UnsupportedOperationException(
-        s"Cannot change data type or rename column for columns in pre-aggregate table ${
-          carbonTable.getDatabaseName
-        }.${ carbonTable.getTableName }")
-    }
-  }
-}
-
-object PreAggregateAddColumnsPreListener extends OperationEventListener {
-  /**
-   * Called on a specified event occurrence
-   *
-   * @param event
-   * @param operationContext
-   */
-  override def onEvent(event: Event, operationContext: OperationContext): Unit = {
-    val dataTypeChangePreListener = event.asInstanceOf[AlterTableAddColumnPreEvent]
-    val carbonTable = dataTypeChangePreListener.carbonTable
-    if (carbonTable.isChildDataMap) {
-      throw new UnsupportedOperationException(
-        s"Cannot add columns in pre-aggreagate table ${ carbonTable.getDatabaseName
-        }.${ carbonTable.getTableName }")
-    }
-  }
-}
-
-object PreAggregateDeleteSegmentByDatePreListener extends OperationEventListener {
-  /**
-   * Called on a specified event occurrence
-   *
-   * @param event
-   * @param operationContext
-   */
-  override def onEvent(event: Event, operationContext: OperationContext): Unit = {
-    val deleteSegmentByDatePreEvent = event.asInstanceOf[DeleteSegmentByDatePreEvent]
-    val carbonTable = deleteSegmentByDatePreEvent.carbonTable
-    if (carbonTable != null) {
-      if (CarbonUtil.hasAggregationDataMap(carbonTable)) {
-        throw new UnsupportedOperationException(
-          "Delete segment operation is not supported on tables which have a pre-aggregate table. " +
-          "Drop pre-aggregation table to continue")
-      }
-      if (carbonTable.isChildDataMap) {
-        throw new UnsupportedOperationException(
-          "Delete segment operation is not supported on pre-aggregate table")
-      }
-    }
-  }
-}
-
-object PreAggregateDeleteSegmentByIdPreListener extends OperationEventListener {
-  /**
-   * Called on a specified event occurrence
-   *
-   * @param event
-   * @param operationContext
-   */
-  override def onEvent(event: Event, operationContext: OperationContext): Unit = {
-    val tableEvent = event.asInstanceOf[DeleteSegmentByIdPreEvent]
-    val carbonTable = tableEvent.carbonTable
-    if (carbonTable != null) {
-      if (CarbonUtil.hasAggregationDataMap(carbonTable)) {
-        throw new UnsupportedOperationException(
-          "Delete segment operation is not supported on tables which have a pre-aggregate table")
-      }
-      if (carbonTable.isChildDataMap) {
-        throw new UnsupportedOperationException(
-          "Delete segment operation is not supported on pre-aggregate table")
-      }
-    }
-  }
-
-}
-
-object PreAggregateDropColumnPreListener extends OperationEventListener {
-  /**
-   * Called on a specified event occurrence
-   *
-   * @param event
-   * @param operationContext
-   */
-  override def onEvent(event: Event, operationContext: OperationContext): Unit = {
-    val dataTypeChangePreListener = event.asInstanceOf[AlterTableDropColumnPreEvent]
-    val carbonTable = dataTypeChangePreListener.carbonTable
-    val alterTableDropColumnModel = dataTypeChangePreListener.alterTableDropColumnModel
-    val columnsToBeDropped = alterTableDropColumnModel.columns
-    if (CarbonUtil.hasAggregationDataMap(carbonTable)) {
-      val dataMapSchemas = carbonTable.getTableInfo.getDataMapSchemaList
-      dataMapSchemas.asScala.foreach { dataMapSchema =>
-        val parentColumnNames = dataMapSchema.getChildSchema.getListOfColumns.asScala
-          .flatMap(_.getParentColumnTableRelations.asScala.map(_.getColumnName))
-        val columnExistsInChild = parentColumnNames.collectFirst {
-          case parentColumnName if columnsToBeDropped.contains(parentColumnName) =>
-            parentColumnName
-        }
-        if (columnExistsInChild.isDefined) {
-          throw new UnsupportedOperationException(
-            s"Column ${ columnExistsInChild.head } cannot be dropped because it exists in a " +
-            s"pre-aggregate table ${ dataMapSchema.getRelationIdentifier.toString }")
-        }
-      }
-    }
-    if (carbonTable.isChildDataMap) {
-      throw new UnsupportedOperationException(s"Cannot drop columns in pre-aggreagate table ${
-        carbonTable.getDatabaseName}.${ carbonTable.getTableName }")
-    }
-  }
-}
-
-object RenameTablePreListener extends OperationEventListener {
-  /**
-   * Called on a specified event occurrence
-   *
-   * @param event
-   * @param operationContext
-   */
-  override def onEvent(event: Event,
-      operationContext: OperationContext): Unit = {
-    val renameTablePostListener = event.asInstanceOf[AlterTableRenamePreEvent]
-    val carbonTable = renameTablePostListener.carbonTable
-    if (carbonTable.isChildDataMap) {
-      throw new UnsupportedOperationException(
-        "Rename operation for datamaps is not supported.")
-    }
-    if (carbonTable.hasAggregationDataMap) {
-      throw new UnsupportedOperationException(
-        "Rename operation is not supported for table with pre-aggregate tables")
-    }
-  }
-}
-
-object UpdatePreAggregatePreListener extends OperationEventListener {
-  /**
-   * Called on a specified event occurrence
-   *
-   * @param event
-   * @param operationContext
-   */
-  override def onEvent(event: Event, operationContext: OperationContext): Unit = {
-    val tableEvent = event.asInstanceOf[UpdateTablePreEvent]
-    val carbonTable = tableEvent.carbonTable
-    if (carbonTable != null) {
-      if (CarbonUtil.hasAggregationDataMap(carbonTable)) {
-        throw new UnsupportedOperationException(
-          "Update operation is not supported for tables which have a pre-aggregate table. Drop " +
-          "pre-aggregate tables to continue.")
-      }
-      if (carbonTable.isChildDataMap) {
-        throw new UnsupportedOperationException(
-          "Update operation is not supported for pre-aggregate table")
-      }
-    }
-  }
-}
-
-object DeletePreAggregatePreListener extends OperationEventListener {
-  /**
-   * Called on a specified event occurrence
-   *
-   * @param event
-   * @param operationContext
-   */
-  override def onEvent(event: Event, operationContext: OperationContext): Unit = {
-    val tableEvent = event.asInstanceOf[DeleteFromTablePreEvent]
-    val carbonTable = tableEvent.carbonTable
-    if (carbonTable != null) {
-      if (CarbonUtil.hasAggregationDataMap(carbonTable)) {
-        throw new UnsupportedOperationException(
-          "Delete operation is not supported for tables which have a pre-aggregate table. Drop " +
-          "pre-aggregate tables to continue.")
-      }
-      if (carbonTable.isChildDataMap) {
-        throw new UnsupportedOperationException(
-          "Delete operation is not supported for pre-aggregate table")
-      }
-    }
-  }
-}
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateTableHelper.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateTableHelper.scala
deleted file mode 100644
index 86b2d00..0000000
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateTableHelper.scala
+++ /dev/null
@@ -1,232 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.execution.command.preaaggregate
-
-import scala.collection.JavaConverters._
-import scala.collection.mutable
-
-import org.apache.spark.sql._
-import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.execution.command._
-import org.apache.spark.sql.execution.command.datamap.CarbonDropDataMapCommand
-import org.apache.spark.sql.execution.command.management.CarbonLoadDataCommand
-import org.apache.spark.sql.execution.command.table.CarbonCreateTableCommand
-import org.apache.spark.sql.execution.command.timeseries.TimeSeriesUtil
-import org.apache.spark.sql.optimizer.CarbonFilters
-import org.apache.spark.sql.parser.CarbonSpark2SqlParser
-import org.apache.spark.util.{DataMapUtil, PartitionUtils}
-
-import org.apache.carbondata.common.exceptions.MetadataProcessException
-import org.apache.carbondata.common.exceptions.sql.MalformedDataMapCommandException
-import org.apache.carbondata.common.logging.LogServiceFactory
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.datamap.Segment
-import org.apache.carbondata.core.metadata.schema.partition.PartitionType
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable
-import org.apache.carbondata.core.statusmanager.{SegmentStatus, SegmentStatusManager}
-
-/**
- * Below helper class will be used to create pre-aggregate table
- * and updating the parent table about the child table information
- * It will be either success or nothing happen in case of failure:
- * 1. failed to create pre aggregate table.
- * 2. failed to update main table
- *
- */
-case class PreAggregateTableHelper(
-    var parentTable: CarbonTable,
-    dataMapName: String,
-    dataMapClassName: String,
-    dataMapProperties: java.util.Map[String, String],
-    queryString: String,
-    timeSeriesFunction: Option[String] = None,
-    ifNotExistsSet: Boolean = false) {
-
-  var loadCommand: CarbonLoadDataCommand = _
-
-  private val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
-
-  def initMeta(sparkSession: SparkSession): Seq[Row] = {
-    val dmProperties = dataMapProperties.asScala
-    val updatedQuery = new CarbonSpark2SqlParser().addPreAggFunction(queryString)
-    val df = sparkSession.sql(updatedQuery)
-    val fieldRelationMap = PreAggregateUtil.validateActualSelectPlanAndGetAttributes(
-      df.logicalPlan, queryString)
-
-    val partitionInfo = parentTable.getPartitionInfo
-    val fields = fieldRelationMap.keySet.toSeq
-    val tableProperties = mutable.Map[String, String]()
-    val usePartitioning = dataMapProperties.getOrDefault("partitioning", "true").toBoolean
-    val parentPartitionColumns = if (!usePartitioning) {
-      Seq.empty
-    } else if (parentTable.isHivePartitionTable) {
-      partitionInfo.getColumnSchemaList.asScala.map(_.getColumnName)
-    } else {
-      Seq()
-    }
-    // Generate child table partition columns in the same order as the parent table.
-    val partitionerFields =
-      PartitionUtils.getPartitionerFields(parentPartitionColumns, fieldRelationMap)
-
-    dmProperties.foreach(t => tableProperties.put(t._1, t._2))
-
-    val selectTable = PreAggregateUtil.getParentCarbonTable(df.logicalPlan)
-    if (!parentTable.getTableName.equalsIgnoreCase(selectTable.getTableName)) {
-      throw new MalformedDataMapCommandException(
-        "Parent table name is different in select and create")
-    }
-    DataMapUtil
-      .inheritTablePropertiesFromMainTable(parentTable, fields, fieldRelationMap, tableProperties)
-    val tableIdentifier =
-      TableIdentifier(parentTable.getTableName + "_" + dataMapName,
-        Some(parentTable.getDatabaseName))
-    // prepare table model of the collected tokens
-    val tableModel: TableModel = new CarbonSpark2SqlParser().prepareTableModel(
-      ifNotExistPresent = ifNotExistsSet,
-      new CarbonSpark2SqlParser().convertDbNameToLowerCase(tableIdentifier.database),
-      tableIdentifier.table.toLowerCase,
-      fields,
-      partitionerFields,
-      tableProperties,
-      None,
-      isAlterFlow = false,
-      true,
-      None)
-
-    // 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
-    if(timeSeriesFunction != null) {
-      TimeSeriesUtil.validateTimeSeriesEventTime(dmProperties.toMap, parentTable)
-      TimeSeriesUtil.validateEventTimeColumnExitsInSelect(
-        fieldRelationMap,
-        dmProperties(TimeSeriesUtil.TIMESERIES_EVENTTIME))
-      TimeSeriesUtil.updateTimeColumnSelect(
-        fieldRelationMap,
-        dmProperties(TimeSeriesUtil.TIMESERIES_EVENTTIME),
-        timeSeriesFunction.get)
-    }
-    tableModel.parentTable = Some(parentTable)
-    tableModel.dataMapRelation = Some(fieldRelationMap)
-    val tablePath = if (dmProperties.contains("path")) {
-      dmProperties("path")
-    } else {
-      CarbonEnv.getTablePath(tableModel.databaseNameOp, tableModel.tableName)(sparkSession)
-    }
-    CarbonCreateTableCommand(TableNewProcessor(tableModel),
-      tableModel.ifNotExistsSet, Some(tablePath), isVisible = false).run(sparkSession)
-
-    val table = CarbonEnv.getCarbonTable(tableIdentifier)(sparkSession)
-    val tableInfo = table.getTableInfo
-
-    // child schema object will be saved on parent table schema
-    val childSchema = tableInfo.getFactTable.buildChildSchema(
-      dataMapName,
-      dataMapClassName,
-      tableInfo.getDatabaseName,
-      queryString,
-      "AGGREGATION")
-    dmProperties.foreach(f => childSchema.getProperties.put(f._1, f._2))
-
-    try {
-      // updating the parent table about child table
-      PreAggregateUtil.updateMainTable(parentTable, childSchema, sparkSession)
-    } catch {
-      case e: MetadataProcessException =>
-        throw e
-      case ex: Exception =>
-        // If updation failed then forcefully remove datamap from metastore.
-        val dropTableCommand = CarbonDropDataMapCommand(childSchema.getDataMapName,
-          ifExistsSet = true,
-          Some(TableIdentifier
-            .apply(parentTable.getTableName, Some(parentTable.getDatabaseName))),
-          forceDrop = true)
-        dropTableCommand.processMetadata(sparkSession)
-        throw ex
-    }
-    // After updating the parent carbon table with data map entry extract the latest table object
-    // to be used in further create process.
-    parentTable = CarbonEnv.getCarbonTable(Some(parentTable.getDatabaseName),
-      parentTable.getTableName)(sparkSession)
-
-    val updatedLoadQuery = if (timeSeriesFunction != null) {
-      PreAggregateUtil.createTimeSeriesSelectQueryFromMain(
-        childSchema.getChildSchema,
-        parentTable.getTableName,
-        parentTable.getDatabaseName)
-    } else {
-      queryString
-    }
-    val dataFrame = sparkSession.sql(new CarbonSpark2SqlParser().addPreAggLoadFunction(
-      updatedLoadQuery)).drop("preAggLoad")
-    loadCommand = PreAggregateUtil.createLoadCommandForChild(
-      childSchema.getChildSchema.getListOfColumns,
-      tableIdentifier,
-      dataFrame,
-      isOverwrite = false,
-      sparkSession = sparkSession,
-      mutable.Map.empty[String, String])
-    loadCommand.processMetadata(sparkSession)
-    Seq.empty
-  }
-
-  def initData(sparkSession: SparkSession): Seq[Row] = {
-    // load child table if parent table has existing segments
-    // This will be used to check if the parent table has any segments or not. If not then no
-    // need to fire load for pre-aggregate table. Therefore reading the load details for PARENT
-    // table.
-    SegmentStatusManager.deleteLoadsAndUpdateMetadata(
-      parentTable,
-      false,
-      CarbonFilters.getCurrentPartitions(
-        sparkSession,
-        TableIdentifier(parentTable.getTableName,
-          Some(parentTable.getDatabaseName))
-      ).map(_.asJava).orNull)
-
-    if (SegmentStatusManager.isLoadInProgressInTable(parentTable)) {
-      throw new UnsupportedOperationException(
-        "Cannot create pre-aggregate table when insert is in progress on parent table")
-    }
-    // check if any segment if available for load in the parent table
-    val loadAvailable = SegmentStatusManager.readLoadMetadata(parentTable.getMetadataPath)
-      .collect {
-        case segment if segment.getSegmentStatus == SegmentStatus.SUCCESS ||
-                        segment.getSegmentStatus == SegmentStatus.LOAD_PARTIAL_SUCCESS =>
-          new Segment(segment.getLoadName, segment.getSegmentFile).toString
-      }
-    if (loadAvailable.nonEmpty) {
-      // Passing segmentToLoad as * because we want to load all the segments into the
-      // pre-aggregate table even if the user has set some segments on the parent table.
-      loadCommand.dataFrame = Some(PreAggregateUtil
-        .getDataFrame(sparkSession, loadCommand.logicalPlan.get))
-      PreAggregateUtil.startDataLoadForDataMap(
-        TableIdentifier(parentTable.getTableName, Some(parentTable.getDatabaseName)),
-        segmentToLoad = loadAvailable.mkString(","),
-        validateSegments = false,
-        loadCommand,
-        isOverwrite = false,
-        sparkSession)
-    } else {
-      LOGGER.info(s"No segment available for load in table ${parentTable.getTableUniqueName}")
-    }
-    Seq.empty
-  }
-}
-
-
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
deleted file mode 100644
index 1d4abcb..0000000
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala
+++ /dev/null
@@ -1,905 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.spark.sql.execution.command.preaaggregate
-
-import scala.collection.JavaConverters._
-import scala.collection.mutable
-import scala.collection.mutable.{ArrayBuffer, ListBuffer}
-
-import org.apache.spark.sql.{CarbonDatasourceHadoopRelation, CarbonEnv, CarbonSession, SparkSession, _}
-import org.apache.spark.sql.CarbonExpressions.{CarbonSubqueryAlias => SubqueryAlias}
-import org.apache.spark.sql.CarbonExpressions.MatchCastExpression
-import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.catalyst.analysis.{UnresolvedAlias, UnresolvedFunction, UnresolvedRelation}
-import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.expressions.aggregate.{Count, _}
-import org.apache.spark.sql.catalyst.plans.logical._
-import org.apache.spark.sql.execution.command.{ColumnTableRelation, DataMapField, Field}
-import org.apache.spark.sql.execution.command.management.CarbonLoadDataCommand
-import org.apache.spark.sql.execution.datasources.LogicalRelation
-import org.apache.spark.sql.hive.{CarbonMetaStore, CarbonRelation}
-import org.apache.spark.sql.parser.CarbonSpark2SqlParser
-import org.apache.spark.sql.types.DataType
-
-import org.apache.carbondata.common.exceptions.MetadataProcessException
-import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
-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.datamap.DataMapProperty
-import org.apache.carbondata.core.metadata.schema.table.{AggregationDataMapSchema, CarbonTable, DataMapSchema, TableSchema}
-import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema
-import org.apache.carbondata.core.util.CarbonUtil
-import org.apache.carbondata.format.TableInfo
-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(_, _, SubqueryAlias(_, logicalRelation: LogicalRelation))
-        if logicalRelation.relation.isInstanceOf[CarbonDatasourceHadoopRelation] =>
-        logicalRelation.relation.asInstanceOf[CarbonDatasourceHadoopRelation].
-          carbonRelation.metaData.carbonTable
-      case Aggregate(_, _, logicalRelation: LogicalRelation)
-        if logicalRelation.relation.isInstanceOf[CarbonDatasourceHadoopRelation] =>
-        logicalRelation.relation.asInstanceOf[CarbonDatasourceHadoopRelation].
-          carbonRelation.metaData.carbonTable
-      case _ => throw new MalformedCarbonCommandException("table does not exist")
-    }
-  }
-
-  /**
-   * 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 validateActualSelectPlanAndGetAttributes(plan: LogicalPlan,
-      selectStmt: String): scala.collection.mutable.LinkedHashMap[Field, DataMapField] = {
-    plan match {
-      case Aggregate(groupByExp, aggExp, SubqueryAlias(_, logicalRelation: LogicalRelation)) =>
-        getFieldsFromPlan(groupByExp, aggExp, logicalRelation, selectStmt)
-      case Aggregate(groupByExp, aggExp, logicalRelation: LogicalRelation) =>
-        getFieldsFromPlan(groupByExp, aggExp, logicalRelation, selectStmt)
-    }
-  }
-
-  /**
-   * Below method will be used to get the fields from expressions
-   * @param groupByExp grouping expression
-   * @param aggExp aggregate expression
-   * @param logicalRelation logical relation
-   * @param selectStmt select statement
-   * @return fields from expressions
-   */
-  def getFieldsFromPlan(groupByExp: Seq[Expression],
-      aggExp: Seq[NamedExpression], logicalRelation: LogicalRelation, selectStmt: String):
-  scala.collection.mutable.LinkedHashMap[Field, DataMapField] = {
-    val fieldToDataMapFieldMap = scala.collection.mutable.LinkedHashMap.empty[Field, DataMapField]
-    if (!logicalRelation.relation.isInstanceOf[CarbonDatasourceHadoopRelation]) {
-      throw new MalformedCarbonCommandException("Un-supported table")
-    }
-    val carbonTable = logicalRelation.relation.
-      asInstanceOf[CarbonDatasourceHadoopRelation].carbonRelation
-      .metaData.carbonTable
-    val parentTableName = carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier
-      .getTableName
-    val parentDatabaseName = carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier
-      .getDatabaseName
-    val parentTableId = carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier
-      .getTableId
-    if (!carbonTable.getTableInfo.getParentRelationIdentifiers.isEmpty) {
-      throw new MalformedCarbonCommandException(
-        "Pre Aggregation is not supported on Pre-Aggregated Table")
-    }
-    var counter = 0
-    aggExp.map {
-      case Alias(attr: AggregateExpression, name) =>
-        if (attr.isDistinct) {
-          throw new MalformedCarbonCommandException(
-            "Distinct is not supported On Pre Aggregation")
-        }
-        fieldToDataMapFieldMap ++= validateAggregateFunctionAndGetFields(
-          carbonTable,
-          attr.aggregateFunction,
-          parentTableName,
-          parentDatabaseName,
-          parentTableId,
-          "column_" + counter)
-        counter = counter + 1
-      case attr: AttributeReference =>
-        val columnRelation = getColumnRelation(
-          attr.name,
-          parentTableId,
-          parentTableName,
-          parentDatabaseName,
-          carbonTable)
-        fieldToDataMapFieldMap += createField(
-          attr.name,
-          attr.dataType,
-          parentTableName = parentTableName,
-          columnTableRelationList = Seq(columnRelation))
-      case Alias(attr: AttributeReference, _) =>
-        val columnRelation = getColumnRelation(
-          attr.name,
-          parentTableId,
-          parentTableName,
-          parentDatabaseName,
-          carbonTable)
-        fieldToDataMapFieldMap += createField(
-          attr.name,
-          attr.dataType,
-          parentTableName = parentTableName,
-          columnTableRelationList = Seq(columnRelation))
-      case _@Alias(s: ScalaUDF, name) if name.equals("preAgg") =>
-      case _ =>
-        throw new MalformedCarbonCommandException(s"Unsupported Select Statement:${
-          selectStmt } ")
-    }
-    groupByExp map {
-      case attr: AttributeReference =>
-        val columnRelation = getColumnRelation(
-          attr.name,
-          parentTableId,
-          parentTableName,
-          parentDatabaseName,
-          carbonTable)
-        fieldToDataMapFieldMap += createField(
-          attr.name,
-          attr.dataType,
-          parentTableName = parentTableName,
-          columnTableRelationList = Seq(columnRelation))
-      case _ =>
-        throw new MalformedCarbonCommandException(s"Unsupported Function in select Statement:${
-          selectStmt }")
-    }
-    fieldToDataMapFieldMap
-  }
-
-  /**
-   * Below method will be used to get the column relation
-   * with the parent column which will be used during query and data loading
-   * @param parentColumnName parent column name
-   * @param parentTableId parent column id
-   * @param parentTableName parent table name
-   * @param parentDatabaseName parent database name
-   * @param carbonTable carbon table
-   * @return column relation object
-   */
-  def getColumnRelation(parentColumnName: String,
-      parentTableId: String,
-      parentTableName: String,
-      parentDatabaseName: String,
-      carbonTable: CarbonTable) : ColumnTableRelation = {
-    val parentColumnId = carbonTable.getColumnByName(parentColumnName).getColumnId
-    val columnTableRelation = ColumnTableRelation(parentColumnName = parentColumnName.toLowerCase(),
-      parentColumnId = parentColumnId,
-      parentTableName = parentTableName,
-      parentDatabaseName = parentDatabaseName, parentTableId = parentTableId)
-    columnTableRelation
-  }
-
-  /**
-   * 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 parent carbon table
-   * @param aggFunctions aggregation function
-   * @param parentTableName parent table name
-   * @param parentDatabaseName parent database name
-   * @param parentTableId parent column id
-   * @param newColumnName
-   * In case of any expression this will be used as a column name for pre aggregate
-   * @return list of fields
-   */
-  def validateAggregateFunctionAndGetFields(carbonTable: CarbonTable,
-      aggFunctions: AggregateFunction,
-      parentTableName: String,
-      parentDatabaseName: String,
-      parentTableId: String,
-      newColumnName: String) : scala.collection.mutable.ListBuffer[(Field, DataMapField)] = {
-    val list = scala.collection.mutable.ListBuffer.empty[(Field, DataMapField)]
-    aggFunctions match {
-      case sum@Sum(MatchCastExpression(exp: Expression, changeDataType: DataType)) =>
-        list += createFieldForAggregateExpression(
-          exp,
-          changeDataType,
-          carbonTable,
-          newColumnName,
-          sum.prettyName)
-      case sum@Sum(exp: Expression) =>
-        list += createFieldForAggregateExpression(
-          exp,
-          sum.dataType,
-          carbonTable,
-          newColumnName,
-          sum.prettyName)
-      case count@Count(Seq(MatchCastExpression(exp: Expression, changeDataType: DataType))) =>
-        list += createFieldForAggregateExpression(
-          exp,
-          changeDataType,
-          carbonTable,
-          newColumnName,
-          count.prettyName)
-      case count@Count(Seq(exp: Expression)) =>
-        list += createFieldForAggregateExpression(
-          exp,
-          count.dataType,
-          carbonTable,
-          newColumnName,
-          count.prettyName)
-      case min@Min(MatchCastExpression(exp: Expression, changeDataType: DataType)) =>
-        list += createFieldForAggregateExpression(
-          exp,
-          changeDataType,
-          carbonTable,
-          newColumnName,
-          min.prettyName)
-      case min@Min(exp: Expression) =>
-        list += createFieldForAggregateExpression(
-          exp,
-          min.dataType,
-          carbonTable,
-          newColumnName,
-          min.prettyName)
-      case max@Max(MatchCastExpression(exp: Expression, changeDataType: DataType)) =>
-        list += createFieldForAggregateExpression(
-          exp,
-          changeDataType,
-          carbonTable,
-          newColumnName,
-          max.prettyName)
-      case max@Max(exp: Expression) =>
-        list += createFieldForAggregateExpression(
-          exp,
-          max.dataType,
-          carbonTable,
-          newColumnName,
-          max.prettyName)
-      case Average(MatchCastExpression(exp: Expression, changeDataType: DataType)) =>
-        list += createFieldForAggregateExpression(
-          exp,
-          changeDataType,
-          carbonTable,
-          newColumnName,
-          "sum")
-        list += createFieldForAggregateExpression(
-          exp,
-          changeDataType,
-          carbonTable,
-          newColumnName,
-          "count")
-      case avg@Average(exp: Expression) =>
-        list += createFieldForAggregateExpression(
-          exp,
-          avg.dataType,
-          carbonTable,
-          newColumnName,
-          "sum")
-        list += createFieldForAggregateExpression(
-          exp,
-          avg.dataType,
-          carbonTable,
-          newColumnName,
-          "count")
-      case others@_ =>
-        throw new MalformedCarbonCommandException(s"Un-Supported Aggregation Type: ${
-          others.prettyName}")
-    }
-  }
-
-  /**
-   * Below method will be used to get the field and its data map field object
-   * for aggregate expression
-   * @param expression expression in aggregate function
-   * @param dataType data type
-   * @param carbonTable parent carbon table
-   * @param newColumnName column name of aggregate table
-   * @param aggregationName aggregate function name
-   * @return field and its metadata tuple
-   */
-  def createFieldForAggregateExpression(
-      expression: Expression,
-      dataType: DataType,
-      carbonTable: CarbonTable,
-      newColumnName: String,
-      aggregationName: String): (Field, DataMapField) = {
-    val parentColumnsName = new ArrayBuffer[String]()
-    expression.transform {
-      case attr: AttributeReference =>
-        parentColumnsName += attr.name
-        attr
-    }
-    val arrayBuffer = parentColumnsName.map { name =>
-       getColumnRelation(name,
-        carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier.getTableId,
-        carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier.getTableName,
-        carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier.getDatabaseName,
-        carbonTable)
-    }
-    // if parent column relation is of size more than one that means aggregate table
-    // column is derived from multiple column of main table or if size is zero then it means
-    // column is present in select statement is some constants for example count(*)
-    // and if expression is not a instance of attribute reference
-    // then use column name which is passed
-    val columnName =
-    if ((parentColumnsName.size > 1 || parentColumnsName.isEmpty) &&
-        !expression.isInstanceOf[AttributeReference]) {
-      newColumnName
-    } else {
-      if (expression.isInstanceOf[GetStructField] || expression.isInstanceOf[GetArrayItem]) {
-        throw new UnsupportedOperationException(
-          "Preaggregate is unsupported for ComplexData type column: " +
-          expression.simpleString.replaceAll("#[0-9]*", ""))
-      } else {
-        expression.asInstanceOf[AttributeReference].name
-      }
-    }
-    createField(columnName,
-      dataType,
-      aggregationName,
-      carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier.getTableName,
-      arrayBuffer)
-  }
-
-  /**
-   * Below method will be used to get the fields object for pre aggregate table
-   *
-   * @param columnName
-   * @param dataType
-   * @param aggregateType
-   * @param parentTableName
-   * @param columnTableRelationList
-   *                                List of column relation with parent
-   * @return fields object
-   */
-  def createField(columnName: String,
-      dataType: DataType,
-      aggregateType: String = "",
-      parentTableName: String,
-      columnTableRelationList: Seq[ColumnTableRelation]): (Field, DataMapField) = {
-    var actualColumnName = if (aggregateType.equals("")) {
-      parentTableName + '_' + columnName
-    } else {
-      parentTableName + '_' + columnName + '_' + aggregateType
-    }
-    val rawSchema = '`' + actualColumnName + '`' + ' ' + dataType.typeName
-    val dataMapField = DataMapField(aggregateType, Some(columnTableRelationList))
-    actualColumnName = actualColumnName.toLowerCase()
-    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 exception it will throw error so pre aggregate table creation will fail
-   *
-   * @return the existing TableInfo object before updating, it can be used to recover if any
-   *         operation failed later
-   */
-  def updateMainTable(carbonTable: CarbonTable,
-      childSchema: DataMapSchema, sparkSession: SparkSession): TableInfo = {
-    val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
-    val locksToBeAcquired = List(LockUsage.METADATA_LOCK,
-      LockUsage.DROP_TABLE_LOCK)
-    var locks = List.empty[ICarbonLock]
-    val dbName = carbonTable.getDatabaseName
-    val tableName = carbonTable.getTableName
-    try {
-      val metastore = CarbonEnv.getInstance(sparkSession).carbonMetaStore
-      locks = acquireLock(dbName, tableName, locksToBeAcquired, carbonTable)
-      // get the latest carbon table and check for column existence
-      // read the latest schema file
-      val thriftTableInfo: TableInfo = metastore.getThriftTableInfo(carbonTable)
-      val schemaConverter = new ThriftWrapperSchemaConverterImpl()
-      val wrapperTableInfo = schemaConverter.fromExternalToWrapperTableInfo(
-        thriftTableInfo,
-        dbName,
-        tableName,
-        carbonTable.getTablePath)
-      if (wrapperTableInfo.getDataMapSchemaList.asScala.
-        exists(f => f.getDataMapName.equalsIgnoreCase(childSchema.getDataMapName))) {
-        throw new MetadataProcessException("DataMap name already exist")
-      }
-      wrapperTableInfo.getDataMapSchemaList.add(childSchema)
-      val thriftTable = schemaConverter.fromWrapperToExternalTableInfo(
-        wrapperTableInfo, dbName, tableName)
-      updateSchemaInfo(carbonTable, thriftTable)(sparkSession)
-      LOGGER.info(s"Parent table updated is successful for table $dbName.$tableName")
-      thriftTableInfo
-    } catch {
-      case e: Exception =>
-        LOGGER.error("Pre Aggregate Parent table update failed reverting changes", e)
-        throw e
-    } finally {
-      // release lock after command execution completion
-      releaseLocks(locks)
-    }
-  }
-
-  /**
-   * Below method will be used to update the main table schema
-   *
-   * @param carbonTable
-   * @param thriftTable
-   * @param sparkSession
-   */
-  def updateSchemaInfo(carbonTable: CarbonTable,
-      thriftTable: TableInfo)(sparkSession: SparkSession): Unit = {
-    val dbName = carbonTable.getDatabaseName
-    val tableName = carbonTable.getTableName
-    CarbonEnv.getInstance(sparkSession).carbonMetaStore
-      .updateTableSchema(carbonTable.getCarbonTableIdentifier,
-        carbonTable.getCarbonTableIdentifier,
-        thriftTable,
-        carbonTable.getAbsoluteTableIdentifier.getTablePath)(sparkSession)
-    val tableIdentifier = TableIdentifier(tableName, Some(dbName))
-    sparkSession.catalog.refreshTable(tableIdentifier.quotedString)
-  }
-
-  /**
-   * 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.getAbsoluteTableIdentifier, 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 = CarbonEnv.getCarbonTable(Some(dbName), tableName)(sparkSession)
-    carbonTable.getTableLastUpdatedTime
-    val thriftTable: TableInfo = metastore.getThriftTableInfo(carbonTable)
-    if (thriftTable.dataMapSchemas.size > numberOfChildSchema) {
-      metastore.revertTableSchemaForPreAggCreationFailure(
-        carbonTable.getAbsoluteTableIdentifier, thriftTable)(sparkSession)
-    }
-  }
-
-  /**
-   * Below method will be used to update logical plan
-   * this is required for creating pre aggregate tables,
-   * so @CarbonPreAggregateRules will not be applied during creation
-   * @param logicalPlan actual logical plan
-   * @return updated plan
-   */
-  def updatePreAggQueyPlan(logicalPlan: LogicalPlan): LogicalPlan = {
-    val updatedPlan = logicalPlan.transform {
-      case _@Project(projectList, child) =>
-        val buffer = new ArrayBuffer[NamedExpression]()
-        buffer ++= projectList
-        buffer += UnresolvedAlias(Alias(UnresolvedFunction("preAgg",
-          Seq.empty, isDistinct = false), "preAgg")())
-        Project(buffer, child)
-      case Aggregate(groupByExp, aggExp, l: UnresolvedRelation) =>
-        val buffer = new ArrayBuffer[NamedExpression]()
-        buffer ++= aggExp
-        buffer += UnresolvedAlias(Alias(UnresolvedFunction("preAgg",
-          Seq.empty, isDistinct = false), "preAgg")())
-        Aggregate(groupByExp, buffer, l)
-    }
-    updatedPlan
-  }
-    /**
-   * This method will start load process on the data map
-   */
-  def startDataLoadForDataMap(
-      parentTableIdentifier: TableIdentifier,
-      segmentToLoad: String,
-      validateSegments: Boolean,
-      loadCommand: CarbonLoadDataCommand,
-      isOverwrite: Boolean,
-      sparkSession: SparkSession): Boolean = {
-    CarbonSession.threadSet(
-      CarbonCommonConstants.CARBON_INPUT_SEGMENTS +
-      parentTableIdentifier.database.getOrElse(sparkSession.catalog.currentDatabase) + "." +
-      parentTableIdentifier.table,
-      segmentToLoad)
-    CarbonSession.threadSet(
-      CarbonCommonConstants.VALIDATE_CARBON_INPUT_SEGMENTS +
-      parentTableIdentifier.database.getOrElse(sparkSession.catalog.currentDatabase) + "." +
-      parentTableIdentifier.table, validateSegments.toString)
-    CarbonSession.threadSet(CarbonCommonConstants.SUPPORT_DIRECT_QUERY_ON_DATAMAP,
-      "true")
-    try {
-      loadCommand.processData(sparkSession)
-      true
-    } catch {
-      case ex: Exception =>
-        LOGGER.error("Data Load failed for DataMap: ", ex)
-        false
-    } finally {
-      CarbonSession.threadUnset(
-        CarbonCommonConstants.CARBON_INPUT_SEGMENTS +
-        parentTableIdentifier.database.getOrElse(sparkSession.catalog.currentDatabase) + "." +
-        parentTableIdentifier.table)
-      CarbonSession.threadUnset(
-        CarbonCommonConstants.VALIDATE_CARBON_INPUT_SEGMENTS +
-        parentTableIdentifier.database.getOrElse(sparkSession.catalog.currentDatabase) + "." +
-        parentTableIdentifier.table)
-    }
-  }
-
-  def createChildSelectQuery(tableSchema: TableSchema, databaseName: String): String = {
-    val aggregateColumns = scala.collection.mutable.ArrayBuffer.empty[String]
-    val groupingExpressions = scala.collection.mutable.ArrayBuffer.empty[String]
-    val columns = tableSchema.getListOfColumns.asScala
-      .filter(f => !f.getColumnName.equals(CarbonCommonConstants.DEFAULT_INVISIBLE_DUMMY_MEASURE))
-    //  schema ordinal should be considered
-    columns.sortBy(_.getSchemaOrdinal).foreach { a =>
-      if (a.getAggFunction.nonEmpty) {
-        aggregateColumns += s"${a.getAggFunction match {
-          case "count" => "sum"
-          case _ => a.getAggFunction}}(${a.getColumnName})"
-      } else {
-        groupingExpressions += a.getColumnName
-        aggregateColumns += a.getColumnName
-      }
-    }
-    val groupByString = if (groupingExpressions.nonEmpty) {
-      s" group by ${ groupingExpressions.mkString(",") }"
-    } else { "" }
-    s"select ${ aggregateColumns.mkString(",") } " +
-    s"from $databaseName.${ tableSchema.getTableName }" + groupByString
-  }
-
-  /**
-   * Below method will be used to get the select query when rollup policy is
-   * applied in case of timeseries table
-   * @param tableSchema main data map schema
-   * @param selectedDataMapSchema selected data map schema for rollup
-   * @return select query based on rolloup
-   */
-  def createTimeseriesSelectQueryForRollup(
-      tableSchema: TableSchema,
-      selectedDataMapSchema: AggregationDataMapSchema,
-      databaseName: String): String = {
-    val aggregateColumns = scala.collection.mutable.ArrayBuffer.empty[String]
-    val groupingExpressions = scala.collection.mutable.ArrayBuffer.empty[String]
-    val columns = tableSchema.getListOfColumns.asScala
-      .filter(f => !f.getColumnName.equals(CarbonCommonConstants.DEFAULT_INVISIBLE_DUMMY_MEASURE))
-      .sortBy(_.getSchemaOrdinal)
-    columns.foreach { a =>
-      if (a.getAggFunction.nonEmpty) {
-        aggregateColumns += s"${a.getAggFunction match {
-          case "count" => "sum"
-          case others@_ => others}}(${selectedDataMapSchema.getAggChildColByParent(
-          a.getParentColumnTableRelations.get(0).getColumnName, a.getAggFunction).getColumnName})"
-      } else if (a.getTimeSeriesFunction.nonEmpty) {
-        groupingExpressions += s"timeseries(${
-          selectedDataMapSchema
-            .getNonAggChildColBasedByParent(a.getParentColumnTableRelations.
-              get(0).getColumnName).getColumnName
-        } , '${ a.getTimeSeriesFunction }')"
-        aggregateColumns += s"timeseries(${
-          selectedDataMapSchema
-            .getNonAggChildColBasedByParent(a.getParentColumnTableRelations.
-              get(0).getColumnName).getColumnName
-        } , '${ a.getTimeSeriesFunction }')"
-      } else {
-        groupingExpressions += selectedDataMapSchema
-          .getNonAggChildColBasedByParent(a.getParentColumnTableRelations.
-            get(0).getColumnName).getColumnName
-        aggregateColumns += selectedDataMapSchema
-          .getNonAggChildColBasedByParent(a.getParentColumnTableRelations.
-            get(0).getColumnName).getColumnName
-      }
-    }
-    s"select ${ aggregateColumns.mkString(",")
-    } from $databaseName.${selectedDataMapSchema.getChildSchema.getTableName } " +
-    s"group by ${ groupingExpressions.mkString(",") }"
-  }
-
-  /**
-   * Below method will be used to creating select query for timeseries
-   * for lowest level for aggergation like second level, in that case it will
-   * hit the maintable
-   * @param tableSchema data map schema
-   * @param parentTableName parent schema
-   * @return select query for loading
-   */
-  def createTimeSeriesSelectQueryFromMain(tableSchema: TableSchema,
-      parentTableName: String,
-      databaseName: String): String = {
-    val aggregateColumns = scala.collection.mutable.ArrayBuffer.empty[String]
-    val groupingExpressions = scala.collection.mutable.ArrayBuffer.empty[String]
-    val columns = tableSchema.getListOfColumns.asScala
-      .filter(f => !f.getColumnName.equals(CarbonCommonConstants.DEFAULT_INVISIBLE_DUMMY_MEASURE))
-      .sortBy(_.getSchemaOrdinal)
-    columns.foreach {a =>
-        if (a.getAggFunction.nonEmpty) {
-          aggregateColumns +=
-          s"${ a.getAggFunction }(${ a.getParentColumnTableRelations.get(0).getColumnName })"
-        } else if (a.getTimeSeriesFunction.nonEmpty) {
-          groupingExpressions +=
-          s"timeseries(${ a.getParentColumnTableRelations.get(0).getColumnName },'${
-            a.getTimeSeriesFunction}')"
-          aggregateColumns +=
-          s"timeseries(${ a.getParentColumnTableRelations.get(0).getColumnName },'${
-            a.getTimeSeriesFunction
-          }')"
-        } else {
-          groupingExpressions += a.getParentColumnTableRelations.get(0).getColumnName
-          aggregateColumns += a.getParentColumnTableRelations.get(0).getColumnName
-        }
-    }
-    s"select ${
-      aggregateColumns.mkString(",")
-    } from $databaseName.${ parentTableName } group by ${ groupingExpressions.mkString(",") }"
-
-  }
-    /**
-   * Below method will be used to select rollup table in case of
-   * timeseries data map loading
-   * @param list list of timeseries datamap
-   * @param dataMapSchema datamap schema
-   * @return select table name
-   */
-  def getRollupDataMapNameForTimeSeries(
-      list: scala.collection.mutable.ListBuffer[AggregationDataMapSchema],
-      dataMapSchema: AggregationDataMapSchema): Option[AggregationDataMapSchema] = {
-    if (list.isEmpty) {
-      None
-    } else {
-      val rollupDataMapSchema = scala.collection.mutable.ListBuffer.empty[AggregationDataMapSchema]
-      list.foreach{f =>
-        if (dataMapSchema.canSelectForRollup(f)) {
-          rollupDataMapSchema += f
-        } }
-      rollupDataMapSchema.lastOption
-    }
-  }
-
-  /**
-   * Below method will be used to validate aggregate function and get the attribute information
-   * which is applied on select query.
-   * Currently sum, max, min, count, avg is supported
-   * in case of any other aggregate function it will return empty sequence
-   * In case of avg it will return two fields one for count
-   * and other of sum of that column to support rollup
-   *
-   * @param aggExp aggregate expression
-   * @return list of fields
-   */
-  def validateAggregateFunctionAndGetFields(aggExp: AggregateExpression)
-  : Seq[AggregateExpression] = {
-    aggExp.aggregateFunction match {
-      case Sum(MatchCastExpression(exp: Expression, changeDataType: DataType)) =>
-        Seq(AggregateExpression(Sum(Cast(
-          exp,
-          changeDataType)),
-          aggExp.mode,
-          aggExp.isDistinct))
-      case Sum(_: Expression) =>
-        Seq(aggExp)
-      case Count(MatchCastExpression(exp: Seq[_], changeDataType: DataType)) =>
-        Seq(AggregateExpression(Count(Cast(
-          exp,
-          changeDataType)),
-          aggExp.mode,
-          aggExp.isDistinct))
-      case Count(_: Seq[Expression]) =>
-        Seq(aggExp)
-      case Min(MatchCastExpression(exp: Expression, changeDataType: DataType)) =>
-        Seq(AggregateExpression(Min(Cast(
-          exp,
-          changeDataType)),
-          aggExp.mode,
-          aggExp.isDistinct))
-      case Min(exp: Expression) =>
-        Seq(aggExp)
-      case Max(MatchCastExpression(exp: Expression, changeDataType: DataType)) =>
-        Seq(AggregateExpression(Max(Cast(
-          exp,
-          changeDataType)),
-          aggExp.mode,
-          aggExp.isDistinct))
-      case Max(exp: Expression) =>
-        Seq(aggExp)
-      // in case of average need to return two columns
-      // sum and count of the column to added during table creation to support rollup
-      case Average(MatchCastExpression(exp: Expression, changeDataType: DataType)) =>
-        Seq(AggregateExpression(Sum(Cast(
-          exp,
-          changeDataType)),
-          aggExp.mode,
-          aggExp.isDistinct),
-          AggregateExpression(Count(exp),
-            aggExp.mode,
-            aggExp.isDistinct))
-      // in case of average need to return two columns
-      // sum and count of the column to added during table creation to support rollup
-      case Average(exp: Expression) =>
-        Seq(AggregateExpression(Sum(exp),
-          aggExp.mode,
-          aggExp.isDistinct),
-          AggregateExpression(Count(exp),
-            aggExp.mode,
-            aggExp.isDistinct))
-      case _ =>
-        Seq.empty
-    }
-  }
-
-  /**
-   * Below method will be used to get the logical plan from aggregate expression
-   * @param aggExp aggregate expression
-   * @param tableName parent table name
-   * @param databaseName database name
-   * @param logicalRelation logical relation
-   * @return logical plan
-   */
-  def getLogicalPlanFromAggExp(aggExp: AggregateExpression,
-      tableName: String,
-      databaseName: String,
-      logicalRelation: LogicalRelation,
-      sparkSession: SparkSession,
-      parser: CarbonSpark2SqlParser): LogicalPlan = {
-    // adding the preAGG UDF, so pre aggregate data loading rule and query rule will not
-    // be applied
-    val query = parser.addPreAggFunction(s"Select ${ aggExp.sql } from $databaseName.$tableName")
-    // updating the logical relation of logical plan to so when two logical plan
-    // will be compared it will not consider relation
-    updateLogicalRelation(sparkSession.sql(query).logicalPlan, logicalRelation)
-  }
-
-  /**
-   * Below method will be used to update the logical plan of expression
-   * with parent table logical relation
-   * @param logicalPlan logial plan
-   * @param logicalRelation maintable logical relation
-   * @return updated plan
-   */
-  def updateLogicalRelation(logicalPlan: LogicalPlan,
-      logicalRelation: LogicalRelation): LogicalPlan = {
-    logicalPlan transform {
-      case l: LogicalRelation =>
-        l.copy(relation = logicalRelation.relation)
-    }
-  }
-
-  /**
-   * Normalize the exprIds in the given expression, by updating the exprId in `AttributeReference`
-   * with its referenced ordinal from input attributes. It's similar to `BindReferences` but we
-   * do not use `BindReferences` here as the plan may take the expression as a parameter with type
-   * `Attribute`, and replace it with `BoundReference` will cause error.
-   */
-  def normalizeExprId[T <: Expression](e: T, input: AttributeSeq): T = {
-    e.transformUp {
-      case ar: AttributeReference =>
-        val ordinal = input.indexOf(ar.exprId)
-        if (ordinal == -1) {
-          ar
-        } else {
-          ar.withExprId(ExprId(ordinal))
-        }
-    }.canonicalized.asInstanceOf[T]
-  }
-
-  /**
-   * Gives child query from schema
-   * @param aggDataMapSchema
-   * @return
-   */
-  def getChildQuery(aggDataMapSchema: AggregationDataMapSchema): String = {
-    new String(
-      CarbonUtil.decodeStringToBytes(
-        aggDataMapSchema.getProperties.get(DataMapProperty.CHILD_SELECT_QUERY).replace("&", "=")),
-      CarbonCommonConstants.DEFAULT_CHARSET)
-  }
-
-  /**
-   * This method will start load process on the data map
-   */
-  def createLoadCommandForChild(
-      columns: java.util.List[ColumnSchema],
-      dataMapIdentifier: TableIdentifier,
-      dataFrame: DataFrame,
-      isOverwrite: Boolean,
-      sparkSession: SparkSession,
-      options: mutable.Map[String, String],
-      timeseriesParentTableName: String = ""): CarbonLoadDataCommand = {
-    val headers = columns.asScala.filter { column =>
-      !column.getColumnName.equalsIgnoreCase(CarbonCommonConstants.DEFAULT_INVISIBLE_DUMMY_MEASURE)
-    }.sortBy(_.getSchemaOrdinal).map(_.getColumnName).mkString(",")
-    val loadCommand = CarbonLoadDataCommand(dataMapIdentifier.database,
-      dataMapIdentifier.table,
-      null,
-      Nil,
-      Map("fileheader" -> headers),
-      isOverwriteTable = isOverwrite,
-      dataFrame = None,
-      internalOptions = Map(CarbonCommonConstants.IS_INTERNAL_LOAD_CALL -> "true",
-        "timeseriesParent" -> timeseriesParentTableName),
-      logicalPlan = Some(dataFrame.queryExecution.logical))
-    loadCommand
-  }
-
-  def getDataFrame(sparkSession: SparkSession, child: LogicalPlan): DataFrame = {
-    Dataset.ofRows(sparkSession, child)
-  }
-
-}
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala
index 6e3e398..d708529 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala
@@ -81,7 +81,7 @@
       throw new MalformedCarbonCommandException("alter rename is not supported for index datamap")
     }
     // if table have create mv datamap, not support table rename
-    if (DataMapUtil.hasMVDataMap(oldCarbonTable) || oldCarbonTable.isChildTable) {
+    if (DataMapUtil.hasMVDataMap(oldCarbonTable) || oldCarbonTable.isChildTableForMV) {
       throw new MalformedCarbonCommandException(
         "alter rename is not supported for datamap table or for tables which have child datamap")
     }
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonCreateTableLikeCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonCreateTableLikeCommand.scala
index 4e8911b..006e480 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonCreateTableLikeCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonCreateTableLikeCommand.scala
@@ -48,7 +48,7 @@
     if (!srcTable.isTransactionalTable) {
       throw new MalformedCarbonCommandException("Unsupported operation on non transactional table")
     }
-    if (srcTable.isChildTable || srcTable.isChildDataMap) {
+    if (srcTable.isChildTableForMV) {
       throw new MalformedCarbonCommandException("Unsupported operation on child table or datamap")
     }
 
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDropTableCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDropTableCommand.scala
index 751f52a..ba84262 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDropTableCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDropTableCommand.scala
@@ -69,7 +69,7 @@
                 CarbonLockUtil.getLockObject(identifier, lock)
       }
       // check for directly drop datamap table
-      if (carbonTable.isChildTable && !dropChildTable) {
+      if (carbonTable.isChildTableForMV && !dropChildTable) {
         if (!ifExistsSet) {
           throwMetadataException(dbName, tableName,
             "Child table which is associated with datamap cannot be dropped, " +
@@ -91,35 +91,6 @@
       val relationIdentifiers = carbonTable.getTableInfo.getParentRelationIdentifiers
       if (relationIdentifiers != null && !relationIdentifiers.isEmpty) {
         var ignoreParentTableCheck = false
-        if (carbonTable.getTableInfo.getParentRelationIdentifiers.size() == 1) {
-          /**
-           * below handling in case when pre aggregation creation failed in scenario
-           * while creating a pre aggregate data map it created pre aggregate table and registered
-           * in hive, but failed to register in main table because of some exception.
-           * in this case if it will not allow user to drop datamap and data map table
-           * for this if user run drop table command for pre aggregate it should allow user to drop
-           * the same
-           */
-          val parentDbName =
-            carbonTable.getTableInfo.getParentRelationIdentifiers.get(0).getDatabaseName
-          val parentTableName =
-            carbonTable.getTableInfo.getParentRelationIdentifiers.get(0).getTableName
-          val parentCarbonTable = try {
-            Some(CarbonEnv.getCarbonTable(Some(parentDbName), parentTableName)(sparkSession))
-          } catch {
-            case _: Exception => None
-          }
-          if (parentCarbonTable.isDefined) {
-            val dataMapSchemaName = CarbonUtil.getDatamapNameFromTableName(carbonTable.getTableName)
-            if (null != dataMapSchemaName) {
-              val dataMapSchema = parentCarbonTable.get.getDataMapSchema(dataMapSchemaName)
-              if (null == dataMapSchema) {
-                LOGGER.info(s"Force dropping datamap ${carbonTable.getTableName}")
-                ignoreParentTableCheck = true
-              }
-            }
-          }
-        }
         if (!ignoreParentTableCheck && !dropChildTable) {
           if (!ifExistsSet) {
             throwMetadataException(dbName, tableName,
@@ -223,7 +194,7 @@
 
   override def processData(sparkSession: SparkSession): Seq[Row] = {
     // clear driver side index and dictionary cache
-    if (carbonTable != null && !(carbonTable.isChildTable && !dropChildTable)) {
+    if (carbonTable != null && !(carbonTable.isChildTableForMV && !dropChildTable)) {
       ManageDictionaryAndBTree.clearBTreeAndDictionaryLRUCache(carbonTable)
       // delete the table folder
       val tablePath = carbonTable.getTablePath
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/timeseries/TimeSeriesFunction.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/timeseries/TimeSeriesFunction.scala
index ad9ace7..3ec02c9 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/timeseries/TimeSeriesFunction.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/timeseries/TimeSeriesFunction.scala
@@ -24,10 +24,9 @@
  * Time series udf class
  */
 
-class TimeSeriesFunction extends Function2[Timestamp, String, Timestamp] with Serializable{
+class TimeSeriesFunction extends ((Timestamp, String) => Timestamp) with Serializable{
 
-  override def apply(v1: Timestamp,
-      v2: String): Timestamp = {
+  override def apply(v1: Timestamp, v2: String): Timestamp = {
     TimeSeriesUDF.INSTANCE.applyUDF(v1, v2)
   }
 }
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/timeseries/TimeSeriesUtil.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/timeseries/TimeSeriesUtil.scala
index 9ebbcde..e72cf9b 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/timeseries/TimeSeriesUtil.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/timeseries/TimeSeriesUtil.scala
@@ -24,7 +24,6 @@
 import org.apache.carbondata.common.exceptions.sql.{MalformedCarbonCommandException, MalformedDataMapCommandException}
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.metadata.datatype.DataTypes
-import org.apache.carbondata.core.metadata.schema.datamap.DataMapClassProvider.TIMESERIES
 import org.apache.carbondata.core.metadata.schema.datamap.Granularity
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 
@@ -81,19 +80,7 @@
         }
       }
     }
-
-    // 2. check whether timeseries and granularity match
-    if (isFound && !dmClassName.equalsIgnoreCase(TIMESERIES.toString)) {
-      throw new MalformedDataMapCommandException(
-        s"${TIMESERIES.toString} keyword missing")
-    } else if (!isFound && dmClassName.equalsIgnoreCase(TIMESERIES.toString)) {
-      throw new MalformedDataMapCommandException(
-        s"${TIMESERIES.toString} should define time granularity")
-    } else if (isFound) {
-      true
-    } else {
-      false
-    }
+    isFound
   }
 
   /**
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 d51e85a..7c8993e 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
@@ -309,7 +309,7 @@
               throw new MalformedCarbonCommandException(
                 "The table which has MV datamap does not support set streaming property")
             }
-            if (carbonTable.isChildTable) {
+            if (carbonTable.isChildTableForMV) {
               throw new MalformedCarbonCommandException(
                 "Datamap table does not support set streaming property")
             }
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
index d11bf1e..0c6b5aa 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
@@ -60,15 +60,6 @@
       val projList = Seq(UnresolvedAlias(UnresolvedStar(alias.map(Seq(_)))), tupleId)
       val carbonTable = CarbonEnv.getCarbonTable(table.tableIdentifier)(sparkSession)
       if (carbonTable != null) {
-        if (CarbonUtil.hasAggregationDataMap(carbonTable)) {
-          throw new UnsupportedOperationException(
-            "Update operation is not supported for tables which have a pre-aggregate table. " +
-            "Drop pre-aggregate tables to continue.")
-        }
-        if (carbonTable.isChildDataMap) {
-          throw new UnsupportedOperationException(
-            "Update operation is not supported for pre-aggregate table")
-        }
         val indexSchemas = DataMapStoreManager.getInstance().getDataMapSchemasOfTable(carbonTable)
         if (DataMapUtil.hasMVDataMap(carbonTable)) {
           val allDataMapSchemas = DataMapStoreManager.getInstance
@@ -82,7 +73,7 @@
           throw new UnsupportedOperationException(
             "Update operation is not supported for table which has index datamaps")
         }
-        if (carbonTable.isChildTable) {
+        if (carbonTable.isChildTableForMV) {
           throw new UnsupportedOperationException(
             "Update operation is not supported for mv datamap table")
         }
@@ -211,7 +202,7 @@
         val projList = Seq(UnresolvedAlias(UnresolvedStar(alias.map(Seq(_)))), tupleId)
         val carbonTable = CarbonEnv.getCarbonTable(table.tableIdentifier)(sparkSession)
         if (carbonTable != null) {
-          if (carbonTable.isChildTable) {
+          if (carbonTable.isChildTableForMV) {
             throw new UnsupportedOperationException(
               "Delete operation is not supported for datamap table")
           }
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 6c9c6cd..57755da 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
@@ -69,11 +69,11 @@
   }
 }
 
-case class CarbonMetaData(dims: Seq[String],
+case class CarbonMetaData(
+    dims: Seq[String],
     msrs: Seq[String],
     carbonTable: CarbonTable,
-    dictionaryMap: DictionaryMap,
-    hasAggregateDataMapSchema: Boolean)
+    dictionaryMap: DictionaryMap)
 
 case class DictionaryMap(dictionaryMap: Map[String, Boolean]) {
   def get(name: String): Option[Boolean] = {
@@ -355,23 +355,6 @@
     path
   }
 
-  override def revertTableSchemaForPreAggCreationFailure(
-      absoluteTableIdentifier: AbsoluteTableIdentifier,
-      thriftTableInfo: org.apache.carbondata.format.TableInfo)
-    (sparkSession: SparkSession): String = {
-    val schemaConverter = new ThriftWrapperSchemaConverterImpl
-    val wrapperTableInfo = schemaConverter.fromExternalToWrapperTableInfo(
-      thriftTableInfo,
-      absoluteTableIdentifier.getCarbonTableIdentifier.getDatabaseName,
-      absoluteTableIdentifier.getCarbonTableIdentifier.getTableName,
-      absoluteTableIdentifier.getTablePath)
-    val childSchemaList = wrapperTableInfo.getDataMapSchemaList
-    childSchemaList.remove(childSchemaList.size() - 1)
-    val path = createSchemaThriftFile(absoluteTableIdentifier, thriftTableInfo)
-    addCarbonTableToCache(wrapperTableInfo, absoluteTableIdentifier)
-    path
-  }
-
   /**
    *
    * Prepare Thrift Schema from wrapper TableInfo and write to Schema file.
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 a0a8c0f..9067d28 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
@@ -208,19 +208,4 @@
       schemaConverter)
   }
 
-  override def revertTableSchemaForPreAggCreationFailure(absoluteTableIdentifier:
-  AbsoluteTableIdentifier,
-      thriftTableInfo: org.apache.carbondata.format.TableInfo)
-    (sparkSession: SparkSession): String = {
-    val schemaConverter = new ThriftWrapperSchemaConverterImpl
-    val childSchemas = thriftTableInfo.dataMapSchemas
-    childSchemas.remove(childSchemas.size() - 1)
-    val carbonTableIdentifier = absoluteTableIdentifier.getCarbonTableIdentifier
-    updateHiveMetaStoreForAlter(carbonTableIdentifier,
-      carbonTableIdentifier,
-      thriftTableInfo,
-      absoluteTableIdentifier.getTablePath,
-      sparkSession,
-      schemaConverter)
-  }
 }
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 95efa6a..6003f81 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
@@ -101,9 +101,6 @@
     (sparkSession: SparkSession): String
 
 
-  def revertTableSchemaForPreAggCreationFailure(absoluteTableIdentifier: AbsoluteTableIdentifier,
-      thriftTableInfo: org.apache.carbondata.format.TableInfo)(sparkSession: SparkSession): String
-
   /**
    * Prepare Thrift Schema from wrapper TableInfo and write to disk
    */
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonPreAggregateRules.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonPreAggregateRules.scala
deleted file mode 100644
index b616c20..0000000
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonPreAggregateRules.scala
+++ /dev/null
@@ -1,1915 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.hive
-
-import scala.collection.JavaConverters._
-import scala.collection.mutable
-import scala.collection.mutable.ArrayBuffer
-
-import org.apache.spark.sql._
-import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.CarbonExpressions.{CarbonScalaUDF, CarbonSubqueryAlias, MatchCastExpression}
-import org.apache.spark.sql.catalyst.analysis.{UnresolvedAlias, UnresolvedAttribute}
-import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, Cast, Divide, Expression, Literal, NamedExpression, ScalaUDF, SortOrder}
-import org.apache.spark.sql.catalyst.expressions.aggregate.{Count, _}
-import org.apache.spark.sql.catalyst.plans.logical._
-import org.apache.spark.sql.catalyst.rules.Rule
-import org.apache.spark.sql.execution.command.preaaggregate.PreAggregateUtil
-import org.apache.spark.sql.execution.datasources.{FindDataSourceTable, LogicalRelation}
-import org.apache.spark.sql.parser.CarbonSpark2SqlParser
-import org.apache.spark.sql.types._
-import org.apache.spark.util.CarbonReflectionUtils
-
-import org.apache.carbondata.core.constants.{CarbonCommonConstants, CarbonCommonConstantsInternal}
-import org.apache.carbondata.core.metadata.schema.datamap.DataMapClassProvider
-import org.apache.carbondata.core.metadata.schema.table.{AggregationDataMapSchema, CarbonTable, DataMapSchema}
-import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema
-import org.apache.carbondata.core.preagg.{AggregateQueryPlan, AggregateTableSelector, QueryColumn}
-import org.apache.carbondata.core.profiler.ExplainCollector
-import org.apache.carbondata.core.statusmanager.SegmentStatusManager
-import org.apache.carbondata.core.util.{CarbonUtil, ThreadLocalSessionInfo}
-
-/**
- * model class to store aggregate expression logical plan
- * and its column schema mapping
- * @param expression aggregate expression
- * @param columnSchema list of column schema from table
- */
-case class AggExpToColumnMappingModel(
-    expression: Expression,
-    var columnSchema: Option[Object] = None) {
-  override def equals(o: Any) : Boolean = o match {
-    case that: AggExpToColumnMappingModel =>
-      that.expression==this.expression
-    case _ => false
-  }
-  // TODO need to update the hash code generation code
-  override def hashCode : Int = 1
-}
-/**
- * Class for applying Pre Aggregate rules
- * Responsibility.
- * 1. Check plan is valid plan for updating the parent table plan with child table
- * 2. Updated the plan based on child schema
- *
- * Rules for Updating the plan
- * 1. Grouping expression rules
- *    1.1 Change the parent attribute reference for of group expression
- * to child attribute reference
- *
- * 2. Aggregate expression rules
- *    2.1 Change the parent attribute reference for of group expression to
- * child attribute reference
- *    2.2 Change the count AggregateExpression to Sum as count
- * is already calculated so in case of aggregate table
- * we need to apply sum to get the count
- *    2.2 In case of average aggregate function select 2 columns from aggregate table with
- * aggregation
- * sum and count. Then add divide(sum(column with sum), sum(column with count)).
- * Note: During aggregate table creation for average table will be created with two columns
- * one for sum(column) and count(column) to support rollup
- *
- * 3. Filter Expression rules.
- *    3.1 Updated filter expression attributes with child table attributes
- * 4. Update the Parent Logical relation with child Logical relation
- * 5. Order By Query rules.
- *    5.1 Update project list based on updated aggregate expression
- *    5.2 Update sort order attributes based on pre aggregate table
- * 6. timeseries function
- *    6.1 validate maintable has timeseries datamap
- *    6.2 timeseries function is valid function or not
- * 7. Streaming
- * Examples1:
- * Query:
- *   SELECT name, sum(Salary) as totalSalary
- *   FROM maintable.
- * UpdatedQuery:
- *   SELECT name, sum(totalSalary) FROM(
- *          SELECT name, sum(Salary) as totalSalary
- *          FROM maintable
- *          GROUP BY name
- *          UNION ALL
- *          SELECT maintable_name,sum(maintable_salary) as totalSalary
- *          FROM maintable_agg
- *          GROUP BY maintable_name)
- *   GROUP BY name)
- * Example2:
- * Query:
- *   SELECT name, AVG(Salary) as avgSalary
- *        FROM maintable.
- * UpdatedQuery:
- *   SELECT name, Divide(sum(sumSalary)/sum(countsalary))
- *   FROM(
- *    SELECT name, sum(Salary) as sumSalary,count(salary) countsalary
- *      FROM maintable
- *      GROUP BY name
- *    UNION ALL
- *    SELECT maintable_name,sum(maintable_salary) as sumSalary, count(maintable_salary) countsalary
- *      FROM maintable_agg
- *      GROUP BY maintable_name)
- *   GROUP BY name)
- *
- * Rules for updating plan in case of streaming table:
- * In case of streaming data will be fetched from both fact and aggregate as aggregate table
- * will be updated only after each hand-off, so current streamed data won't be available on
- * aggregate table.
- * 7.1 Add one union node to add both fact and aggregate table plan to get the data from both table
- * 7.2 On top of Union Node add one Aggregate node to aggregate both table results
- * 7.3 In case of average(avg(column)) special handling is required for streaming
- *     7.3.1 Fact Plan will updated to return sum(column) and count(column) to do rollup
- *     7.3.2 Aggregate Plan will updated to return sum(column) and count(column) to do rollup
- * 7.4 In newly added Aggregate node all the aggregate expression must have same expression id as
- *     fact and fact plan will updated with new expression id. As query like order by this can be
- *     referred. In example1 sum(totalSalary) as totalSalary will have same expression id
- *     as in fact and fact plan sum(salary) will be updated with new expression id
- *
- * @param sparkSession spark session
- */
-case class CarbonPreAggregateQueryRules(sparkSession: SparkSession) extends Rule[LogicalPlan] {
-
-  /**
-   * map for keeping parent attribute reference to child attribute reference
-   * this will be used to updated the plan in case of join or order by
-   */
-  val updatedExpression = mutable.HashMap[AttributeReference, AttributeReference]()
-
-  /**
-   * parser
-   */
-  lazy val parser = new CarbonSpark2SqlParser
-
-  /**
-   * Below method will be used to validate the logical plan
-   * @param logicalPlan query logical plan
-   * @return isvalid or not
-   */
-  private def isValidPlan(logicalPlan: LogicalPlan) : Boolean = {
-    var isValidPlan = true
-    logicalPlan.transform {
-      case aggregate@Aggregate(grp, aExp, child) =>
-        isValidPlan = !aExp.exists { p =>
-          if (p.isInstanceOf[UnresolvedAlias]) return false
-          p.name.equals("preAggLoad") || p.name.equals("preAgg")
-        }
-        val updatedAggExp = aExp.filterNot(_.name.equalsIgnoreCase("preAggLoad"))
-        Aggregate(grp, updatedAggExp, child)
-    }
-    isValidPlan
-  }
-  override def apply(plan: LogicalPlan): LogicalPlan = {
-    var needAnalysis = true
-    plan.transformExpressions {
-      // first check if any preAgg scala function is applied it is present is in plan
-      // then call is from create preaggregate table class so no need to transform the query plan
-      case al@Alias(udf: ScalaUDF, name) if name.equalsIgnoreCase("preAgg") =>
-        needAnalysis = false
-        al
-      case al@Alias(udf: ScalaUDF, name) if name.equalsIgnoreCase("preAggLoad") =>
-        needAnalysis = false
-        al
-      // in case of query if any unresolve alias is present then wait for plan to be resolved
-      // return the same plan as we can tranform the plan only when everything is resolved
-      case unresolveAlias@UnresolvedAlias(_, _) =>
-        needAnalysis = false
-        unresolveAlias
-      case attr@UnresolvedAttribute(_) =>
-        needAnalysis = false
-        attr
-    }
-    if(needAnalysis) {
-      needAnalysis = isValidPlan(plan)
-      if(needAnalysis) {
-        needAnalysis = validateStreamingTablePlan(plan)
-      }
-    }
-    // if plan is not valid for transformation then return same plan
-    if (!needAnalysis) {
-      plan
-    } else {
-      val updatedPlan = transformPreAggQueryPlan(plan)
-      val newPlan = updatePlan(updatedPlan)
-      newPlan
-    }
-
-  }
-
-  /**
-   * Below method will be used validate whether plan is already updated in case of streaming table
-   * In case of streaming table it will add UnionNode to get the data from fact and aggregate both
-   * as aggregate table will be updated after each handoff.
-   * So if plan is already updated no need to transform the plan again
-   * @param logicalPlan
-   * query plan
-   * @return whether need to update the query plan or not
-   */
-  def validateStreamingTablePlan(logicalPlan: LogicalPlan) : Boolean = {
-    var needTransformation: Boolean = true
-    logicalPlan.transform {
-      case union @ Union(Seq(plan1, plan2)) =>
-        plan2.collect{
-          case logicalRelation: LogicalRelation if
-          logicalRelation.relation.isInstanceOf[CarbonDatasourceHadoopRelation] &&
-          logicalRelation.relation.asInstanceOf[CarbonDatasourceHadoopRelation].carbonTable
-            .isChildDataMap =>
-            needTransformation = false
-        }
-        union
-    }
-    needTransformation
-  }
-
-  /**
-   * Below method will be used to update the child plan
-   * This will be used for updating expression like join condition,
-   * order by, project list etc
-   * @param plan child plan
-   * @return updated plan
-   */
-  def updatePlan(plan: LogicalPlan) : LogicalPlan = {
-    val updatedPlan = plan transform {
-      case Aggregate(grp, aggExp, child) =>
-        Aggregate(
-          updateExpression(grp),
-          updateExpression(aggExp.asInstanceOf[Seq[Expression]]).asInstanceOf[Seq[NamedExpression]],
-          child)
-      case Filter(filterExp, child) =>
-        Filter(updateExpression(Seq(filterExp)).head, child)
-      case Project(pList, child) =>
-        Project(
-          updateExpression(pList.asInstanceOf[Seq[Expression]]).asInstanceOf[Seq[NamedExpression]],
-          child)
-      case Sort(sortOrders, global, child) =>
-        Sort(updateSortExpression(sortOrders), global, child)
-      case Join(left, right, joinType, condition) =>
-        val updatedCondition = condition match {
-          case Some(expr) => Some(updateExpression(Seq(expr)).head)
-          case _ => condition
-        }
-        Join(left, right, joinType, updatedCondition)
-    }
-    updatedPlan
-  }
-
-  /**
-   * Below method will be used to update the sort expression
-   * @param sortExp sort order expression in query
-   * @return updated sort expression
-   */
-  def updateSortExpression(sortExp : Seq[SortOrder]) : Seq[SortOrder] = {
-    sortExp map { order =>
-      SortOrder(order.child transform  {
-        case attr: AttributeReference =>
-          updatedExpression.find { p => p._1.sameRef(attr) } match {
-            case Some((_, childAttr)) =>
-              CarbonToSparkAdapter.createAttributeReference(
-                childAttr.name,
-                childAttr.dataType,
-                childAttr.nullable,
-                childAttr.metadata,
-                childAttr.exprId,
-                attr.qualifier,
-                attr)
-            case None =>
-              attr
-          }
-      }, order.direction )
-    }
-  }
-
-  /**
-   * Below method will be used to update the expression like group by expression
-   * @param expressions sequence of expression like group by
-   * @return updated expressions
-   */
-  def updateExpression(expressions : Seq[Expression]) : Seq[Expression] = {
-    expressions map { expression =>
-      expression transform {
-        case attr: AttributeReference =>
-          updatedExpression.find { p => p._1.sameRef(attr) } match {
-            case Some((_, childAttr)) =>
-              CarbonToSparkAdapter.createAttributeReference(
-                childAttr.name,
-                childAttr.dataType,
-                childAttr.nullable,
-                childAttr.metadata,
-                childAttr.exprId,
-                attr.qualifier,
-                attr)
-            case None =>
-              attr
-          }
-      }
-    }
-  }
-
-  /**
-   * Below method will be used to validate and transform the main table plan to child table plan
-   * rules for transforming is as below.
-   * 1. Grouping expression rules
-   *    1.1 Change the parent attribute reference for of group expression
-   * to child attribute reference
-   *
-   * 2. Aggregate expression rules
-   *    2.1 Change the parent attribute reference for of group expression to
-   * child attribute reference
-   *    2.2 Change the count AggregateExpression to Sum as count
-   * is already calculated so in case of aggregate table
-   * we need to apply sum to get the count
-   *    2.2 In case of average aggregate function select 2 columns from aggregate table with
-   * aggregation sum and count. Then add divide(sum(column with sum), sum(column with count)).
-   * Note: During aggregate table creation for average table will be created with two columns
-   * one for sum(column) and count(column) to support rollup
-   * 3. Filter Expression rules.
-   *    3.1 Updated filter expression attributes with child table attributes
-   * 4. Update the Parent Logical relation with child Logical relation
-   * 5. timeseries function
-   *    5.1 validate parent table has timeseries datamap
-   *    5.2 timeseries function is valid function or not
-   * 6. Streaming
-   * Rules for updating plan in case of streaming table:
-   * In case of streaming data will be fetched from both fact and aggregate as aggregate table
-   * will be updated only after each hand-off, so current streamed data won't be available on
-   * aggregate table.
-   * 6.1 Add one union node to add both fact and aggregate table plan to
-   *     get the data from both table
-   * 6.2 On top of Union Node add one Aggregate node to aggregate both table results
-   * 6.3 In case of average(avg(column)) special handling is required for streaming
-   *     7.3.1 Fact Plan will updated to return sum(column) and count(column) to do rollup
-   *     7.3.2 Aggregate Plan will updated to return sum(column) and count(column) to do rollup
-   * 6.4 In newly added Aggregate node all the aggregate expression must have same expression id as
-   *     fact and fact plan will updated with new expression id. As query like order by this can be
-   *     referred. In example1 sum(totalSalary) as totalSalary will have same expression id
-   *     as in fact and fact plan sum(salary) will be updated with new expression id
-   *
-   * @param logicalPlan parent logical plan
-   * @return transformed plan
-   */
-  def transformPreAggQueryPlan(logicalPlan: LogicalPlan): LogicalPlan = {
-    var isPlanUpdated = false
-    val updatedPlan = logicalPlan.transform {
-      case agg@Aggregate(
-        grExp,
-        aggExp,
-        CarbonSubqueryAlias(alias1, child@CarbonSubqueryAlias(alias2, l: LogicalRelation)))
-        if l.relation.isInstanceOf[CarbonDatasourceHadoopRelation] &&
-           l.relation.asInstanceOf[CarbonDatasourceHadoopRelation].carbonRelation.
-             metaData.hasAggregateDataMapSchema && !isPlanUpdated =>
-        val carbonTable = getCarbonTable(l)
-        if (isSpecificSegmentNotPresent(carbonTable)) {
-          val list = scala.collection.mutable.HashSet.empty[QueryColumn]
-          val aggregateExpressions = scala.collection.mutable.HashSet.empty[AggregateExpression]
-          val isValidPlan = extractQueryColumnsFromAggExpression(
-            grExp,
-            aggExp,
-            carbonTable,
-            list,
-            aggregateExpressions)
-          if (isValidPlan) {
-            val (aggDataMapSchema, childPlan) = getChildDataMapForTransformation(list,
-              aggregateExpressions,
-              carbonTable,
-              agg)
-            if (null != aggDataMapSchema && null != childPlan) {
-              val attributes = childPlan.output.asInstanceOf[Seq[AttributeReference]]
-              val (updatedGroupExp, updatedAggExp, newChild, None) =
-                getUpdatedExpressions(grExp,
-                  aggExp,
-                  child,
-                  None,
-                  aggDataMapSchema,
-                  attributes,
-                  childPlan,
-                  carbonTable,
-                  agg)
-              isPlanUpdated = true
-              setExplain(aggDataMapSchema)
-              val updateAggPlan =
-                Aggregate(
-                updatedGroupExp,
-                updatedAggExp,
-                CarbonReflectionUtils.getSubqueryAlias(
-                  sparkSession,
-                  Some(alias1),
-                  CarbonReflectionUtils.getSubqueryAlias(
-                    sparkSession,
-                    Some(alias2),
-                    newChild,
-                    None),
-                  None))
-              getAggregateQueryPlan(
-                updateAggPlan,
-                grExp,
-                aggExp,
-                carbonTable,
-                aggDataMapSchema,
-                agg)
-            } else {
-              agg
-            }
-          } else {
-            agg
-          }
-        } else {
-          agg
-        }
-      // case for aggregation query
-      case agg@Aggregate(
-      grExp,
-      aggExp,
-      child@CarbonSubqueryAlias(alias, l: LogicalRelation))
-        if l.relation.isInstanceOf[CarbonDatasourceHadoopRelation] &&
-           l.relation.asInstanceOf[CarbonDatasourceHadoopRelation].carbonRelation.
-             metaData.hasAggregateDataMapSchema && !isPlanUpdated =>
-        val carbonTable = getCarbonTable(l)
-        if(isSpecificSegmentNotPresent(carbonTable)) {
-          val list = scala.collection.mutable.HashSet.empty[QueryColumn]
-          val aggregateExpressions = scala.collection.mutable.HashSet.empty[AggregateExpression]
-          val isValidPlan = extractQueryColumnsFromAggExpression(
-            grExp,
-            aggExp,
-            carbonTable,
-            list,
-            aggregateExpressions)
-          if (isValidPlan) {
-            val (aggDataMapSchema, childPlan) = getChildDataMapForTransformation(list,
-              aggregateExpressions,
-              carbonTable,
-              agg)
-            if (null != aggDataMapSchema && null != childPlan) {
-              val attributes = childPlan.output.asInstanceOf[Seq[AttributeReference]]
-              val (updatedGroupExp, updatedAggExp, newChild, None) =
-                getUpdatedExpressions(grExp,
-                  aggExp,
-                  child,
-                  None,
-                  aggDataMapSchema,
-                  attributes,
-                  childPlan,
-                  carbonTable,
-                  agg)
-              isPlanUpdated = true
-              setExplain(aggDataMapSchema)
-              val updateAggPlan =
-                Aggregate(
-                updatedGroupExp,
-                updatedAggExp,
-                CarbonReflectionUtils.getSubqueryAlias(
-                  sparkSession,
-                  Some(alias),
-                  newChild,
-                  None))
-              getAggregateQueryPlan(
-                updateAggPlan,
-                grExp,
-                aggExp,
-                carbonTable,
-                aggDataMapSchema,
-                agg)
-            } else {
-              agg
-            }
-          } else {
-            agg
-          }
-        } else {
-          agg
-        }
-      // case of handling aggregation query with filter
-      case agg@Aggregate(
-      grExp,
-      aggExp,
-      Filter(expression, child@CarbonSubqueryAlias(alias, l: LogicalRelation)))
-        if l.relation.isInstanceOf[CarbonDatasourceHadoopRelation] &&
-           l.relation.asInstanceOf[CarbonDatasourceHadoopRelation].carbonRelation.
-             metaData.hasAggregateDataMapSchema && !isPlanUpdated =>
-        val carbonTable = getCarbonTable(l)
-        if(isSpecificSegmentNotPresent(carbonTable)) {
-          val list = scala.collection.mutable.HashSet.empty[QueryColumn]
-          val aggregateExpressions = scala.collection.mutable.HashSet.empty[AggregateExpression]
-          var isValidPlan = extractQueryColumnsFromAggExpression(
-            grExp,
-            aggExp,
-            carbonTable,
-            list,
-            aggregateExpressions)
-          if (isValidPlan) {
-            isValidPlan = !CarbonReflectionUtils.hasPredicateSubquery(expression)
-          }
-          // getting the columns from filter expression
-          if (isValidPlan) {
-            extractColumnFromExpression(expression, list, carbonTable, true)
-          }
-          if (isValidPlan) {
-            val (aggDataMapSchema, childPlan) = getChildDataMapForTransformation(list,
-              aggregateExpressions,
-              carbonTable,
-              agg)
-            if (null != aggDataMapSchema && null != childPlan) {
-              val attributes = childPlan.output.asInstanceOf[Seq[AttributeReference]]
-              val (updatedGroupExp, updatedAggExp, newChild, updatedFilterExpression) =
-                getUpdatedExpressions(grExp,
-                  aggExp,
-                  child,
-                  Some(expression),
-                  aggDataMapSchema,
-                  attributes,
-                  childPlan,
-                  carbonTable,
-                  agg)
-              isPlanUpdated = true
-              setExplain(aggDataMapSchema)
-              val updateAggPlan =
-                Aggregate(
-                updatedGroupExp,
-                updatedAggExp,
-                Filter(
-                  updatedFilterExpression.get,
-                  CarbonReflectionUtils.getSubqueryAlias(
-                    sparkSession,
-                    Some(alias),
-                    newChild,
-                    None)))
-              getAggregateQueryPlan(
-                updateAggPlan,
-                grExp,
-                aggExp,
-                carbonTable,
-                aggDataMapSchema,
-                agg)
-            } else {
-              agg
-            }
-          } else {
-            agg
-          }
-        } else {
-          agg
-        }
-      case agg@Aggregate(
-      grExp,
-      aggExp,
-      Filter(
-      expression,
-      CarbonSubqueryAlias(alias1, child@CarbonSubqueryAlias(alias2, l: LogicalRelation))))
-        if l.relation.isInstanceOf[CarbonDatasourceHadoopRelation] &&
-           l.relation.asInstanceOf[CarbonDatasourceHadoopRelation].carbonRelation.
-             metaData.hasAggregateDataMapSchema && !isPlanUpdated =>
-        val carbonTable = getCarbonTable(l)
-        if(isSpecificSegmentNotPresent(carbonTable)) {
-          val list = scala.collection.mutable.HashSet.empty[QueryColumn]
-          val aggregateExpressions = scala.collection.mutable.HashSet.empty[AggregateExpression]
-          var isValidPlan = extractQueryColumnsFromAggExpression(
-            grExp,
-            aggExp,
-            carbonTable,
-            list,
-            aggregateExpressions)
-          if (isValidPlan) {
-            isValidPlan = !CarbonReflectionUtils.hasPredicateSubquery(expression)
-          }
-          // getting the columns from filter expression
-          if (isValidPlan) {
-            extractColumnFromExpression(expression, list, carbonTable, true)
-          }
-          if (isValidPlan) {
-            val (aggDataMapSchema, childPlan) = getChildDataMapForTransformation(list,
-              aggregateExpressions,
-              carbonTable,
-              agg)
-            if (null != aggDataMapSchema && null != childPlan) {
-              val attributes = childPlan.output.asInstanceOf[Seq[AttributeReference]]
-              val (updatedGroupExp, updatedAggExp, newChild, updatedFilterExpression) =
-                getUpdatedExpressions(grExp,
-                  aggExp,
-                  child,
-                  Some(expression),
-                  aggDataMapSchema,
-                  attributes,
-                  childPlan,
-                  carbonTable,
-                  agg)
-              isPlanUpdated = true
-              setExplain(aggDataMapSchema)
-              val updateAggPlan =
-                Aggregate(
-                  updatedGroupExp,
-                  updatedAggExp,
-                  Filter(
-                    updatedFilterExpression.get,
-                    CarbonReflectionUtils.getSubqueryAlias(
-                      sparkSession,
-                      Some(alias1),
-                      CarbonReflectionUtils.getSubqueryAlias(
-                        sparkSession,
-                        Some(alias2),
-                        newChild,
-                        None),
-                      None)))
-              getAggregateQueryPlan(
-                updateAggPlan,
-                grExp,
-                aggExp,
-                carbonTable,
-                aggDataMapSchema,
-                agg)
-            } else {
-              agg
-            }
-          } else {
-            agg
-          }
-        } else {
-          agg
-        }
-
-    }
-    if(isPlanUpdated) {
-      CarbonSession.threadSet(CarbonCommonConstants.SUPPORT_DIRECT_QUERY_ON_DATAMAP, "true")
-    }
-    updatedPlan
-  }
-
-  // set datamap match information for EXPLAIN command
-  private def setExplain(dataMapSchema: AggregationDataMapSchema): Unit = {
-    ExplainCollector.recordMatchedOlapDataMap(
-      dataMapSchema.getProvider.getShortName, dataMapSchema.getDataMapName)
-  }
-
-  /**
-   * Method to get the aggregate query plan
-   * @param aggPlan
-   * aggregate table query plan
-   * @param grExp
-   * fact group by expression
-   * @param aggExp
-   * fact aggregate expression
-   * @param carbonTable
-   * fact table
-   * @param aggregationDataMapSchema
-   * selected aggregation data map
-   * @param factAggPlan
-   * fact aggregate query plan
-   * @return updated plan
-   */
-  def getAggregateQueryPlan(aggPlan: LogicalPlan,
-      grExp: Seq[Expression],
-      aggExp: Seq[NamedExpression],
-      carbonTable: CarbonTable,
-      aggregationDataMapSchema: DataMapSchema,
-      factAggPlan: LogicalPlan): LogicalPlan = {
-    // to handle streaming table with pre aggregate
-    if (carbonTable.isStreamingSink) {
-      setSegmentsForStreaming(carbonTable, aggregationDataMapSchema)
-      // get new fact expression
-      val factExp = updateFactTablePlanForStreaming(factAggPlan)
-      // get new Aggregate node expression
-      val aggPlanNew = updateAggTablePlanForStreaming(aggPlan)
-      val streamingNodeExp = getExpressionsForStreaming(aggExp)
-      // clear the expression as in case of streaming it is not required
-      updatedExpression.clear
-      // Add Aggregate node to aggregate data from fact and aggregate
-      Aggregate(
-        createNewAggGroupBy(grExp, factAggPlan),
-        streamingNodeExp.asInstanceOf[Seq[NamedExpression]],
-        // add union node to get the result from both
-        Union(
-          factExp,
-      aggPlanNew))
-    } else {
-      aggPlan
-    }
-  }
-
-  /**
-   * create group by expression for newly Added Aggregate node
-   * @param grExp fact group by expression
-   * @param plan fact query plan
-   * @return group by expression
-   */
-  private def createNewAggGroupBy(grExp: Seq[Expression], plan: LogicalPlan): Seq[Expression] = {
-    grExp.map {
-      case attr: AttributeReference =>
-        val aggModel = AggExpToColumnMappingModel(
-          removeQualifiers(PreAggregateUtil.normalizeExprId(attr, plan.allAttributes)))
-        if (factPlanGrpExpForStreaming.get(aggModel).isDefined) {
-          factPlanGrpExpForStreaming.get(aggModel).get
-        } else {
-          attr
-        }
-      case exp: Expression =>
-        val aggModel = AggExpToColumnMappingModel(
-          removeQualifiers(PreAggregateUtil.normalizeExprId(exp, plan.allAttributes)))
-        factPlanGrpExpForStreaming.get(aggModel).get
-    }
-  }
-  /**
-   * Method to set the segments when query is fired on streaming table with pre aggregate
-   * Adding a property streaming_seg so while removing from session params we can differentiate
-   * it was set from CarbonPreAggregateRules
-   * @param parentTable
-   * parent arbon table
-   * @param dataMapSchema
-   * child datamap schema
-   */
-  def setSegmentsForStreaming(parentTable: CarbonTable, dataMapSchema: DataMapSchema): Unit = {
-    val mainTableKey = parentTable.getDatabaseName + '.' + parentTable.getTableName
-    val factManager = new SegmentStatusManager(parentTable.getAbsoluteTableIdentifier)
-    CarbonSession
-      .threadSet(CarbonCommonConstantsInternal.QUERY_ON_PRE_AGG_STREAMING + mainTableKey, "true")
-    CarbonSession
-      .threadSet(
-        CarbonCommonConstants.CARBON_INPUT_SEGMENTS + mainTableKey,
-        factManager.getValidAndInvalidSegments.getValidSegments.asScala.mkString(","))
-    CarbonSession
-      .threadSet(CarbonCommonConstants.VALIDATE_CARBON_INPUT_SEGMENTS + mainTableKey, "true")
-    // below code is for aggregate table
-    val identifier = TableIdentifier(
-      dataMapSchema.getChildSchema.getTableName,
-      Some(parentTable.getDatabaseName))
-    val catalog = CarbonEnv.getInstance(sparkSession).carbonMetaStore
-    val carbonRelation =
-      catalog.lookupRelation(identifier)(sparkSession).asInstanceOf[CarbonRelation]
-    val segmentStatusManager = new SegmentStatusManager(carbonRelation.carbonTable
-      .getAbsoluteTableIdentifier)
-    val validSegments = segmentStatusManager.getValidAndInvalidSegments.getValidSegments.asScala
-      .mkString(",")
-    val childTableKey = carbonRelation.carbonTable.getDatabaseName + '.' +
-                   carbonRelation.carbonTable.getTableName
-    CarbonSession
-      .threadSet(CarbonCommonConstantsInternal.QUERY_ON_PRE_AGG_STREAMING + childTableKey, "true")
-    CarbonSession
-      .threadSet(CarbonCommonConstants.CARBON_INPUT_SEGMENTS + childTableKey, validSegments)
-    CarbonSession
-      .threadSet(CarbonCommonConstants.VALIDATE_CARBON_INPUT_SEGMENTS + childTableKey, "false")
-  }
-
-  /**
-   * Map to keep expression name to its alias mapping. This will be used while adding a node when
-   * plan for streaming table is updated.
-   * Note: In case of average fact table plan will have two alias as sum(column) and count(column)
-   * to support rollup
-   */
-  private val factPlanExpForStreaming = mutable.HashMap[String, Seq[NamedExpression]]()
-
-  private val factPlanGrpExpForStreaming = mutable
-    .HashMap[AggExpToColumnMappingModel, AttributeReference]()
-
-  /**
-   * Below method will be used to get the expression for Aggregate node added for streaming
-   * Expression id will be same as fact plan as it can be referred in query
-   *
-   * @param aggExp
-   * main table aggregate expression
-   * @return updated aggregate expression
-   */
-  private def getExpressionsForStreaming(aggExp: Seq[Expression]): Seq[Expression] = {
-    val updatedExp = aggExp map {
-      case attr: AttributeReference =>
-        attr
-      case alias@Alias(aggExp: AggregateExpression, name) =>
-        // in case of aggregate expression get the fact alias based on expression name
-        val factAlias = factPlanExpForStreaming(name)
-        // create attribute reference object for each expression
-        val attrs = factAlias.map { factAlias =>
-          CarbonToSparkAdapter.createAttributeReference(
-            name,
-            alias.dataType,
-            alias.nullable,
-            Metadata.empty,
-            factAlias.exprId,
-            alias.qualifier,
-            alias)
-        }
-        // add aggregate function in Aggregate node added for handling streaming
-        // to aggregate results from fact and aggregate table
-        val updatedAggExp = getAggregateExpressionForAggregation(aggExp, attrs)
-        // same reference id will be used as it can be used by above nodes in the plan like
-        // sort, project, join
-        CarbonToSparkAdapter.createAliasRef(
-          updatedAggExp.head,
-          name,
-          alias.exprId,
-          alias.qualifier,
-          Option(alias.metadata),
-          Some(alias))
-      case alias@Alias(expression, name) =>
-        CarbonToSparkAdapter.createAttributeReference(
-          name,
-          alias.dataType,
-          alias.nullable,
-          Metadata.empty,
-          alias.exprId,
-          alias.qualifier,
-          alias)
-    }
-    updatedExp
-  }
-
-  /**
-   * Below method will be used to update the fact plan in case of streaming table
-   * This is required to handle average aggregte function as in case of average we need to return
-   * two columns data sum(column) and count(column) to get the correct result
-   *
-   * @param logicalPlan
-   * fact table Aggregate plan
-   * @return updated aggregate plan for fact
-   */
-  private def updateFactTablePlanForStreaming(logicalPlan: LogicalPlan) : LogicalPlan = {
-    // only aggregate expression needs to be updated
-    logicalPlan.transform{
-      case agg@Aggregate(grpExp, aggExp, _) =>
-        agg
-          .copy(aggregateExpressions = updateAggExpInFactForStreaming(aggExp, grpExp, agg)
-            .asInstanceOf[Seq[NamedExpression]])
-    }
-  }
-
-  /**
-   * Below method will be used to update the aggregate table plan for streaming
-   * @param logicalPlan
-   * aggergate table logical plan
-   * @return updated logical plan
-   */
-  private def updateAggTablePlanForStreaming(logicalPlan: LogicalPlan) : LogicalPlan = {
-    // only aggregate expression needs to be updated
-    logicalPlan.transform{
-      case agg@Aggregate(grpExp, aggExp, _) =>
-        agg
-          .copy(aggregateExpressions = updateAggExpInAggForStreaming(aggExp, grpExp, agg)
-            .asInstanceOf[Seq[NamedExpression]])
-    }
-  }
-
-  /**
-   * Below method will be used to update the aggregate plan for streaming
-   * @param namedExp
-   * aggregate expression
-   * @param grpExp
-   * group by expression
-   * @param plan
-   * aggregate query plan
-   * @return updated aggregate expression
-   */
-  private def updateAggExpInAggForStreaming(namedExp : Seq[NamedExpression],
-      grpExp: Seq[Expression], plan: LogicalPlan) : Seq[Expression] = {
-    // removing alias from expression to compare with grouping expression
-    // as in case of alias all the projection column will be updated with alias
-    val updatedExp = namedExp.map {
-      case Alias(attr: AttributeReference, name) =>
-        attr
-      case exp: Expression =>
-        exp
-    }
-    addGrpExpToAggExp(grpExp, updatedExp, plan)
-  }
-
-  /**
-   * below method will be used to updated the aggregate expression with missing
-   * group by expression, when only aggregate expression is selected in query
-   *
-   * @param grpExp
-   * group by expressions
-   * @param aggExp
-   * aggregate expressions
-   * @param plan
-   * logical plan
-   * @return updated aggregate expression
-   */
-  private def addGrpExpToAggExp(grpExp: Seq[Expression],
-      aggExp: Seq[Expression],
-      plan: LogicalPlan): Seq[Expression] = {
-    // set to add all the current aggregate expression
-    val expressions = mutable.LinkedHashSet.empty[AggExpToColumnMappingModel]
-    aggExp.foreach {
-      case Alias(exp, _) =>
-      expressions +=
-      AggExpToColumnMappingModel(
-        PreAggregateUtil.normalizeExprId(exp, plan.allAttributes), None)
-      case attr: AttributeReference =>
-        expressions +=
-        AggExpToColumnMappingModel(
-          PreAggregateUtil.normalizeExprId(attr, plan.allAttributes), None)
-    }
-    val newAggExp = new ArrayBuffer[Expression]
-    newAggExp ++= aggExp
-    // for each group by expression check if already present in set if it is present
-    // then no need to add otherwise add
-    var counter = 0
-    grpExp.foreach{gExp =>
-      val normalizedExp = AggExpToColumnMappingModel(
-        PreAggregateUtil.normalizeExprId(gExp, plan.allAttributes), None)
-      if(!expressions.contains(normalizedExp)) {
-        gExp match {
-          case attr: AttributeReference =>
-            newAggExp += attr
-          case exp: Expression =>
-            newAggExp += CarbonToSparkAdapter.createAliasRef(
-              exp,
-              "dummy_" + counter,
-              NamedExpression.newExprId)
-            counter = counter + 1
-        }
-      }
-    }
-    newAggExp
-  }
-  /**
-   * Below method will be used to update the aggregate expression for streaming fact table plan
-   * @param namedExp
-   * streaming Fact plan aggregate expression
-   * @return
-   * Updated streaming fact plan aggregate expression
-   */
-  private def updateAggExpInFactForStreaming(namedExp : Seq[NamedExpression],
-  grpExp: Seq[Expression], plan: LogicalPlan) : Seq[Expression] = {
-    val addedExp = addGrpExpToAggExp(grpExp, namedExp, plan)
-    val updatedExp = addedExp.flatMap {
-      case attr: AttributeReference =>
-        Seq(attr)
-      case alias@Alias(aggExp: AggregateExpression, name) =>
-        // get the new aggregate expression
-        val newAggExp = getAggFunctionForFactStreaming(aggExp)
-        val updatedExp = newAggExp.map { exp =>
-          CarbonToSparkAdapter.createAliasRef(exp,
-            name,
-            NamedExpression.newExprId,
-            alias.qualifier,
-            Some(alias.metadata),
-            Some(alias))
-        }
-        // adding to map which will be used while Adding an Aggregate node for handling streaming
-        // table plan change
-        factPlanExpForStreaming.put(name, updatedExp)
-        updatedExp
-      case alias@Alias(exp: Expression, name) =>
-        val newAlias = Seq(alias)
-        val attr = CarbonToSparkAdapter.createAttributeReference(name,
-          alias.dataType,
-          alias.nullable,
-          alias.metadata,
-          alias.exprId,
-          alias.qualifier,
-          alias)
-        factPlanGrpExpForStreaming.put(
-          AggExpToColumnMappingModel(
-            removeQualifiers(PreAggregateUtil.normalizeExprId(exp, plan.allAttributes))),
-            attr)
-        factPlanExpForStreaming.put(name, newAlias)
-        newAlias
-    }
-    updatedExp
-  }
-  /**
-   * Below method will be used to update the fact table query aggregate function expression
-   * Rules for updating the expression.
-   * In case of average return sum(expression), count(expression) to get the correct result
-   * @param aggExp
-   * actual query aggregate expression
-   * @return seq of expression as in case of average we need to return two sum and count
-   *
-   */
-  def getAggFunctionForFactStreaming(aggExp: AggregateExpression): Seq[Expression] = {
-    aggExp.aggregateFunction match {
-      case Average(MatchCastExpression(exp: Expression, changeDataType: DataType)) =>
-        val newExp = Seq(AggregateExpression(Sum(Cast(exp, DoubleType)),
-          aggExp.mode,
-          isDistinct = false),
-          Cast(AggregateExpression(Count(exp), aggExp.mode, false), DoubleType))
-        newExp
-      case Average(exp: Expression) =>
-        val dataType =
-          if (exp.dataType.isInstanceOf[DecimalType]) {
-            // decimal must not go as double precision.
-            exp.dataType.asInstanceOf[DecimalType]
-          } else {
-            DoubleType
-          }
-        val newExp = Seq(AggregateExpression(Sum(Cast(exp, dataType)), aggExp.mode, false),
-          Cast(AggregateExpression(Count(exp), aggExp.mode, false), dataType))
-        newExp
-      case _ =>
-        val newExp = Seq(aggExp)
-        newExp
-    }
-  }
-
-  /**
-   * Below method will be used to validate query plan and get the proper aggregation data map schema
-   * and child relation plan object if plan is valid for transformation
-   * @param queryColumns list of query columns from projection and filter
-   * @param aggregateExpressions list of aggregate expression (aggregate function)
-   * @param carbonTable parent carbon table
-   * @param parentLogicalPlan parent logical relation
-   * @return if plan is valid then aggregation data map schema and its relation plan
-   */
-  def getChildDataMapForTransformation(queryColumns: scala.collection.mutable.HashSet[QueryColumn],
-      aggregateExpressions: scala.collection.mutable.HashSet[AggregateExpression],
-      carbonTable: CarbonTable,
-      parentLogicalPlan: LogicalPlan): (AggregationDataMapSchema, LogicalPlan) = {
-    // getting all the projection columns
-    val listProjectionColumn = queryColumns
-      .filter(queryColumn => !queryColumn.isFilterColumn)
-      .toList
-    // getting all the filter columns
-    val listFilterColumn = queryColumns
-      .filter(queryColumn => queryColumn.isFilterColumn)
-      .toList
-    val isProjectionColumnPresent = (listProjectionColumn.size + listFilterColumn.size) > 0
-    // create a query plan object which will be used to select the list of pre aggregate tables
-    // matches with this plan
-    val queryPlan = new AggregateQueryPlan(listProjectionColumn.asJava, listFilterColumn.asJava)
-    // create aggregate table selector object
-    val aggregateTableSelector = new AggregateTableSelector(queryPlan, carbonTable)
-    // select the list of valid child tables
-    val selectedDataMapSchemas = aggregateTableSelector.selectPreAggDataMapSchema()
-    // query has only aggregate expression then selected data map will be empty
-    // the validate all the child data map otherwise validate selected data map
-    var selectedAggMaps = if (isProjectionColumnPresent) {
-      selectedDataMapSchemas
-    } else {
-      carbonTable.getTableInfo.getDataMapSchemaList
-    }
-    // if it does not match with any pre aggregate table return the same plan
-    if (!selectedAggMaps.isEmpty) {
-      // filter the selected child schema based on size to select the pre-aggregate tables
-      // that are enabled
-      val catalog = CarbonEnv.getInstance(sparkSession).carbonMetaStore
-      val relationBuffer = selectedAggMaps.asScala.map { selectedDataMapSchema =>
-        val identifier = TableIdentifier(
-          selectedDataMapSchema.getRelationIdentifier.getTableName,
-          Some(selectedDataMapSchema.getRelationIdentifier.getDatabaseName))
-        val carbonRelation =
-          catalog.lookupRelation(identifier)(sparkSession).asInstanceOf[CarbonRelation]
-        val relation = sparkSession.sessionState.catalog.lookupRelation(identifier)
-        (selectedDataMapSchema, carbonRelation, relation)
-      }.filter(_._2.sizeInBytes != 0L).sortBy(_._2.sizeInBytes)
-      if (relationBuffer.isEmpty) {
-        // If the size of relation Buffer is 0 then it means that none of the pre-aggregate
-        // tables have data yet.
-        // In this case we would return the original plan so that the query hits the parent
-        // table.
-        (null, null)
-      } else {
-        // if query does not have any aggregate function no need to validate the same
-        val tuple = if (aggregateExpressions.nonEmpty && !selectedAggMaps.isEmpty) {
-          relationBuffer.collectFirst {
-            case a@(datamapSchema, _, _)
-              if validateAggregateExpression(datamapSchema,
-                carbonTable,
-                parentLogicalPlan,
-                aggregateExpressions.toSeq) =>
-              a
-          }
-        } else {
-          Some(relationBuffer.head)
-        }
-        tuple match {
-          case Some((dataMapSchema, _, logicalPlan)) => (dataMapSchema
-            .asInstanceOf[AggregationDataMapSchema], new FindDataSourceTable(sparkSession)
-            .apply(logicalPlan))
-          case None => (null, null)
-        }
-        // If the relationBuffer is enabled then find the table with the minimum size.
-      }
-    } else {
-      (null, null)
-    }
-  }
-
-  /**
-   * Below method will be used to validate aggregate expression with the data map
-   * and will return the selected valid data maps
-   * @param selectedDataMap list of data maps
-   * @param carbonTable parent carbon table
-   * @param parentLogicalPlan parent logical plan
-   * @param queryAggExpLogicalPlans query agg expression logical plan
-   * @return valid data map
-   */
-  def validateAggregateExpression(selectedDataMap: DataMapSchema,
-      carbonTable: CarbonTable,
-      parentLogicalPlan: LogicalPlan,
-      queryAggExpLogicalPlans: Seq[AggregateExpression]): Boolean = {
-    val mappingModel = getExpressionToColumnMapping(selectedDataMap,
-      carbonTable,
-      parentLogicalPlan)
-    queryAggExpLogicalPlans.forall{p =>
-      mappingModel.exists{m =>
-        matchExpression(
-          PreAggregateUtil.normalizeExprId(p, parentLogicalPlan.allAttributes),
-          m.expression)}
-    }
-  }
-
-  /**
-   * Below method will be used to update the expression
-   * It will remove the qualifiers
-   * @param expression
-   * expression
-   * @return updated expressions
-   */
-  private def removeQualifiers(expression: Expression) : Expression = {
-    expression.transform {
-      case attr: AttributeReference =>
-        CarbonToSparkAdapter.createAttributeReference(
-          attr.name,
-          attr.dataType,
-          attr.nullable,
-          attr.metadata,
-          attr.exprId,
-          None,
-          attr)
-    }
-  }
-
-  /**
-   * Below method will be used to match two expressions
-   * @param firstExp
-   * first expression
-   * @param secondExp
-   * second expressios
-   * @return is similare
-   */
-  private def matchExpression(firstExp: Expression, secondExp: Expression) : Boolean = {
-    val first = removeQualifiers(firstExp)
-    val second = removeQualifiers(secondExp)
-    first == second
-  }
-
-  /**
-   * Below method will be used to to get the logical plan for each aggregate expression in
-   * child data map and its column schema mapping if mapping is already present
-   * then it will use the same otherwise it will generate and stored in aggregation data map
-   * @param selectedDataMap child data map
-   * @param carbonTable parent table
-   * @param parentLogicalPlan logical relation of actual plan
-   * @return map of logical plan for each aggregate expression in child query and its column mapping
-   */
-  def getExpressionToColumnMapping(selectedDataMap: DataMapSchema,
-      carbonTable: CarbonTable,
-      parentLogicalPlan: LogicalPlan): mutable.Set[AggExpToColumnMappingModel] = {
-    val aggDataMapSchema = selectedDataMap.asInstanceOf[AggregationDataMapSchema]
-    if(null == aggDataMapSchema.getAggExpToColumnMapping) {
-      // add preAGG UDF to avoid all the PreAggregate rule
-      val childDataMapQueryString = parser.addPreAggFunction(
-        PreAggregateUtil.getChildQuery(aggDataMapSchema))
-      // get the logical plan
-      val aggPlan = sparkSession.sql(childDataMapQueryString).logicalPlan
-      // getting all aggregate expression from query
-      val dataMapAggExp = getAggregateExpFromChildDataMap(aggPlan)
-      // in case of average child table will have two columns which will be stored in sequence
-      // so for average expression we need to get two columns for mapping
-      var counter = 0
-      // sorting the columns based on schema ordinal so search will give proper result
-      val sortedColumnList = aggDataMapSchema.getChildSchema.getListOfColumns.asScala
-        .sortBy(_.getSchemaOrdinal)
-      val expressionToColumnMapping = mutable.LinkedHashSet.empty[AggExpToColumnMappingModel]
-      dataMapAggExp.foreach { aggExp =>
-        val updatedExp = PreAggregateUtil.normalizeExprId(aggExp, aggPlan.allAttributes)
-        val model = AggExpToColumnMappingModel(updatedExp, None)
-        if (!expressionToColumnMapping.contains(model)) {
-          // check if aggregate expression is of type avg
-          // get the columns
-          val columnSchema = aggDataMapSchema
-            .getAggColumnBasedOnIndex(counter, sortedColumnList.asJava)
-          // increment the counter so when for next expression above code will be
-          // executed it will search from that schema ordinal
-          counter = columnSchema.getSchemaOrdinal + 1
-          model.columnSchema = Some(columnSchema)
-          expressionToColumnMapping += model
-        }
-      }
-      aggDataMapSchema.setAggExpToColumnMapping(expressionToColumnMapping.asJava)
-      // return the mapping
-      expressionToColumnMapping
-    } else {
-      aggDataMapSchema.getAggExpToColumnMapping
-        .asInstanceOf[java.util.Set[AggExpToColumnMappingModel]].asScala
-        .asInstanceOf[mutable.LinkedHashSet[AggExpToColumnMappingModel]]
-    }
-  }
-
-  /**
-   * Below method will be used to get aggregate expression
-   * @param logicalPlan logical plan
-   * @return list of aggregate expression
-   */
-  def getAggregateExpFromChildDataMap(logicalPlan: LogicalPlan): Seq[AggregateExpression] = {
-    val list = scala.collection.mutable.ListBuffer.empty[AggregateExpression]
-    logicalPlan match {
-      case _@Aggregate(_, aggExp, _) =>
-        aggExp map {
-          case Alias(attr: AggregateExpression, _) =>
-            list ++= PreAggregateUtil.validateAggregateFunctionAndGetFields(attr)
-          case _ =>
-        }
-    }
-    list
-  }
-
-  /**
-   * Below method will be used to check whether specific segment is set for maintable
-   * if it is present then no need to transform the plan and query will be executed on
-   * maintable
-   * @param carbonTable parent table
-   * @return is specific segment is present in session params
-   */
-  def isSpecificSegmentNotPresent(carbonTable: CarbonTable) : Boolean = {
-    val carbonSessionInfo = ThreadLocalSessionInfo.getCarbonSessionInfo
-    if (carbonSessionInfo != null) {
-      carbonSessionInfo.getSessionParams
-        .getProperty(CarbonCommonConstants.CARBON_INPUT_SEGMENTS +
-                     carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier
-                       .getDatabaseName + "." + carbonTable.getTableName, "").isEmpty
-    } else {
-      true
-    }
-  }
-
-  /**
-   * Below method will be used to extract the query columns from
-   * filter expression
-   * @param expression filter expression
-   * @param queryColumns query column set
-   * @param carbonTable parent table
-   * @return isvalid filter expression for aggregate
-   */
-  def extractColumnFromExpression(expression: Expression,
-      queryColumns: scala.collection.mutable.HashSet[QueryColumn],
-      carbonTable: CarbonTable,
-      isFilterColumn: Boolean = false) {
-    // map to maintain attribute reference present in the filter to timeseries function
-    // if applied this is added to avoid duplicate column
-    val mapOfColumnSeriesFun = scala.collection.mutable.HashMap.empty[AttributeReference, String]
-    expression.transform {
-      case attr: AttributeReference =>
-        if (mapOfColumnSeriesFun.get(attr).isEmpty) {
-          mapOfColumnSeriesFun.put(attr, null)
-        }
-        attr
-      case udf@CarbonScalaUDF(_) =>
-        // for handling timeseries function
-        if (udf.asInstanceOf[ScalaUDF].function.getClass.getName.equalsIgnoreCase(
-          "org.apache.spark.sql.execution.command.timeseries.TimeseriesFunction") &&
-            CarbonUtil.hasTimeSeriesDataMap(carbonTable)) {
-          mapOfColumnSeriesFun.put(udf.children.head.asInstanceOf[AttributeReference],
-            udf.children.last.asInstanceOf[Literal].value.toString)
-        } else {
-          // for any other scala udf
-          udf.transform {
-            case attr: AttributeReference =>
-              if (mapOfColumnSeriesFun.get(attr).isEmpty) {
-                mapOfColumnSeriesFun.put(attr, null)
-              }
-              attr
-          }
-        }
-        udf
-    }
-    mapOfColumnSeriesFun.foreach { f =>
-      if (f._2 == null) {
-        queryColumns +=
-        getQueryColumn(f._1.name, carbonTable, isFilterColumn)
-      } else {
-        queryColumns += getQueryColumn(f._1.name,
-          carbonTable,
-          isFilterColumn,
-          timeseriesFunction = f._2)
-      }
-    }
-  }
-
-  /**
-   * Below method will be used to get the child attribute reference
-   * based on parent name
-   *
-   * @param dataMapSchema child schema
-   * @param attributeReference parent attribute reference
-   * @param attributes child logical relation
-   * @param canBeNull this is added for strict validation in which case child attribute can be
-   * null and when it cannot be null
-   * @return child attribute reference
-   */
-  def getChildAttributeReference(dataMapSchema: DataMapSchema,
-      attributeReference: AttributeReference,
-      attributes: Seq[AttributeReference],
-      canBeNull: Boolean = false,
-      timeseriesFunction: String = ""): AttributeReference = {
-    val aggregationDataMapSchema = dataMapSchema.asInstanceOf[AggregationDataMapSchema]
-    val columnSchema = if (timeseriesFunction.isEmpty) {
-      aggregationDataMapSchema.getChildColByParentColName(attributeReference.name.toLowerCase)
-    } else {
-      aggregationDataMapSchema
-        .getTimeseriesChildColByParent(attributeReference.name.toLowerCase,
-          timeseriesFunction)
-    }
-    // here column schema cannot be null, if it is null then aggregate table selection
-    // logic has some problem
-    if (!canBeNull && null == columnSchema) {
-      throw new AnalysisException("Column does not exists in Pre Aggregate table")
-    }
-    if(null == columnSchema && canBeNull) {
-      null
-    } else {
-      // finding the child attribute from child logical relation
-      attributes.find(p => p.name.equals(columnSchema.getColumnName)).get
-    }
-  }
-
-  /**
-   * Below method will be used to get the updated expression for pre aggregated table.
-   * It will replace the attribute of actual plan with child table attributes.
-   * Updation will be done for below expression.
-   * 1. Grouping expression
-   * 2. aggregate expression
-   * 3. child logical plan
-   * 4. filter expression if present
-   *
-   * @param groupingExpressions actual plan grouping expression
-   * @param aggregateExpressions actual plan aggregate expression
-   * @param child child logical plan
-   * @param filterExpression filter expression
-   * @param aggDataMapSchema pre aggregate table schema
-   * @param attributes pre aggregate table logical relation
-   * @param aggPlan aggregate logical plan
-   * @return tuple of(updated grouping expression,
-   * updated aggregate expression,
-   * updated child logical plan,
-   * updated filter expression if present in actual plan)
-   */
-  def getUpdatedExpressions(groupingExpressions: Seq[Expression],
-      aggregateExpressions: Seq[NamedExpression],
-      child: LogicalPlan, filterExpression: Option[Expression] = None,
-      aggDataMapSchema: AggregationDataMapSchema,
-      attributes: Seq[AttributeReference],
-      aggPlan: LogicalPlan,
-      parentTable: CarbonTable,
-      parentLogicalPlan: LogicalPlan): (Seq[Expression], Seq[NamedExpression], LogicalPlan,
-    Option[Expression]) = {
-    val aggExpColumnMapping = if (null != aggDataMapSchema.getAggExpToColumnMapping) {
-      Some(aggDataMapSchema.getAggExpToColumnMapping
-        .asInstanceOf[java.util.Set[AggExpToColumnMappingModel]].asScala
-        .asInstanceOf[mutable.LinkedHashSet[AggExpToColumnMappingModel]])
-    } else {
-      None
-    }
-
-    // transforming the group by expression attributes with child attributes
-    val updatedGroupExp = groupingExpressions.map { exp =>
-      exp.transform {
-        case attr: AttributeReference =>
-          val childAttr = getChildAttributeReference(aggDataMapSchema, attr, attributes)
-          childAttr
-      }
-    }
-    // below code is for updating the aggregate expression.
-    // Note: In case of aggregate expression updation we need to return alias as
-    //       while showing the final result we need to show based on actual query
-    //       for example: If query is "select name from table group by name"
-    //       if we only update the attributes it will show child table column name in final output
-    //       so for handling this if attributes does not have alias we need to return alias of
-    // parent
-    //       table column name
-    // Rules for updating aggregate expression.
-    // 1. If it matches with attribute reference return alias of child attribute reference
-    // 2. If it matches with alias return same alias with child attribute reference
-    // 3. If it matches with alias of any supported aggregate function return aggregate function
-    // with child attribute reference. Please check class level documentation how when aggregate
-    // function will be updated
-
-    val updatedAggExp = aggregateExpressions.flatMap {
-      // case for attribute reference
-      case attr: AttributeReference =>
-        val childAttr = getChildAttributeReference(aggDataMapSchema,
-          attr,
-          attributes)
-        val newExpressionId = NamedExpression.newExprId
-        val childTableAttr = CarbonToSparkAdapter.createAttributeReference(attr.name,
-          childAttr.dataType,
-          childAttr.nullable,
-          childAttr.metadata,
-          newExpressionId,
-          childAttr.qualifier,
-          attr)
-        updatedExpression.put(attr, childTableAttr)
-        // returning the alias to show proper column name in output
-        Seq(Alias(childAttr,
-          attr.name)(newExpressionId,
-          childAttr.qualifier).asInstanceOf[NamedExpression])
-      // case for alias
-      case alias@Alias(attr: AttributeReference, name) =>
-        val childAttr = getChildAttributeReference(aggDataMapSchema,
-          attr,
-          attributes)
-        val newExpressionId = NamedExpression.newExprId
-        val parentTableAttr = CarbonToSparkAdapter.createAttributeReference(name,
-          alias.dataType,
-          alias.nullable,
-          Metadata.empty,
-          alias.exprId,
-          alias.qualifier,
-          alias)
-        val childTableAttr = CarbonToSparkAdapter.createAttributeReference(name,
-          alias.dataType,
-          alias.nullable,
-          Metadata.empty,
-          newExpressionId,
-          alias.qualifier,
-          alias)
-        updatedExpression.put(parentTableAttr, childTableAttr)
-        // returning alias with child attribute reference
-        Seq(Alias(childAttr,
-          name)(newExpressionId,
-          childAttr.qualifier).asInstanceOf[NamedExpression])
-      // for aggregate function case
-      case alias@Alias(attr: AggregateExpression, name) =>
-        // get the updated aggregate aggregate function
-        val aggExp = if (aggExpColumnMapping.isDefined) {
-          getUpdatedAggregateExpressionForChild(attr,
-            aggDataMapSchema,
-            attributes,
-            parentTable,
-            parentLogicalPlan,
-            aggExpColumnMapping.get,
-            parentTable.isStreamingSink)
-        } else {
-          Seq(attr)
-        }
-        if(!parentTable.isStreamingSink) {
-          // for normal table
-          // generate new expression id for child
-          val newExpressionId = NamedExpression.newExprId
-          // create a parent attribute reference which will be replced on node which may be referred
-          // by node like sort join
-          val parentTableAttr = CarbonToSparkAdapter.createAttributeReference(name,
-            alias.dataType,
-            alias.nullable,
-            Metadata.empty,
-            alias.exprId,
-            alias.qualifier,
-            alias)
-          // creating a child attribute reference which will be replced
-          val childTableAttr = CarbonToSparkAdapter.createAttributeReference(name,
-            alias.dataType,
-            alias.nullable,
-            Metadata.empty,
-            newExpressionId,
-            alias.qualifier,
-            alias)
-          // adding to map, will be used during other node updation like sort, join, project
-          updatedExpression.put(parentTableAttr, childTableAttr)
-          // returning alias with child attribute reference
-          Seq(Alias(aggExp.head,
-            name)(newExpressionId,
-            alias.qualifier).asInstanceOf[NamedExpression])
-        } else {
-          // for streaming table
-          // create alias for aggregate table
-          val aggExpForStreaming = aggExp.map{ exp =>
-            CarbonToSparkAdapter.createAliasRef(exp,
-              name,
-              NamedExpression.newExprId,
-              alias.qualifier,
-              Some(alias.metadata),
-              Some(alias)).asInstanceOf[NamedExpression]
-          }
-          aggExpForStreaming
-        }
-      case alias@Alias(expression: Expression, name) =>
-        val updatedExp =
-          if (expression.isInstanceOf[ScalaUDF] &&
-              expression.asInstanceOf[ScalaUDF].function.getClass.getName.equalsIgnoreCase(
-                "org.apache.spark.sql.execution.command.timeseries.TimeseriesFunction")) {
-            expression.asInstanceOf[ScalaUDF].transform {
-              case attr: AttributeReference =>
-                val childAttributeReference = getChildAttributeReference(aggDataMapSchema,
-                  attr,
-                  attributes,
-                  timeseriesFunction =
-                    expression.asInstanceOf[ScalaUDF].children(1).asInstanceOf[Literal].value
-                      .toString)
-                childAttributeReference
-            }
-          } else {
-            expression.transform{
-              case attr: AttributeReference =>
-                val childAttributeReference = getChildAttributeReference(aggDataMapSchema,
-                  attr,
-                  attributes)
-                childAttributeReference
-            }
-          }
-        val newExpressionId = NamedExpression.newExprId
-        val parentTableAttr = CarbonToSparkAdapter.createAttributeReference(name,
-          alias.dataType,
-          alias.nullable,
-          Metadata.empty,
-          alias.exprId,
-          alias.qualifier,
-          alias)
-        val childTableAttr = CarbonToSparkAdapter.createAttributeReference(name,
-          alias.dataType,
-          alias.nullable,
-          Metadata.empty,
-          newExpressionId,
-          alias.qualifier,
-          alias)
-        updatedExpression.put(parentTableAttr, childTableAttr)
-        Seq(Alias(updatedExp, name)(newExpressionId,
-          alias.qualifier).asInstanceOf[NamedExpression])
-    }
-    // transforming the logical relation
-    val newChild = child.transform {
-      case _: LogicalRelation =>
-        aggPlan
-      case _: SubqueryAlias =>
-        aggPlan match {
-          case s: SubqueryAlias => s.child
-          case others => others
-        }
-    }
-    // updating the filter expression if present
-    val updatedFilterExpression = if (filterExpression.isDefined) {
-      val filterExp = filterExpression.get
-      Some(filterExp.transform {
-        case attr: AttributeReference =>
-          getChildAttributeReference(aggDataMapSchema, attr, attributes)
-      })
-    } else {
-      None
-    }
-    (updatedGroupExp, updatedAggExp, newChild, updatedFilterExpression)
-  }
-
-  /**
-   * Below method will be used to get the aggregate expression based on match
-   * Aggregate expression updation rules
-   * 1 Change the count AggregateExpression to Sum as count
-   * is already calculated so in case of aggregate table
-   * we need to apply sum to get the count
-   * 2 In case of average aggregate function select 2 columns from aggregate table
-   * with aggregation sum and count.
-   * Then add divide(sum(column with sum), sum(column with count)).
-   * Note: During aggregate table creation for average aggregation function
-   * table will be created with two columns one for sum(column) and count(column)
-   * to support rollup
-   *
-   * @param aggExp aggregate expression
-   * @param dataMapSchema child data map schema
-   * @param attributes child logical relation
-   * @param parentTable parent carbon table
-   * @param parentLogicalPlan logical relation
-   * @return updated expression
-   */
-  def getUpdatedAggregateExpressionForChild(aggExp: AggregateExpression,
-      dataMapSchema: AggregationDataMapSchema,
-      attributes: Seq[AttributeReference],
-      parentTable: CarbonTable,
-      parentLogicalPlan: LogicalPlan,
-      aggExpColumnMapping: mutable.LinkedHashSet[AggExpToColumnMappingModel],
-      isStreamingTable: Boolean):
-  Seq[Expression] = {
-    // get the updated aggregate expression, in case of average column
-    // it will be divided in two aggergate expression
-    val updatedAggExp = PreAggregateUtil.validateAggregateFunctionAndGetFields(aggExp)
-    // get the attributes to be updated for child table
-    val attrs = aggExpColumnMapping.collect {
-      case (schemaAggExpModel)
-        if updatedAggExp
-          .exists(p =>
-            matchExpression(schemaAggExpModel.expression,
-            PreAggregateUtil.normalizeExprId(p, parentLogicalPlan.allAttributes))) =>
-        attributes filter (_.name.equalsIgnoreCase(
-          schemaAggExpModel.columnSchema.get.asInstanceOf[ColumnSchema].getColumnName))
-    }.flatten
-    // getting aggregate table aggregate expressions
-    getAggregateExpressionForAggregation(aggExp, attrs.toSeq, isStreamingTable)
-  }
-
-  /**
-   * Below method will be used to update the aggregate expression.
-   * 1.In case of average below expression will be returned.
-   * 1.1 Streaming table
-   *    1.1.1 Aggregate table
-   *        It will return sum(expression) and count(expression)
-   *    1.2.1 Aggregate node added for streaming
-   *        It will return Divide(sum(expression)/count(expression))
-   * 2.1 Normal table
-   *    2.1.1 Aggregate table
-   *      It will return Divide(sum(expression)/count(expression))
-   * 2. In case of count for aggregate table and aggregate node added for streaming
-   *    table count will be aggregated to sum
-   *
-   * @param aggExp
-   * aggregate expression
-   * @param attrs
-   * aggregate function Attribute, in case of average it will be two to support rollup
-   * @return
-   * aggregate expression
-   */
-  def getAggregateExpressionForAggregation(aggExp: AggregateExpression,
-      attrs: Seq[AttributeReference],
-      isStreamingTable: Boolean = false): Seq[Expression] = {
-    aggExp.aggregateFunction match {
-      case Sum(MatchCastExpression(_, changeDataType: DataType)) =>
-        Seq(AggregateExpression(Sum(Cast(attrs.head, changeDataType)), aggExp.mode, false))
-      case Sum(_) =>
-        Seq(AggregateExpression(Sum(attrs.head), aggExp.mode, false))
-      case Max(MatchCastExpression(_, changeDataType: DataType)) =>
-        Seq(AggregateExpression(Max(Cast(attrs.head, changeDataType)), aggExp.mode, false))
-      case Max(_) =>
-        Seq(AggregateExpression(Max(attrs.head), aggExp.mode, false))
-      case Min(MatchCastExpression(_, changeDataType: DataType)) =>
-        Seq(AggregateExpression(Min(Cast(attrs.head, changeDataType)), aggExp.mode, false))
-      case Min(_) =>
-        Seq(AggregateExpression(Min(attrs.head), aggExp.mode, false))
-      // Change the count AggregateExpression to Sum as count
-      // is already calculated so in case of aggregate table
-      // we need to apply sum to get the count
-      case Count(Seq(expression: Expression)) =>
-        Seq(AggregateExpression(Sum(Cast(attrs.head, LongType)), aggExp.mode, false))
-
-      case Average(MatchCastExpression(exp: Expression, changeDataType: DataType)) =>
-        // for handling Normal table case/Aggregate node added in case of streaming table
-        if (!isStreamingTable) {
-          // In case of average aggregate function select 2 columns from aggregate table
-          // with aggregation sum and count.
-          // Then add divide(sum(column with sum), sum(column with count)).
-          Seq(Divide(AggregateExpression(Sum(Cast(
-            attrs.head,
-            DoubleType)),
-            aggExp.mode,
-            false),
-            AggregateExpression(Sum(Cast(
-              attrs.last,
-              DoubleType)),
-              aggExp.mode,
-              false)))
-        } else {
-          // in case of streaming aggregate table return two aggregate function sum and count
-          Seq(AggregateExpression(Sum(Cast(
-            attrs.head,
-            DoubleType)),
-            aggExp.mode,
-            false),
-            AggregateExpression(Sum(Cast(
-              attrs.last,
-              DoubleType)),
-              aggExp.mode,
-              false))
-        }
-      case Average(exp: Expression) =>
-        val dataType =
-          if (exp.dataType.isInstanceOf[DecimalType]) {
-            // decimal must not go as double precision.
-            exp.dataType.asInstanceOf[DecimalType]
-          } else {
-            DoubleType
-          }
-        // for handling Normal table case/Aggregate node added in case of streaming table
-        if (!isStreamingTable) {
-          // In case of average aggregate function select 2 columns from aggregate table
-          // with aggregation sum and count.
-          // Then add divide(sum(column with sum), sum(column with count)).
-          Seq(Divide(AggregateExpression(Sum(Cast(
-            attrs.head,
-            dataType)),
-            aggExp.mode,
-            false),
-            AggregateExpression(Sum(Cast(
-              attrs.last,
-              dataType)),
-              aggExp.mode,
-              false)))
-        } else {
-          // in case of streaming aggregate table return two aggregate function sum and count
-          Seq(AggregateExpression(Sum(Cast(
-            attrs.head,
-            dataType)),
-            aggExp.mode,
-            false),
-            AggregateExpression(Sum(Cast(
-              attrs.last,
-              dataType)),
-              aggExp.mode,
-              false))
-        }
-    }
-  }
-  /**
-   * Method to get the carbon table and table name
-   * @param parentLogicalRelation parent table relation
-   * @return tuple of carbon table
-   */
-  def getCarbonTable(parentLogicalRelation: LogicalRelation): CarbonTable = {
-    val carbonTable = parentLogicalRelation.relation.asInstanceOf[CarbonDatasourceHadoopRelation]
-      .carbonRelation
-      .metaData.carbonTable
-    carbonTable
-  }
-
-  /**
-   * Below method will be used to get the query columns from plan
-   * @param groupByExpression group by expression
-   * @param aggregateExpressions aggregate expression
-   * @param carbonTable parent carbon table
-   * @param queryColumns list of attributes
-   * @return plan is valid
-   */
-  def extractQueryColumnsFromAggExpression(groupByExpression: Seq[Expression],
-      aggregateExpressions: Seq[NamedExpression],
-      carbonTable: CarbonTable,
-      queryColumns: scala.collection.mutable.HashSet[QueryColumn],
-      aggreagteExps: scala.collection.mutable.HashSet[AggregateExpression]): Boolean = {
-    var isValid = true
-    groupByExpression foreach  { expression =>
-      extractColumnFromExpression(expression, queryColumns, carbonTable)
-    }
-    aggregateExpressions.map {
-      case attr: AttributeReference =>
-        queryColumns += getQueryColumn(attr.name,
-          carbonTable)
-      case Alias(attr: AttributeReference, _) =>
-        queryColumns += getQueryColumn(attr.name,
-          carbonTable);
-      case Alias(attr: AggregateExpression, _) =>
-        if (attr.isDistinct) {
-          isValid = false
-        }
-        val aggExp = PreAggregateUtil.validateAggregateFunctionAndGetFields(attr)
-        if (aggExp.nonEmpty) {
-          aggreagteExps ++= aggExp
-        } else {
-          isValid = false
-        }
-      case Alias(expression: Expression, _) =>
-        if (expression.isInstanceOf[ScalaUDF] &&
-            expression.asInstanceOf[ScalaUDF].function.getClass.getName.equalsIgnoreCase(
-              "org.apache.spark.sql.execution.command.timeseries.TimeseriesFunction") &&
-            CarbonUtil.hasTimeSeriesDataMap(carbonTable)) {
-          queryColumns += getQueryColumn(expression.asInstanceOf[ScalaUDF].children(0)
-            .asInstanceOf[AttributeReference].name,
-            carbonTable,
-            timeseriesFunction = expression.asInstanceOf[ScalaUDF].children(1).asInstanceOf[Literal]
-              .value.toString)
-        } else {
-          expression.transform {
-            case attr: AttributeReference =>
-              queryColumns += getQueryColumn(attr.name,
-                carbonTable)
-              attr
-            case attr: AggregateExpression =>
-              if (attr.isDistinct) {
-                isValid = false
-              }
-              val aggExp = PreAggregateUtil.validateAggregateFunctionAndGetFields(attr)
-              if (aggExp.nonEmpty) {
-                aggreagteExps ++= aggExp
-              } else {
-                isValid = false
-              }
-              attr
-
-          }
-        }
-    }
-    isValid
-  }
-
-  /**
-   * Below method will be used to get the query column object which
-   * will have details of the column and its property
-   *
-   * @param columnName parent column name
-   * @param carbonTable parent carbon table
-   * @param isFilterColumn is filter is applied on column
-   * @return query column
-   */
-  def getQueryColumn(columnName: String,
-      carbonTable: CarbonTable,
-      isFilterColumn: Boolean = false,
-      timeseriesFunction: String = ""): QueryColumn = {
-    val columnSchema = carbonTable.getColumnByName(columnName.toLowerCase)
-    if(null == columnSchema) {
-      null
-    } else {
-        new QueryColumn(
-          columnSchema.getColumnSchema,
-        isFilterColumn,
-        timeseriesFunction.toLowerCase)
-    }
-  }
-}
-
-/**
- * Data loading rule class to validate and update the data loading query plan
- * Validation rule:
- * 1. update the avg aggregate expression with two columns sum and count
- * 2. Remove duplicate sum and count expression if already there in plan
- * @param sparkSession spark session
- */
-case class CarbonPreAggregateDataLoadingRules(sparkSession: SparkSession)
-  extends Rule[LogicalPlan] {
-  lazy val parser = new CarbonSpark2SqlParser
-  override def apply(plan: LogicalPlan): LogicalPlan = {
-    val validExpressionsMap = scala.collection.mutable.HashSet.empty[AggExpToColumnMappingModel]
-    val namedExpressionList = scala.collection.mutable.LinkedHashSet.empty[NamedExpression]
-    plan transform {
-      case aggregate@Aggregate(groupingExpressions,
-      aExp,
-      CarbonSubqueryAlias(_, logicalRelation: LogicalRelation))
-        if validateAggregateExpressions(aExp) &&
-           logicalRelation.relation.isInstanceOf[CarbonDatasourceHadoopRelation] =>
-        aExp.foreach {
-          case attr: AttributeReference =>
-            namedExpressionList += attr
-          case alias@Alias(_: AttributeReference, _) =>
-            namedExpressionList += alias
-          case alias@Alias(aggExp: AggregateExpression, name) =>
-            // get the updated expression for avg convert it to two expression
-            // sum and count
-            val expressions = PreAggregateUtil.validateAggregateFunctionAndGetFields(aggExp)
-            // if size is more than one then it was for average
-            if(expressions.size > 1) {
-              val sumExp = PreAggregateUtil.normalizeExprId(
-                expressions.head,
-                aggregate.allAttributes)
-              // get the logical plan fro count expression
-              val countExp = PreAggregateUtil.normalizeExprId(
-                expressions.last,
-                aggregate.allAttributes)
-              // check with same expression already sum is present then do not add to
-              // named expression list otherwise update the list and add it to set
-              if (!validExpressionsMap.contains(AggExpToColumnMappingModel(sumExp))) {
-                namedExpressionList +=
-                CarbonToSparkAdapter.createAliasRef(expressions.head,
-                  name + "_ sum",
-                  NamedExpression.newExprId,
-                  alias.qualifier,
-                  Some(alias.metadata),
-                  Some(alias))
-                validExpressionsMap += AggExpToColumnMappingModel(sumExp)
-              }
-              // check with same expression already count is present then do not add to
-              // named expression list otherwise update the list and add it to set
-              if (!validExpressionsMap.contains(AggExpToColumnMappingModel(countExp))) {
-                namedExpressionList +=
-                CarbonToSparkAdapter.createAliasRef(expressions.last, name + "_ count",
-                    NamedExpression.newExprId,
-                    alias.qualifier,
-                    Some(alias.metadata),
-                    Some(alias))
-                validExpressionsMap += AggExpToColumnMappingModel(countExp)
-              }
-            } else {
-              // get the logical plan for expression
-              val exp = PreAggregateUtil.normalizeExprId(
-                expressions.head,
-                aggregate.allAttributes)
-              // check with same expression already  present then do not add to
-              // named expression list otherwise update the list and add it to set
-              if (!validExpressionsMap.contains(AggExpToColumnMappingModel(exp))) {
-                namedExpressionList+=alias
-                validExpressionsMap += AggExpToColumnMappingModel(exp)
-              }
-            }
-          case alias@Alias(_: Expression, _) =>
-            namedExpressionList += alias
-        }
-        groupingExpressions foreach {
-          case namedExpr: NamedExpression => namedExpressionList += namedExpr
-          case _ => namedExpressionList
-        }
-        aggregate.copy(aggregateExpressions = namedExpressionList.toSeq)
-      case plan: LogicalPlan => plan
-    }
-  }
-
-  /**
-   * Called by PreAggregateLoadingRules to validate if plan is valid for applying rules or not.
-   * If the plan has PreAggLoad i.e Loading UDF and does not have PreAgg i.e Query UDF then it is
-   * valid.
-   * @param namedExpression named expressions
-   * @return valid or not
-   */
-  private def validateAggregateExpressions(namedExpression: Seq[NamedExpression]): Boolean = {
-    val filteredExpressions = namedExpression.filterNot(_.isInstanceOf[UnresolvedAlias])
-    filteredExpressions.exists { expr =>
-      !expr.name.equalsIgnoreCase("PreAgg") && expr.name.equalsIgnoreCase("preAggLoad")
-    }
-  }
-}
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 cc22d50..9b3ff87 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
@@ -176,7 +176,7 @@
         carbonTable.getAbsoluteTableIdentifier)
       if (tableStatusLastUpdateTime != tableStatusNewLastUpdatedTime) {
         val allSegments = new SegmentStatusManager(carbonTable.getAbsoluteTableIdentifier)
-          .getValidAndInvalidSegments(carbonTable.isChildTable)
+          .getValidAndInvalidSegments(carbonTable.isChildTableForMV)
         if (allSegments.getValidSegments.isEmpty) {
           sizeInBytesLocalValue = 0L
         } else {
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/execution/command/CarbonHiveCommands.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/execution/command/CarbonHiveCommands.scala
index 0d2a4c3..94b75fd 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/execution/command/CarbonHiveCommands.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/execution/command/CarbonHiveCommands.scala
@@ -96,10 +96,6 @@
           "property should be in \" carbon.input.segments.<database_name>" +
           ".<table_name>=<seg_id list> \" format.")
       }
-    } else if (key.startsWith(CarbonCommonConstants.VALIDATE_CARBON_INPUT_SEGMENTS)) {
-      sessionParams.addProperty(key.toLowerCase(), value)
-    } else if (key.startsWith(CarbonCommonConstantsInternal.QUERY_ON_PRE_AGG_STREAMING)) {
-      sessionParams.addProperty(key.toLowerCase(), value)
     } else if (key.startsWith(CarbonCommonConstants.CARBON_DATAMAP_VISIBLE)) {
       if (key.split("\\.").length == 6) {
         val keyArray = key.split("\\.")
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/cache/DropCacheEventListeners.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/listeners/DropCacheEventListeners.scala
similarity index 92%
rename from integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/cache/DropCacheEventListeners.scala
rename to integration/spark2/src/main/scala/org/apache/spark/sql/listeners/DropCacheEventListeners.scala
index c2181f2..3be0606 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/cache/DropCacheEventListeners.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/listeners/DropCacheEventListeners.scala
@@ -15,13 +15,14 @@
 * limitations under the License.
 */
 
-package org.apache.spark.sql.execution.command.cache
+package org.apache.spark.sql.listeners
 
 import scala.collection.JavaConverters._
 import scala.collection.mutable
 
 import org.apache.spark.sql.{CarbonEnv, SparkSession}
 import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.execution.command.cache.{CacheUtil, CarbonDropCacheCommand}
 import org.apache.spark.util.DataMapUtil
 
 import org.apache.carbondata.common.logging.LogServiceFactory
@@ -30,7 +31,6 @@
 import org.apache.carbondata.core.metadata.schema.datamap.DataMapClassProvider
 import org.apache.carbondata.core.metadata.schema.table.DataMapSchema
 import org.apache.carbondata.events.{DropTableCacheEvent, Event, OperationContext, OperationEventListener}
-import org.apache.carbondata.processing.merger.CarbonDataMergerUtil
 
 object DropCacheDataMapEventListener extends OperationEventListener {
 
@@ -48,7 +48,7 @@
         val carbonTable = dropCacheEvent.carbonTable
         val sparkSession = dropCacheEvent.sparkSession
         val internalCall = dropCacheEvent.internalCall
-        if ((carbonTable.isChildDataMap || carbonTable.isChildTable) && !internalCall) {
+        if (carbonTable.isChildTableForMV && !internalCall) {
           throw new UnsupportedOperationException("Operation not allowed on child table.")
         }
 
@@ -108,10 +108,6 @@
         val cache = CacheProvider.getInstance().getCarbonCache
         val datamaps = DataMapStoreManager.getInstance().getDataMapSchemasOfTable(carbonTable)
           .asScala.toList
-        val segments = CarbonDataMergerUtil
-          .getValidSegmentList(carbonTable.getAbsoluteTableIdentifier, carbonTable.isChildTable)
-          .asScala.toList
-
         datamaps.foreach {
           case datamap if datamap.getProviderName
             .equalsIgnoreCase(DataMapClassProvider.BLOOMFILTER.getShortName) =>
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mv/DataMapListeners.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/listeners/MVListeners.scala
similarity index 93%
rename from integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mv/DataMapListeners.scala
rename to integration/spark2/src/main/scala/org/apache/spark/sql/listeners/MVListeners.scala
index d554b1b..3304d31 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mv/DataMapListeners.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/listeners/MVListeners.scala
@@ -15,7 +15,7 @@
 * limitations under the License.
 */
 
-package org.apache.spark.sql.execution.command.mv
+package org.apache.spark.sql.listeners
 
 import scala.collection.JavaConverters._
 import scala.collection.mutable
@@ -33,10 +33,9 @@
 import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, DataMapSchema}
 import org.apache.carbondata.datamap.DataMapManager
 import org.apache.carbondata.events._
-import org.apache.carbondata.processing.loading.events.LoadEvents.LoadTablePostExecutionEvent
+import org.apache.carbondata.processing.loading.events.LoadEvents.{LoadTablePostExecutionEvent, LoadTablePreExecutionEvent}
 import org.apache.carbondata.processing.merger.CompactionType
 
-
 object DataMapListeners {
   def getDataMapTableColumns(dataMapSchema: DataMapSchema,
       carbonTable: CarbonTable): mutable.Buffer[String] = {
@@ -90,6 +89,21 @@
   }
 }
 
+object LoadMVTablePreListener extends OperationEventListener {
+  /**
+   * Called on LoadTablePreExecutionEvent event occurrence
+   */
+  override def onEvent(event: Event, operationContext: OperationContext): Unit = {
+    val loadEvent = event.asInstanceOf[LoadTablePreExecutionEvent]
+    val carbonLoadModel = loadEvent.getCarbonLoadModel
+    val table = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
+    val isInternalLoadCall = carbonLoadModel.isAggLoadRequest
+    if (table.isChildTableForMV && !isInternalLoadCall) {
+      throw new UnsupportedOperationException("Cannot insert data directly into MV table")
+    }
+  }
+}
+
 /**
  * Listener to trigger data load on mv datamap after main table data load
  */
@@ -175,7 +189,7 @@
         throw new UnsupportedOperationException(
           "Delete segment operation is not supported on tables having child datamap")
       }
-      if (carbonTable.isChildTable) {
+      if (carbonTable.isChildTableForMV) {
         throw new UnsupportedOperationException(
           "Delete segment operation is not supported on datamap table")
       }
@@ -193,7 +207,7 @@
   override def onEvent(event: Event, operationContext: OperationContext): Unit = {
     val dataTypeChangePreListener = event.asInstanceOf[AlterTableAddColumnPreEvent]
     val carbonTable = dataTypeChangePreListener.carbonTable
-    if (carbonTable.isChildTable) {
+    if (carbonTable.isChildTableForMV) {
       throw new UnsupportedOperationException(
         s"Cannot add columns in a DataMap table " +
         s"${ carbonTable.getDatabaseName }.${ carbonTable.getTableName }")
@@ -233,7 +247,7 @@
         }
       }
     }
-    if (carbonTable.isChildTable) {
+    if (carbonTable.isChildTableForMV) {
       throw new UnsupportedOperationException(
         s"Cannot drop columns present in a datamap table ${ carbonTable.getDatabaseName }." +
         s"${ carbonTable.getTableName }")
@@ -270,7 +284,7 @@
         }
       }
     }
-    if (carbonTable.isChildTable) {
+    if (carbonTable.isChildTableForMV) {
       throw new UnsupportedOperationException(
         s"Cannot change data type or rename column for columns present in mv datamap table " +
         s"${ carbonTable.getDatabaseName }.${ carbonTable.getTableName }")
@@ -354,7 +368,7 @@
         operationContext.setProperty("dropPartitionCommands", childDropPartitionCommands)
         childDropPartitionCommands.foreach(_.processMetadata(SparkSession.getActiveSession.get))
       }
-    } else if (parentCarbonTable.isChildTable) {
+    } else if (parentCarbonTable.isChildTableForMV) {
       if (operationContext.getProperty("isInternalDropCall") == null) {
         throw new UnsupportedOperationException("Cannot drop partition directly on child table")
       }
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/indexserver/PrePrimingListener.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/listeners/PrePrimingListener.scala
similarity index 92%
rename from integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/indexserver/PrePrimingListener.scala
rename to integration/spark2/src/main/scala/org/apache/spark/sql/listeners/PrePrimingListener.scala
index 0c2e4cf..60f5ef8 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/indexserver/PrePrimingListener.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/listeners/PrePrimingListener.scala
@@ -15,14 +15,13 @@
  * limitations under the License.
  */
 
-package org.apache.spark.sql.execution.command.indexserver
+package org.apache.spark.sql.listeners
 
 import scala.collection.JavaConverters._
 
 import org.apache.carbondata.common.logging.LogServiceFactory
-import org.apache.carbondata.core.datamap.{DistributableDataMapFormat, Segment}
-import org.apache.carbondata.events.{Event, IndexServerLoadEvent,
-  OperationContext, OperationEventListener}
+import org.apache.carbondata.core.datamap.DistributableDataMapFormat
+import org.apache.carbondata.events.{Event, IndexServerLoadEvent, OperationContext, OperationEventListener}
 import org.apache.carbondata.indexserver.IndexServer
 
 // Listener for the PrePriming Event. This listener calls the index server using an Asynccall
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/cache/ShowCacheEventListeners.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/listeners/ShowCacheEventListeners.scala
similarity index 95%
rename from integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/cache/ShowCacheEventListeners.scala
rename to integration/spark2/src/main/scala/org/apache/spark/sql/listeners/ShowCacheEventListeners.scala
index 409ff4f..cfd88e5 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/cache/ShowCacheEventListeners.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/listeners/ShowCacheEventListeners.scala
@@ -15,7 +15,7 @@
 * limitations under the License.
 */
 
-package org.apache.spark.sql.execution.command.cache
+package org.apache.spark.sql.listeners
 
 import scala.collection.JavaConverters._
 
@@ -25,7 +25,7 @@
 import org.apache.carbondata.core.metadata.schema.table.DataMapSchema
 import org.apache.carbondata.events._
 
-object ShowCachePreAggEventListener extends OperationEventListener {
+object ShowCachePreMVEventListener extends OperationEventListener {
 
   val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
 
@@ -40,7 +40,7 @@
       case showTableCacheEvent: ShowTableCacheEvent =>
         val carbonTable = showTableCacheEvent.carbonTable
         val internalCall = showTableCacheEvent.internalCall
-        if ((carbonTable.isChildDataMap || carbonTable.isChildTable) && !internalCall) {
+        if (carbonTable.isChildTableForMV && !internalCall) {
           throw new UnsupportedOperationException("Operation not allowed on child table.")
         }
 
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonFilters.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonFilters.scala
index 0fd07bb..2765c5f 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonFilters.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonFilters.scala
@@ -517,11 +517,6 @@
     if (!carbonTable.isHivePartitionTable) {
       return None
     }
-    // first try to read partitions in case if the trigger comes from the aggregation table load.
-    val partitionsForAggTable = getPartitionsForAggTable(sparkSession, carbonTable)
-    if (partitionsForAggTable.isDefined) {
-      return partitionsForAggTable
-    }
     val partitions = {
       try {
         if (CarbonProperties.getInstance().
@@ -554,38 +549,4 @@
     })
   }
 
-  /**
-   * In case of loading aggregate tables it needs to be get only from the main table load in
-   * progress segment. So we should read from the segment file of that segment
-   */
-  def getPartitionsForAggTable(sparkSession: SparkSession,
-      table: CarbonTable): Option[Seq[PartitionSpec]] = {
-    // when validate segments is disabled then only read from partitionmap
-    val carbonSessionInfo = ThreadLocalSessionInfo.getCarbonSessionInfo
-    if (carbonSessionInfo != null) {
-      val validateSegments = carbonSessionInfo.getThreadParams
-        .getProperty(CarbonCommonConstants.VALIDATE_CARBON_INPUT_SEGMENTS +
-                     table.getDatabaseName + "." +
-                     table.getTableName, "true").toBoolean
-      if (!validateSegments) {
-        val segmentNumbersFromProperty = carbonSessionInfo.getThreadParams
-          .getProperty(CarbonCommonConstants.CARBON_INPUT_SEGMENTS +
-                       table.getDatabaseName + "." + table.getTableName, "*")
-        // In case of compaction multiple segments will be passed as CARBON_INPUT_SEGMENTS.
-        // Therefore partitionSpec will be extracted from all segments.
-        val segments = segmentNumbersFromProperty.split(",").flatMap { a =>
-          val segment = Segment.toSegment(a, null)
-          val segmentFile = new SegmentFileStore(table.getTablePath, segment.getSegmentFileName)
-          segmentFile.getPartitionSpecs.asScala
-        }
-        Some(segments.toSet.toSeq)
-      } else {
-        None
-      }
-    } else {
-      None
-    }
-  }
-
-
 }
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonLateDecodeRule.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonLateDecodeRule.scala
index 99a8e70..b124e9a 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonLateDecodeRule.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonLateDecodeRule.scala
@@ -33,9 +33,8 @@
 import org.apache.spark.sql.types._
 
 import org.apache.carbondata.common.logging.LogServiceFactory
-import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.stats.QueryStatistic
-import org.apache.carbondata.core.util.{CarbonProperties, CarbonTimeStatisticsFactory, ThreadLocalSessionInfo}
+import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory
 import org.apache.carbondata.spark.CarbonAliasDecoderRelation
 
 
@@ -76,7 +75,6 @@
 
   def checkIfRuleNeedToBeApplied(plan: LogicalPlan, removeSubQuery: Boolean = false): Boolean = {
     relations = collectCarbonRelation(plan)
-    validateQueryDirectlyOnDataMap(relations)
     if (relations.nonEmpty && !isOptimized(plan)) {
       // In case scalar subquery skip the transformation and update the flag.
       if (relations.exists(_.carbonRelation.isSubquery.nonEmpty)) {
@@ -97,48 +95,6 @@
     }
   }
 
-  /**
-   * Below method will be used to validate if query is directly fired on pre aggregate
-   * data map or not
-   * @param relations all relations from query
-   *
-   */
-  def validateQueryDirectlyOnDataMap(relations: Seq[CarbonDecoderRelation]): Unit = {
-    var isPreAggDataMapExists = false
-    // first check if pre aggregate data map exists or not
-    relations.foreach { relation =>
-      if (relation.carbonRelation.carbonTable.isChildDataMap) {
-        isPreAggDataMapExists = true
-      }
-    }
-    var isThrowException = false
-    // if relation contains pre aggregate data map
-    if (isPreAggDataMapExists) {
-      val carbonSessionInfo = ThreadLocalSessionInfo.getCarbonSessionInfo
-      if (null != carbonSessionInfo) {
-        lazy val sessionPropertyValue = CarbonProperties.getInstance
-          .getProperty(CarbonCommonConstants.SUPPORT_DIRECT_QUERY_ON_DATAMAP,
-            CarbonCommonConstants.SUPPORT_DIRECT_QUERY_ON_DATAMAP_DEFAULTVALUE)
-        // Check if property is set in thread params which would mean this is an internal call
-        // (from load or compaction) and should be of highest priority. Otherwise get from
-        // session(like user has dynamically given the value using set command). If not found in
-        // session then look for the property in carbon.properties file, otherwise use default
-        // value 'false'.
-        val supportQueryOnDataMap = CarbonEnv
-          .getThreadParam(CarbonCommonConstants.SUPPORT_DIRECT_QUERY_ON_DATAMAP,
-            sessionPropertyValue).toBoolean
-        if (!supportQueryOnDataMap) {
-          isThrowException = true
-        }
-      }
-    }
-    if (isThrowException) {
-      throw new AnalysisException("Query On DataMap not supported because "
-        + CarbonCommonConstants.SUPPORT_DIRECT_QUERY_ON_DATAMAP + " is false. " +
-        "Please change the value to true by set command or other if you want to query on DataMap.")
-    }
-  }
-
   private def collectCarbonRelation(plan: LogicalPlan): Seq[CarbonDecoderRelation] = {
     plan collect {
       case l: LogicalRelation if l.relation.isInstanceOf[CarbonDatasourceHadoopRelation] =>
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 7e0b13e..7797b42 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
@@ -671,16 +671,8 @@
     }
   }
 
-  def addPreAggFunction(sql: String): String = {
-    addPreAgg(new lexical.Scanner(sql)) match {
-      case Success(query, _) => query
-      case _ =>
-        throw new MalformedCarbonCommandException(s"Unsupported query")
-    }
-  }
-
-  def addPreAggLoadFunction(sql: String): String = {
-    addPreAggLoad(new lexical.Scanner(sql)) match {
+  def addMVSkipFunction(sql: String): String = {
+    addMVSkipUDF(new lexical.Scanner(sql)) match {
       case Success(query, _) => query
       case _ =>
         throw new MalformedCarbonCommandException(s"Unsupported query")
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParserUtil.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParserUtil.scala
index 9591bc0..60e5cad 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParserUtil.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParserUtil.scala
@@ -215,7 +215,6 @@
         tableProperties,
         bucketFields,
         isAlterFlow = false,
-        false,
         tableComment)
       TableNewProcessor(tableModel)
     }
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 f21e4db..4177152 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
@@ -569,11 +569,6 @@
       propertiesMap: mutable.Map[String, String]): Unit = {
     // validate column meta cache property
     if (propertiesMap.get(CarbonCommonConstants.COLUMN_META_CACHE).isDefined) {
-      // Column meta cache is not allowed for child tables and dataMaps
-      if (carbonTable.isChildDataMap) {
-        throw new MalformedCarbonCommandException(s"Table property ${
-          CarbonCommonConstants.COLUMN_META_CACHE} is not allowed for child datamaps")
-      }
       val schemaList: util.List[ColumnSchema] = CarbonUtil
         .getColumnSchemaList(carbonTable.getVisibleDimensions, carbonTable.getVisibleMeasures)
       val tableColumns: Seq[String] = schemaList.asScala
@@ -589,11 +584,6 @@
     }
     // validate cache level property
     if (propertiesMap.get(CarbonCommonConstants.CACHE_LEVEL).isDefined) {
-      // Cache level is not allowed for child tables and dataMaps
-      if (carbonTable.isChildDataMap) {
-        throw new MalformedCarbonCommandException(s"Table property ${
-          CarbonCommonConstants.CACHE_LEVEL} is not allowed for child datamaps")
-      }
       CommonUtil.validateCacheLevel(
         propertiesMap.get(CarbonCommonConstants.CACHE_LEVEL).get,
         propertiesMap)
@@ -922,13 +912,7 @@
       propertiesMap: mutable.Map[String, String]): Unit = {
     // validate load min size property
     if (propertiesMap.get(CarbonCommonConstants.CARBON_LOAD_MIN_SIZE_INMB).isDefined) {
-      // load min size is not allowed for child tables and dataMaps
-      if (carbonTable.isChildDataMap) {
-        throw new MalformedCarbonCommandException(s"Table property ${
-          CarbonCommonConstants.CARBON_LOAD_MIN_SIZE_INMB} is not allowed for child datamaps")
-      }
-      CommonUtil.validateLoadMinSize(propertiesMap,
-        CarbonCommonConstants.CARBON_LOAD_MIN_SIZE_INMB)
+      CommonUtil.validateLoadMinSize(propertiesMap, CarbonCommonConstants.CARBON_LOAD_MIN_SIZE_INMB)
     }
   }
 }
diff --git a/integration/spark2/src/main/scala/org/apache/spark/util/DataMapUtil.scala b/integration/spark2/src/main/scala/org/apache/spark/util/DataMapUtil.scala
index ffbe765..0333cab 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/util/DataMapUtil.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/util/DataMapUtil.scala
@@ -32,7 +32,7 @@
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 
 /**
- * Utility class for keeping all the utility methods common for pre-aggregate and mv datamap
+ * Utility class for keeping all the utility methods common for MV
  */
 object DataMapUtil {
 
diff --git a/integration/spark2/src/main/scala/org/apache/spark/util/MergeIndexUtil.scala b/integration/spark2/src/main/scala/org/apache/spark/util/MergeIndexUtil.scala
index d2d39e4..6ed5ac7 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/util/MergeIndexUtil.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/util/MergeIndexUtil.scala
@@ -20,7 +20,6 @@
 import java.util
 
 import scala.collection.JavaConverters._
-import scala.collection.mutable
 
 import org.apache.spark.rdd.CarbonMergeFilesRDD
 import org.apache.spark.sql.SparkSession
@@ -28,11 +27,9 @@
 
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.datamap.Segment
 import org.apache.carbondata.core.metadata.SegmentFileStore
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager
-import org.apache.carbondata.processing.loading.model.CarbonLoadModel
 import org.apache.carbondata.processing.merger.CarbonDataMergerUtil
 
 object MergeIndexUtil {
@@ -72,8 +69,7 @@
     carbonTable: CarbonTable,
     mergedLoads: util.List[String]): Unit = {
     // get only the valid segments of the table
-    val validSegments: mutable.Buffer[Segment] = CarbonDataMergerUtil.getValidSegmentList(
-      carbonTable.getAbsoluteTableIdentifier, carbonTable.isChildTable).asScala
+    val validSegments = CarbonDataMergerUtil.getValidSegmentList(carbonTable).asScala
     val mergedSegmentIds = new util.ArrayList[String]()
     mergedLoads.asScala.foreach(mergedLoad => {
       val loadName = mergedLoad
diff --git a/integration/spark2/src/main/spark2.1/org/apache/spark/sql/hive/CarbonSessionState.scala b/integration/spark2/src/main/spark2.1/org/apache/spark/sql/hive/CarbonSessionState.scala
index 75e7d89..6b94806 100644
--- a/integration/spark2/src/main/spark2.1/org/apache/spark/sql/hive/CarbonSessionState.scala
+++ b/integration/spark2/src/main/spark2.1/org/apache/spark/sql/hive/CarbonSessionState.scala
@@ -287,9 +287,7 @@
   }
 
   override def execute(plan: LogicalPlan): LogicalPlan = {
-    var logicalPlan = analyzer.execute(plan)
-    logicalPlan = CarbonPreAggregateDataLoadingRules(sparkSession).apply(logicalPlan)
-    CarbonPreAggregateQueryRules(sparkSession).apply(logicalPlan)
+    val logicalPlan = analyzer.execute(plan)
     if (mvPlan != null) {
       mvPlan.apply(logicalPlan)
     } else {
diff --git a/integration/spark2/src/test/scala/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMapFunctionSuite.scala b/integration/spark2/src/test/scala/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMapFunctionSuite.scala
index 4010ef6..ad379fb 100644
--- a/integration/spark2/src/test/scala/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMapFunctionSuite.scala
+++ b/integration/spark2/src/test/scala/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMapFunctionSuite.scala
@@ -834,103 +834,6 @@
       CarbonV3DataFormatConstants.BLOCKLET_SIZE_IN_MB_DEFAULT_VALUE)
   }
 
-  /**
-   * create bloom and preagg on base table, then create bloom on preagg table,
-   * index column and group by column is dictionary column.
-   * note that the test steps are copied from issue.
-   * In the CI env, sometime it will become timeout, so we ignore the newly added tests
-   */
-  ignore("test bloom datamap: CARBONDATA-2799 bloom datamap on preaggregate") {
-    sql(
-      s"""
-         | CREATE TABLE $normalTable (id int, name string, salary float,dob date)
-         | STORED BY 'carbondata'
-         | TBLPROPERTIES('dictionary_include'='id')
-       """.stripMargin)
-    sql(
-      s"""
-         | CREATE TABLE $bloomDMSampleTable (id int, name string, salary float,dob date)
-         | STORED BY 'carbondata'
-         | TBLPROPERTIES('dictionary_include'='id')
-       """.stripMargin)
-    (1 to 2).foreach { _ =>
-      sql(
-        s"""
-           | INSERT INTO $bloomDMSampleTable VALUES
-           | ('1', 'name1', '11.1', '2018-07-01'),
-           | ('2', 'name2', '21.1', '2018-07-02'),
-           | ('3', 'name3', '31.1', '2018-07-03'),
-           | ('4', 'name4', '41.1', '2018-07-04')
-       """.stripMargin)
-      sql(
-        s"""
-           | INSERT INTO $normalTable VALUES
-           | ('1', 'name1', '11.1', '2018-07-01'),
-           | ('2', 'name2', '21.1', '2018-07-02'),
-           | ('3', 'name3', '31.1', '2018-07-03'),
-           | ('4', 'name4', '41.1', '2018-07-04')
-       """.stripMargin)
-    }
-    sql(
-      s"""
-         | CREATE DATAMAP $dataMapName ON TABLE $bloomDMSampleTable
-         | USING 'bloomfilter'
-         | DMPROPERTIES('INDEX_COLUMNS'='id', 'BLOOM_SIZE'='320000', 'BLOOM_FPP'='0.01', 'BLOOM_COMPRESS'='TRUE')
-       """.stripMargin)
-    sql(
-      s"""
-         | INSERT INTO $bloomDMSampleTable VALUES
-         | ('1', 'name1', '11.1', '2018-07-01'),
-         | ('2', 'name2', '21.1', '2018-07-02'),
-         | ('3', 'name3', '31.1', '2018-07-03'),
-         | ('4', 'name4', '41.1', '2018-07-04')
-       """.stripMargin)
-    sql(
-      s"""
-         | INSERT INTO $normalTable VALUES
-         | ('1', 'name1', '11.1', '2018-07-01'),
-         | ('2', 'name2', '21.1', '2018-07-02'),
-         | ('3', 'name3', '31.1', '2018-07-03'),
-         | ('4', 'name4', '41.1', '2018-07-04')
-       """.stripMargin)
-    val preAggOnBase = "preagg_on_base"
-    sql(
-      s"""
-         | CREATE DATAMAP $preAggOnBase ON TABLE $bloomDMSampleTable
-         | USING 'preaggregate' AS
-         | select id, count(id) from $bloomDMSampleTable group by id
-       """.stripMargin)
-    checkAnswer(sql(s"SELECT id, count(id) from $bloomDMSampleTable where id = 3 group by id"),
-      sql(s"SELECT id, count(id) from $normalTable where id = 3 group by id"))
-
-    val bloomOnPreAgg = "bloom_on_pre_agg"
-    sql(
-      s"""
-         | CREATE DATAMAP $bloomOnPreAgg ON TABLE ${bloomDMSampleTable}_${preAggOnBase}
-         | USING 'bloomfilter'
-         | DMPROPERTIES('INDEX_COLUMNS'='${bloomDMSampleTable}_id')
-       """.stripMargin)
-    checkAnswer(sql(s"SELECT id, count(id) from $bloomDMSampleTable where id = 3 group by id"),
-      sql(s"SELECT id, count(id) from $normalTable where id = 3 group by id"))
-
-    sql(s"DROP DATAMAP $bloomOnPreAgg on table ${bloomDMSampleTable}_${preAggOnBase}")
-    checkAnswer(sql(s"SELECT id, count(id) from $bloomDMSampleTable where id = 3 group by id"),
-      sql(s"SELECT id, count(id) from $normalTable where id = 3 group by id"))
-
-    sql(
-      s"""
-         | CREATE DATAMAP $bloomOnPreAgg ON TABLE ${bloomDMSampleTable}_${preAggOnBase}
-         | USING 'bloomfilter'
-         | DMPROPERTIES('INDEX_COLUMNS'='${bloomDMSampleTable}_id')
-       """.stripMargin)
-    checkAnswer(sql(s"SELECT id, count(id) from $bloomDMSampleTable where id = 3 group by id"),
-      sql(s"SELECT id, count(id) from $normalTable where id = 3 group by id"))
-
-    sql(s"DROP DATAMAP $bloomOnPreAgg on table ${bloomDMSampleTable}_${preAggOnBase}")
-    checkAnswer(sql(s"SELECT id, count(id) from $bloomDMSampleTable where id = 3 group by id"),
-      sql(s"SELECT id, count(id) from $normalTable where id = 3 group by id"))
-  }
-
   override def afterAll(): Unit = {
     deleteFile(bigFile)
     sql(s"DROP TABLE IF EXISTS $normalTable")
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOpName.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOpName.scala
index ade3567b..2576616 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOpName.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOpName.scala
@@ -38,7 +38,6 @@
 import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datastore.impl.FileFactory
-import org.apache.carbondata.core.metadata.schema.datamap.DataMapClassProvider.TIMESERIES
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.statusmanager.{FileFormat, SegmentStatus}
 import org.apache.carbondata.core.util.CarbonProperties
@@ -121,9 +120,6 @@
     // 18. block drop table while streaming is in progress
     createTable(tableName = "stream_table_drop", streaming = true, withBatchLoad = false)
 
-    // 19. block streaming on 'preaggregate' main table
-    createTable(tableName = "agg_table_block", streaming = false, withBatchLoad = false)
-
     createTable(tableName = "agg_table", streaming = true, withBatchLoad = false)
 
     createTable(tableName = "stream_table_empty", streaming = true, withBatchLoad = false)
@@ -279,278 +275,6 @@
     assertResult(exceptedRow)(row)
   }
 
-  test("test preaggregate table creation on streaming table without handoff") {
-    val identifier = new TableIdentifier("agg_table", Option("streaming"))
-    val carbonTable = CarbonEnv.getInstance(spark).carbonMetaStore.lookupRelation(identifier)(spark)
-      .asInstanceOf[CarbonRelation].metaData.carbonTable
-    val csvDataDir = new File("target/csvdatanew").getCanonicalPath
-    // streaming ingest 10 rows
-    val thread = createFileStreamingThread(spark, carbonTable, csvDataDir, intervalSecond = 1,
-      identifier)
-    thread.start()
-    Thread.sleep(5000)
-    thread.interrupt()
-    checkAnswer(
-      sql("select count(*) from streaming.agg_table"),
-      Seq(Row(10)))
-    sql("create datamap p1 on table agg_table using 'preaggregate' as select name, sum(salary) from agg_table group by name")
-    // No data should be loaded into aggregate table as hand-off is not yet fired
-    checkAnswer(sql("select * from agg_table_p1"), Seq())
-  }
-
-  test("test if data is loaded into preaggregate after handoff is fired") {
-    createTable(tableName = "agg_table2", streaming = true, withBatchLoad = false)
-    val identifier = new TableIdentifier("agg_table2", Option("streaming"))
-    val carbonTable = CarbonEnv.getInstance(spark).carbonMetaStore.lookupRelation(identifier)(spark)
-      .asInstanceOf[CarbonRelation].metaData.carbonTable
-    val csvDataDir = new File("target/csvdatanew").getCanonicalPath
-    // streaming ingest 10 rows
-    val thread = createFileStreamingThread(spark, carbonTable, csvDataDir, intervalSecond = 1,
-      identifier)
-    thread.start()
-    Thread.sleep(5000)
-    thread.interrupt()
-    checkAnswer(
-      sql("select count(*) from streaming.agg_table2"),
-      Seq(Row(10)))
-    sql("create datamap p1 on table agg_table2 using 'preaggregate' as select name, sum(salary) from agg_table2 group by name")
-    sql("create datamap p2 on table agg_table2 using 'preaggregate' as select name, avg(salary) from agg_table2 group by name")
-    sql("create datamap p3 on table agg_table2 using 'preaggregate' as select name, min(salary) from agg_table2 group by name")
-    sql("create datamap p4 on table agg_table2 using 'preaggregate' as select name, max(salary) from agg_table2 group by name")
-    sql("create datamap p5 on table agg_table2 using 'preaggregate' as select name, count(salary) from agg_table2 group by name")
-    sql("alter table agg_table2 finish streaming")
-    sql("alter table agg_table2 compact 'streaming'")
-    // Data should be loaded into aggregate table as hand-off is fired
-    checkAnswer(sql("select * from agg_table2_p1"),
-      Seq(
-        Row("name_10", 200000.0),
-        Row("name_11", 220000.0),
-        Row("name_12", 240000.0),
-        Row("name_13", 260000.0),
-        Row("name_14", 280000.0)))
-    checkAnswer(sql("select * from agg_table2_p2"),
-      Seq(
-        Row("name_10", 200000.0, 2.0),
-        Row("name_11", 220000.0, 2.0),
-        Row("name_12", 240000.0, 2.0),
-        Row("name_13", 260000.0, 2.0),
-        Row("name_14", 280000.0, 2.0)))
-    checkAnswer(sql("select * from agg_table2_p3"),
-      Seq(
-        Row("name_10", 100000.0),
-        Row("name_11", 110000.0),
-        Row("name_12", 120000.0),
-        Row("name_13", 130000.0),
-        Row("name_14", 140000.0)))
-    checkAnswer(sql("select * from agg_table2_p4"),
-      Seq(
-        Row("name_10", 100000.0),
-        Row("name_11", 110000.0),
-        Row("name_12", 120000.0),
-        Row("name_13", 130000.0),
-        Row("name_14", 140000.0)))
-    checkAnswer(sql("select * from agg_table2_p5"),
-      Seq(
-        Row("name_10", 2.0),
-        Row("name_11", 2.0),
-        Row("name_12", 2.0),
-        Row("name_13", 2.0),
-        Row("name_14", 2.0)))
-    sql("drop table agg_table2")
-  }
-
-  test("test whether data is loaded into preaggregate after handoff is fired") {
-    createTable(tableName = "agg_table2", streaming = true, withBatchLoad = false)
-    val identifier = new TableIdentifier("agg_table2", Option("streaming"))
-    val carbonTable = CarbonEnv.getInstance(spark).carbonMetaStore.lookupRelation(identifier)(spark)
-      .asInstanceOf[CarbonRelation].metaData.carbonTable
-    val csvDataDir = new File("target/csvdatanew").getCanonicalPath
-    // streaming ingest 10 rows
-    val thread = createFileStreamingThread(spark, carbonTable, csvDataDir, intervalSecond = 1,
-      identifier)
-    thread.start()
-    Thread.sleep(5000)
-    thread.interrupt()
-    checkAnswer(
-      sql("select count(*) from streaming.agg_table2"),
-      Seq(Row(10)))
-    sql(s"load data inpath '$csvDataDir' into table agg_table2 options('FILEHEADER'='id, name, city, salary, tax, percent, birthday, register, updated, file')")
-    sql("create datamap p1 on table agg_table2 using 'preaggregate' as select name, sum(salary) from agg_table2 group by name")
-    sql("alter table agg_table2 finish streaming")
-    sql("alter table agg_table2 compact 'streaming'")
-    // Data should be loaded into aggregate table as hand-off is fired
-    checkAnswer(sql("select name, sum(salary) from agg_table2 group by name"),
-      Seq(
-        Row("name_10", 400000.0),
-        Row("name_14", 560000.0),
-        Row("name_12", 480000.0),
-        Row("name_11", 440000.0),
-        Row("name_13", 520000.0)))
-    checkAnswer(sql("select * from agg_table2_p1"),
-      Seq(
-        Row("name_10", 200000.0),
-        Row("name_11", 220000.0),
-        Row("name_12", 240000.0),
-        Row("name_13", 260000.0),
-        Row("name_14", 280000.0),
-        Row("name_10", 200000.0),
-        Row("name_11", 220000.0),
-        Row("name_12", 240000.0),
-        Row("name_13", 260000.0),
-        Row("name_14", 280000.0)))
-
-    sql("drop table agg_table2")
-  }
-
-  test("test whether data is loaded into preaggregate before handoff is fired") {
-    createTable(tableName = "agg_table2", streaming = true, withBatchLoad = false)
-    val identifier = new TableIdentifier("agg_table2", Option("streaming"))
-    val carbonTable = CarbonEnv.getInstance(spark).carbonMetaStore.lookupRelation(identifier)(spark)
-      .asInstanceOf[CarbonRelation].metaData.carbonTable
-    val csvDataDir = new File("target/csvdatanew").getCanonicalPath
-    // streaming ingest 10 rows
-    val thread = createFileStreamingThread(spark, carbonTable, csvDataDir, intervalSecond = 1,
-      identifier)
-    thread.start()
-    Thread.sleep(5000)
-    thread.interrupt()
-    checkAnswer(
-      sql("select count(*) from streaming.agg_table2"),
-      Seq(Row(10)))
-    sql(s"load data inpath '$csvDataDir' into table agg_table2 options('FILEHEADER'='id, name, city, salary, tax, percent, birthday, register, updated, file')")
-    sql("create datamap p1 on table agg_table2 using 'preaggregate' as select name, sum(salary) from agg_table2 group by name")
-    // Data should be loaded into aggregate table as hand-off is fired
-    checkAnswer(sql("select name, sum(salary) from agg_table2 group by name"),
-      Seq(
-        Row("name_10", 400000.0),
-        Row("name_14", 560000.0),
-        Row("name_12", 480000.0),
-        Row("name_11", 440000.0),
-        Row("name_13", 520000.0)))
-    checkAnswer(sql("select * from agg_table2_p1"),
-      Seq(
-        Row("name_10", 200000.0),
-        Row("name_11", 220000.0),
-        Row("name_12", 240000.0),
-        Row("name_13", 260000.0),
-        Row("name_14", 280000.0)))
-
-    sql("drop table agg_table2")
-  }
-
-  test("test if timeseries load is successful when created on streaming table") {
-    sql("drop table if exists timeseries_table")
-    createTable(tableName = "timeseries_table", streaming = true, withBatchLoad = false)
-    val identifier = new TableIdentifier("timeseries_table", Option("streaming"))
-    val carbonTable = CarbonEnv.getInstance(spark).carbonMetaStore.lookupRelation(identifier)(spark)
-      .asInstanceOf[CarbonRelation].metaData.carbonTable
-    val csvDataDir = new File("target/csvdatanew").getCanonicalPath
-    val thread = createFileStreamingThread(spark, carbonTable, csvDataDir, intervalSecond = 1,
-      identifier)
-    thread.start()
-    Thread.sleep(5000)
-    thread.interrupt()
-    sql(
-      s"""
-         | CREATE DATAMAP agg0_second ON TABLE timeseries_table
-         | USING '${TIMESERIES.toString}'
-         | DMPROPERTIES (
-         | 'EVENT_TIME'='register',
-         | 'SECOND_GRANULARITY'='1')
-         | AS SELECT register, SUM(id) FROM timeseries_table
-         | GROUP BY register
-       """.stripMargin)
-    sql("alter table timeseries_table finish streaming")
-    sql("alter table timeseries_table compact 'streaming'")
-    checkAnswer( sql("select * FROM timeseries_table_agg0_second"), Seq(Row(Timestamp.valueOf("2010-01-01 10:01:01.0"), 120)))
-  }
-
-  test("test if timeseries load is successful when created on streaming table with day granularity") {
-    sql("drop table if exists timeseries_table")
-    createTable(tableName = "timeseries_table", streaming = true, withBatchLoad = false)
-    val identifier = new TableIdentifier("timeseries_table", Option("streaming"))
-    val carbonTable = CarbonEnv.getInstance(spark).carbonMetaStore.lookupRelation(identifier)(spark)
-      .asInstanceOf[CarbonRelation].metaData.carbonTable
-    val csvDataDir = new File("target/csvdatanew").getCanonicalPath
-    val thread = createFileStreamingThread(spark, carbonTable, csvDataDir, intervalSecond = 1,
-      identifier)
-    thread.start()
-    Thread.sleep(5000)
-    thread.interrupt()
-    sql(
-      s"""
-         | CREATE DATAMAP agg0_day ON TABLE timeseries_table
-         | USING '${TIMESERIES.toString}'
-         | DMPROPERTIES (
-         | 'EVENT_TIME'='register',
-         | 'DAY_GRANULARITY'='1')
-         | AS SELECT register, SUM(id) FROM timeseries_table
-         | GROUP BY register
-       """.stripMargin)
-    sql("alter table timeseries_table finish streaming")
-    sql("alter table timeseries_table compact 'streaming'")
-    checkAnswer( sql("select * FROM timeseries_table_agg0_day"), Seq(Row(Timestamp.valueOf("2010-01-01 00:00:00.0"), 120)))
-  }
-
-  test("test if minor compaction is successful for streaming and preaggregate tables") {
-    createTable(tableName = "agg_table2", streaming = true, withBatchLoad = false)
-    sql("create datamap p1 on table agg_table2 using 'preaggregate' as select name, sum(salary) from agg_table2 group by name")
-    sql("create datamap p2 on table agg_table2 using 'preaggregate' as select name, min(salary) from agg_table2 group by name")
-    loadData()
-    sql("alter table agg_table2 finish streaming")
-    sql("alter table agg_table2 compact 'streaming'")
-    loadData()
-    sql("alter table agg_table2 finish streaming")
-    sql("alter table agg_table2 compact 'streaming'")
-    loadData()
-    sql("alter table agg_table2 finish streaming")
-    sql("alter table agg_table2 compact 'streaming'")
-    loadData()
-    sql("alter table agg_table2 finish streaming")
-    sql("alter table agg_table2 compact 'streaming'")
-    sql("alter table agg_table2 compact 'minor'")
-    checkAnswer(sql("select * from agg_table2_p1"),
-      Seq(
-        Row("name_10", 800000.0),
-        Row("name_11", 880000.0),
-        Row("name_12", 960000.0),
-        Row("name_13", 1040000.0),
-        Row("name_14", 1120000.0)))
-    assert(sql("show segments for table agg_table2").collect().map(_.get(0)).contains("1.1"))
-    assert(sql("show segments for table agg_table2_p1").collect().map(_.get(0)).contains("0.1"))
-    assert(sql("show segments for table agg_table2_p2").collect().map(_.get(0)).contains("0.1"))
-    sql("drop table if exists agg_table2")
-  }
-
-  test("test if major compaction is successful for streaming and preaggregate tables") {
-    sql("drop table if exists agg_table2")
-    createTable(tableName = "agg_table2", streaming = true, withBatchLoad = false)
-    sql("create datamap p1 on table agg_table2 using 'preaggregate' as select name, sum(salary) from agg_table2 group by name")
-    loadData()
-    sql("alter table agg_table2 finish streaming")
-    sql("alter table agg_table2 compact 'streaming'")
-    loadData()
-    sql("alter table agg_table2 finish streaming")
-    sql("alter table agg_table2 compact 'streaming'")
-    loadData()
-    sql("alter table agg_table2 finish streaming")
-    sql("alter table agg_table2 compact 'streaming'")
-    loadData()
-    sql("alter table agg_table2 finish streaming")
-    sql("alter table agg_table2 compact 'streaming'")
-    sql("alter table agg_table2 compact 'major'")
-    checkAnswer(sql("select * from agg_table2_p1"),
-      Seq(
-        Row("name_10", 800000.0),
-        Row("name_11", 880000.0),
-        Row("name_12", 960000.0),
-        Row("name_13", 1040000.0),
-        Row("name_14", 1120000.0)))
-    assert(sql("show segments for table agg_table2").collect().map(_.get(0)).contains("1.1"))
-    assert(sql("show segments for table agg_table2_p1").collect().map(_.get(0)).contains("0.1"))
-    sql("drop table if exists agg_table2")
-  }
-
   def loadData() {
     val identifier = new TableIdentifier("agg_table2", Option("streaming"))
     val carbonTable = CarbonEnv.getInstance(spark).carbonMetaStore.lookupRelation(identifier)(spark)
@@ -567,68 +291,6 @@
     thread.interrupt()
   }
 
-  test("test if data is displayed when alias is used for column name") {
-    sql("drop table if exists agg_table2")
-    createTable(tableName = "agg_table2", streaming = true, withBatchLoad = false)
-    val identifier = new TableIdentifier("agg_table2", Option("streaming"))
-    val carbonTable = CarbonEnv.getInstance(spark).carbonMetaStore.lookupRelation(identifier)(spark)
-      .asInstanceOf[CarbonRelation].metaData.carbonTable
-    val csvDataDir = new File("target/csvdata1").getCanonicalPath
-    generateCSVDataFile(spark, idStart = 10, rowNums = 5, csvDataDir)
-    generateCSVDataFile(spark, idStart = 10, rowNums = 5, csvDataDir, SaveMode.Append)
-    // streaming ingest 10 rows
-    val thread = createFileStreamingThread(spark, carbonTable, csvDataDir, intervalSecond = 1,
-      identifier)
-    thread.start()
-    Thread.sleep(5000)
-    thread.interrupt()
-    checkAnswer(
-      sql("select count(*) from streaming.agg_table2"),
-      Seq(Row(10)))
-    sql(s"load data inpath '$csvDataDir' into table agg_table2 options('FILEHEADER'='id, name, city, salary, tax, percent, birthday, register, updated, file')")
-    sql("create datamap p1 on table agg_table2 using 'preaggregate' as select name, sum(salary) from agg_table2 group by name")
-    // Data should be loaded into aggregate table as hand-off is fired
-    checkAnswer(sql("select name as abc, sum(salary) as sal from agg_table2 group by name"),
-      Seq(
-        Row("name_14", 560000.0),
-        Row("name_10", 400000.0),
-        Row("name_12", 480000.0),
-        Row("name_11", 440000.0),
-        Row("name_13", 520000.0)))
-
-    sql("drop table agg_table2")
-  }
-
-  test("test if data is loaded in aggregate table after handoff is done for streaming table") {
-    createTable(tableName = "agg_table3", streaming = true, withBatchLoad = false)
-    val identifier = new TableIdentifier("agg_table3", Option("streaming"))
-    val carbonTable = CarbonEnv.getInstance(spark).carbonMetaStore.lookupRelation(identifier)(spark)
-      .asInstanceOf[CarbonRelation].metaData.carbonTable
-    val csvDataDir = new File("target/csvdatanew").getCanonicalPath
-    generateCSVDataFile(spark, idStart = 10, rowNums = 5, csvDataDir)
-    generateCSVDataFile(spark, idStart = 10, rowNums = 5, csvDataDir, SaveMode.Append)
-    // streaming ingest 10 rows
-    val thread = createFileStreamingThread(spark, carbonTable, csvDataDir, intervalSecond = 1,
-      identifier)
-    thread.start()
-    Thread.sleep(5000)
-    thread.interrupt()
-    checkAnswer(
-      sql("select count(*) from streaming.agg_table3"),
-      Seq(Row(10)))
-    sql("alter table agg_table3 finish streaming")
-    sql("alter table agg_table3 compact 'streaming'")
-    sql("create datamap p1 on table agg_table3 using 'preaggregate' as select name, sum(salary) from agg_table3 group by name")
-    // Data should be loaded into aggregate table as hand-off is fired
-    checkAnswer(sql("select * from agg_table3_p1"),
-      Seq(
-        Row("name_10", 200000.0),
-        Row("name_11", 220000.0),
-        Row("name_12", 240000.0),
-        Row("name_13", 260000.0),
-        Row("name_14", 280000.0)))
-  }
-
   // bad records
   test("streaming table with bad records action: fail") {
     executeStreamingIngest(
@@ -1594,35 +1256,6 @@
     )
   }
 
-  test("test autohandoff with preaggregate tables") {
-    sql("drop table if exists maintable")
-    createTable(tableName = "maintable", streaming = true, withBatchLoad = false)
-    sql("create datamap p1 on table maintable using 'preaggregate' as select name, sum(id) from maintable group by name")
-    executeStreamingIngest(
-      tableName = "maintable",
-      batchNums = 2,
-      rowNumsEachBatch = 100,
-      intervalOfSource = 5,
-      intervalOfIngest = 5,
-      continueSeconds = 20,
-      generateBadRecords = false,
-      badRecordAction = "force",
-      handoffSize = 1L,
-      autoHandoff = false)
-    executeStreamingIngest(
-      tableName = "maintable",
-      batchNums = 2,
-      rowNumsEachBatch = 100,
-      intervalOfSource = 5,
-      intervalOfIngest = 5,
-      continueSeconds = 20,
-      generateBadRecords = false,
-      badRecordAction = "force",
-      handoffSize = 1L,
-      autoHandoff = true)
-    checkAnswer(sql("select count(*) from maintable_p1"), Seq(Row(200)))
-  }
-
   test("block drop streaming table while streaming is in progress") {
     val identifier = new TableIdentifier("stream_table_drop", Option("streaming"))
     val carbonTable = CarbonEnv.getInstance(spark).carbonMetaStore.lookupRelation(identifier)(spark)
@@ -1648,12 +1281,6 @@
     }
   }
 
-  test("support creating datamap on streaming table") {
-    sql("CREATE DATAMAP datamap ON TABLE source " +
-        "USING 'preaggregate'" +
-        " AS SELECT c1, sum(c2) FROM source GROUP BY c1")
-  }
-
   test("check streaming property of table") {
     checkExistence(sql("DESC FORMATTED batch_table"), true, "Streaming")
     val result =
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/register/TestRegisterCarbonTable.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/register/TestRegisterCarbonTable.scala
index ccdee35..0b0ac75 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/register/TestRegisterCarbonTable.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/register/TestRegisterCarbonTable.scala
@@ -91,70 +91,6 @@
     }
   }
 
-   test("register pre aggregate tables test") {
-    sql(s"create database carbon location '$dblocation'")
-    sql("use carbon")
-    sql("""create table carbon.carbontable (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
-    sql("insert into carbontable select 'a',1,'aa','aaa'")
-    sql("insert into carbontable select 'b',1,'aa','aaa'")
-    sql("insert into carbontable select 'a',10,'aa','aaa'")
-    sql("create datamap preagg1 on table carbontable using 'preaggregate' as select c1,sum(c2) from carbontable group by c1")
-    backUpData(dblocation, "carbontable")
-    backUpData(dblocation, "carbontable_preagg1")
-    sql("drop table carbontable")
-    if (!CarbonEnv.getInstance(sqlContext.sparkSession).carbonMetaStore.isReadFromHiveMetaStore) {
-      restoreData(dblocation, "carbontable")
-      restoreData(dblocation, "carbontable_preagg1")
-      sql("refresh table carbontable")
-      checkAnswer(sql("select count(*) from carbontable"), Row(3))
-      checkAnswer(sql("select c1 from carbontable"), Seq(Row("a"), Row("b"), Row("a")))
-      checkAnswer(sql("select count(*) from carbontable_preagg1"), Row(2))
-      checkAnswer(sql("select carbontable_c1 from carbontable_preagg1"), Seq(Row("a"), Row("b")))
-    }
-  }
-
-  test("register pre aggregate table test") {
-    sql(s"create database carbon location '$dblocation'")
-    sql("use carbon")
-    sql("""create table carbon.carbontable (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
-    sql("insert into carbontable select 'a',1,'aa','aaa'")
-    sql("insert into carbontable select 'b',1,'aa','aaa'")
-    sql("insert into carbontable select 'a',10,'aa','aaa'")
-    sql("create datamap preagg1 on table carbontable using 'preaggregate' as select c1,sum(c2) from carbontable group by c1")
-    backUpData(dblocation, "carbontable")
-    backUpData(dblocation, "carbontable_preagg1")
-    sql("drop table carbontable")
-    if (!CarbonEnv.getInstance(sqlContext.sparkSession).carbonMetaStore.isReadFromHiveMetaStore) {
-      restoreData(dblocation, "carbontable")
-      restoreData(dblocation, "carbontable_preagg1")
-      sql("refresh table carbontable")
-      checkAnswer(sql("select count(*) from carbontable"), Row(3))
-      checkAnswer(sql("select c1 from carbontable"), Seq(Row("a"), Row("b"), Row("a")))
-      checkAnswer(sql("select count(*) from carbontable_preagg1"), Row(2))
-      checkAnswer(sql("select carbontable_c1 from carbontable_preagg1"), Seq(Row("a"), Row("b")))
-    }
-  }
-
-  test("register pre aggregate table should fail if the aggregate table not copied") {
-    sql(s"create database carbon location '$dblocation'")
-    sql("use carbon")
-    sql("""create table carbon.carbontable (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
-    sql("insert into carbontable select 'a',1,'aa','aaa'")
-    sql("insert into carbontable select 'b',1,'aa','aaa'")
-    sql("insert into carbontable select 'a',10,'aa','aaa'")
-    sql("create datamap preagg1 on table carbontable using 'preaggregate' as select c1,sum(c2) from carbontable group by c1")
-    backUpData(dblocation, "carbontable")
-    backUpData(dblocation, "carbontable_preagg1")
-    sql("drop table carbontable")
-    if (!CarbonEnv.getInstance(sqlContext.sparkSession).carbonMetaStore.isReadFromHiveMetaStore) {
-      restoreData(dblocation, "carbontable")
-      intercept[ProcessMetaDataException] {
-        sql("refresh table carbontable")
-      }
-      restoreData(dblocation, "carbontable_preagg1")
-    }
-  }
-
   test("Update operation on carbon table should pass after registration or refresh") {
     sql(s"create database carbon1 location '$dblocation'")
     sql("use carbon1")
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableValidationTestCase.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableValidationTestCase.scala
index 4cf6b54..dce7c73 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableValidationTestCase.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableValidationTestCase.scala
@@ -539,24 +539,6 @@
     checkExistence(sql("describe formatted specifiedSortColumnsWithAlter"),true,"Sort Columns empno, empname, role, doj")
   }
 
-  test("test to check if new parent table name is reflected in pre-aggregate tables") {
-    sql("drop table if exists preaggMain")
-    sql("drop table if exists preaggmain_new")
-    sql("drop table if exists preaggMain_preagg1")
-    sql("create table preaggMain (a string, b string, c string) stored by 'carbondata'")
-    sql(
-      "create datamap preagg1 on table PreAggMain using 'preaggregate' as select" +
-      " a,sum(b) from PreAggMain group by a")
-    assert(intercept[ProcessMetaDataException] {
-      sql("alter table preAggmain_preagg1 rename to preagg2")
-    }.getMessage.contains("Rename operation for datamaps is not supported."))
-    assert(intercept[ProcessMetaDataException] {
-      sql("alter table preaggmain rename to preaggmain_new")
-    }.getMessage.contains("Rename operation is not supported for table with pre-aggregate tables"))
-    sql("drop table if exists preaggMain")
-    sql("drop table if exists preaggmain_new")
-    sql("drop table if exists preaggMain_preagg1")
-  }
   test("test to check select columns after alter commands with null values"){
     sql("drop table if exists restructure")
     sql("drop table if exists restructure1")
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/AddColumnTestCases.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/AddColumnTestCases.scala
index 5ea79e3..fd15e89 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/AddColumnTestCases.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/AddColumnTestCases.scala
@@ -662,20 +662,6 @@
     checkExistence(df, true, "Inverted Index Columns city, col2")
   }
 
-  test("test if adding column in pre-aggregate table throws exception") {
-    sql("drop table if exists preaggMain")
-    sql("drop table if exists preagg1")
-    sql("create table preaggMain (a string, b string, c string) stored by 'carbondata'")
-    sql(
-      "create datamap preagg1 on table PreAggMain using 'preaggregate' as select" +
-      " a,sum(b) from PreAggMain group by a")
-    assert(intercept[ProcessMetaDataException] {
-      sql("alter table preaggmain_preagg1 add columns(d string)")
-    }.getMessage.contains("Cannot add columns"))
-    sql("drop table if exists preaggMain")
-    sql("drop table if exists preagg1")
-  }
-
   test("test rename textFileTable") {
     sql("drop table if exists renameTextFileTable")
     sql("drop table if exists new_renameTextFileTable")
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/AlterTableColumnRenameTestCase.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/AlterTableColumnRenameTestCase.scala
index e36e814..4f3a0cf 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/AlterTableColumnRenameTestCase.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/AlterTableColumnRenameTestCase.scala
@@ -244,22 +244,6 @@
     sql("drop table if exists bloomtable")
   }
 
-  test("test rename column on table where preagg exists") {
-    sql("DROP TABLE IF EXISTS maintable")
-    sql(
-      """
-        | CREATE TABLE maintable(id int, name string, city string, age int)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    sql(
-      s"""create datamap preagg_avg on table maintable using 'preaggregate' as select id,avg(age) from maintable group by id"""
-        .stripMargin)
-    intercept[Exception] {
-      sql("alter table maintable change id ids int")
-    }
-    sql("DROP TABLE IF EXISTS maintable")
-  }
-
   test("test rename on complex column") {
     sql("drop table if exists complex")
     sql(
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/ChangeDataTypeTestCases.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/ChangeDataTypeTestCases.scala
index dcf7e6d..eff7593 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/ChangeDataTypeTestCases.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/ChangeDataTypeTestCases.scala
@@ -149,23 +149,6 @@
     test_change_int_to_long()
   }
 
-  test("test data type change for with pre-aggregate table should throw exception") {
-    sql("drop table if exists preaggMain")
-    sql("drop table if exists PreAggMain_preagg1")
-    sql("create table preaggMain (a int, b string, c string) stored by 'carbondata'")
-    sql(
-      "create datamap preagg1 on table PreAggMain using 'preaggregate' as select" +
-      " a,sum(b) from PreAggMain group by a")
-    assert(intercept[ProcessMetaDataException] {
-      sql("alter table preaggmain change a a long").show
-    }.getMessage.contains("exists in a pre-aggregate table"))
-    assert(intercept[ProcessMetaDataException] {
-      sql("alter table preaggmain_preagg1 change a a long").show
-    }.getMessage.contains("Cannot change data type or rename column for columns in pre-aggregate table"))
-    sql("drop table if exists preaggMain")
-    sql("drop table if exists PreAggMain_preagg1")
-  }
-
   test("test data type change for dictionary exclude INT type column") {
     def test_change_data_type() = {
       beforeAll
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/DropColumnTestCases.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/DropColumnTestCases.scala
index e6285dd..990027e 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/DropColumnTestCases.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/DropColumnTestCases.scala
@@ -96,22 +96,6 @@
     test_drop_and_compaction()
   }
 
-  test("test dropping of column in pre-aggregate should throw exception") {
-    sql("drop table if exists preaggMain")
-    sql("drop table if exists preaggMain_preagg1")
-    sql("create table preaggMain (a string, b string, c string) stored by 'carbondata'")
-    sql(
-      "create datamap preagg1 on table PreAggMain using 'preaggregate' as select" +
-      " a,sum(b) from PreAggMain group by a")
-    sql("alter table preaggmain drop columns(c)")
-//    checkExistence(sql("desc table preaggmain"), false, "c")
-    assert(intercept[ProcessMetaDataException] {
-      sql("alter table preaggmain_preagg1 drop columns(preaggmain_b_sum)").show
-    }.getMessage.contains("Cannot drop columns in pre-aggreagate table"))
-    sql("drop table if exists preaggMain")
-    sql("drop table if exists preaggMain_preagg1")
-  }
-
   test("test dropping of complex column should throw exception") {
     sql("drop table if exists maintbl")
     sql("create table maintbl (a string, b string, c struct<si:int>) stored by 'carbondata'")
diff --git a/integration/spark2/src/test/scala/org/apache/spark/util/CarbonCommandSuite.scala b/integration/spark2/src/test/scala/org/apache/spark/util/CarbonCommandSuite.scala
index c2f7a44..2cb7f65 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/util/CarbonCommandSuite.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/util/CarbonCommandSuite.scala
@@ -152,21 +152,6 @@
     dropTable(table)
   }
 
-  test("test if delete segments by id is unsupported for pre-aggregate tables") {
-    dropTable("preaggMain")
-    dropTable("preaggMain_preagg1")
-    sql("create table preaggMain (a string, b string, c string) stored by 'carbondata'")
-    sql("create datamap preagg1 on table PreAggMain using 'preaggregate' as select a,sum(b) from PreAggMain group by a")
-    intercept[UnsupportedOperationException] {
-      sql("delete from table preaggMain where segment.id in (1,2)")
-    }.getMessage.contains("Delete segment operation is not supported on tables")
-    intercept[UnsupportedOperationException] {
-      sql("delete from table preaggMain_preagg1 where segment.id in (1,2)")
-    }.getMessage.contains("Delete segment operation is not supported on pre-aggregate tables")
-    dropTable("preaggMain")
-    dropTable("preagg1")
-  }
-
   test("separate visible and invisible segments info into two files") {
     val tableName = "test_tablestatus_history"
     sql(s"drop table if exists ${tableName}")
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/events/LoadEvents.java b/processing/src/main/java/org/apache/carbondata/processing/loading/events/LoadEvents.java
index 1365e49..144cb7a 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/events/LoadEvents.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/events/LoadEvents.java
@@ -50,7 +50,7 @@
 
   /**
    * Class for handling operations after data load completion and before final
-   * commit of load operation. Example usage: For loading pre-aggregate tables
+   * commit of load operation. Example usage: For loading MV
    */
 
   public static class LoadTablePostExecutionEvent extends Event {
@@ -74,7 +74,7 @@
 
   /**
    * Event for handling operations after data load completion and before final
-   * commit of load operation. Example usage: For loading pre-aggregate tables
+   * commit of load operation. Example usage: For loading MV
    */
 
   public static class LoadTablePreStatusUpdateEvent extends Event {
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 163526a..c923800 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
@@ -353,7 +353,7 @@
           loadMetadataDetails.setMajorCompacted("true");
         }
 
-        if (carbonTable.isChildTable()) {
+        if (carbonTable.isChildTableForMV()) {
           // If table is child table, then get segment mapping and set to extraInfo
           DataMapSchema dataMapSchema = null;
           try {
@@ -946,18 +946,14 @@
 
   /**
    * This method returns the valid segments attached to the table Identifier.
-   *
-   * @param absoluteTableIdentifier
-   * @return
    */
-  public static List<Segment> getValidSegmentList(AbsoluteTableIdentifier absoluteTableIdentifier,
-      Boolean isChildTable)
+  public static List<Segment> getValidSegmentList(CarbonTable carbonTable)
           throws IOException {
 
     SegmentStatusManager.ValidAndInvalidSegmentsInfo validAndInvalidSegments = null;
     try {
-      validAndInvalidSegments = new SegmentStatusManager(absoluteTableIdentifier)
-          .getValidAndInvalidSegments(isChildTable);
+      validAndInvalidSegments = new SegmentStatusManager(carbonTable.getAbsoluteTableIdentifier())
+          .getValidAndInvalidSegments(carbonTable.isChildTableForMV());
     } catch (IOException e) {
       LOGGER.error("Error while getting valid segment list for a table identifier");
       throw new IOException();
diff --git a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
index 622dfaa..e4d45a9 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
@@ -249,13 +249,7 @@
         FileFactory.mkdirs(metadataPath);
       }
     }
-    String tableStatusPath;
-    if (loadModel.getCarbonDataLoadSchema().getCarbonTable().isChildDataMap() && !uuid.isEmpty()) {
-      tableStatusPath = CarbonTablePath.getTableStatusFilePathWithUUID(
-          identifier.getTablePath(), uuid);
-    } else {
-      tableStatusPath = CarbonTablePath.getTableStatusFilePath(identifier.getTablePath());
-    }
+    String tableStatusPath = CarbonTablePath.getTableStatusFilePath(identifier.getTablePath());
     SegmentStatusManager segmentStatusManager = new SegmentStatusManager(identifier);
     ICarbonLock carbonLock = segmentStatusManager.getTableStatusLock();
     int retryCount = CarbonLockUtil
@@ -282,12 +276,7 @@
               String.valueOf(SegmentStatusManager.createNewSegmentId(listOfLoadFolderDetailsArray));
           loadModel.setLoadMetadataDetails(listOfLoadFolderDetails);
           LoadMetadataDetails entryTobeRemoved = null;
-          // Segment id would be provided in case this is compaction flow for aggregate data map.
-          // If that is true then used the segment id as the load name.
-          if (loadModel.getCarbonDataLoadSchema().getCarbonTable().isChildDataMap() && !loadModel
-              .getSegmentId().isEmpty()) {
-            newMetaEntry.setLoadName(loadModel.getSegmentId());
-          } else if (loadModel.getCarbonDataLoadSchema().getCarbonTable().isChildTable()
+          if (loadModel.getCarbonDataLoadSchema().getCarbonTable().isChildTableForMV()
               && !loadModel.getSegmentId().isEmpty()) {
             for (LoadMetadataDetails entry : listOfLoadFolderDetails) {
               if (entry.getLoadName().equalsIgnoreCase(loadModel.getSegmentId())) {
@@ -369,14 +358,9 @@
           }
         }
 
-        if (loadModel.getCarbonDataLoadSchema().getCarbonTable().isChildDataMap() && !loadStartEntry
-            && !uuid.isEmpty() && segmentsToBeDeleted.isEmpty() && !insertOverwrite) {
-          SegmentStatusManager.writeLoadDetailsIntoFile(tableStatusPath,
-              new LoadMetadataDetails[] { newMetaEntry });
-        } else {
-          SegmentStatusManager.writeLoadDetailsIntoFile(tableStatusPath, listOfLoadFolderDetails
-              .toArray(new LoadMetadataDetails[listOfLoadFolderDetails.size()]));
-        }
+        SegmentStatusManager.writeLoadDetailsIntoFile(tableStatusPath, listOfLoadFolderDetails
+            .toArray(new LoadMetadataDetails[0]));
+
         // Delete all old stale segment folders
         for (CarbonFile staleFolder : staleFolders) {
           // try block is inside for loop because even if there is failure in deletion of 1 stale