You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by ak...@apache.org on 2020/02/14 10:33:04 UTC

[carbondata] branch master updated: Why is this PR needed?

This is an automated email from the ASF dual-hosted git repository.

akashrn5 pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/carbondata.git


The following commit(s) were added to refs/heads/master by this push:
     new 60e9b85  Why is this PR needed?
60e9b85 is described below

commit 60e9b8571d3fcd0afe6bb400cd246767ef48d514
Author: Jacky Li <ja...@qq.com>
AuthorDate: Thu Feb 13 18:24:29 2020 +0800

    Why is this PR needed?
    
    In TableInfo, dataMapSchemaList and parentRelationIdentifiers are always empty.
    If user create index or mv, neither of their schema is stored in TableInfo,
    but only stored in systemfolder.
    
    What changes were proposed in this PR?
    
    This PR removes dataMapSchemaList and parentRelationIdentifiers in TableInfo class
    
    This closes #3619
---
 .../ThriftWrapperSchemaConverterImpl.java          | 15 +---
 .../core/metadata/schema/table/CarbonTable.java    | 40 -----------
 .../metadata/schema/table/CarbonTableBuilder.java  |  2 -
 .../core/metadata/schema/table/TableInfo.java      | 82 ----------------------
 .../ThriftWrapperSchemaConverterImplTest.java      |  1 -
 .../core/metadata/schema/table/TableInfoTest.java  | 13 ++--
 .../datamap/CarbonCreateDataMapCommand.scala       |  8 ---
 .../command/datamap/CarbonDataMapShowCommand.scala |  5 +-
 .../management/RefreshCarbonTableCommand.scala     | 70 +-----------------
 .../schema/CarbonAlterTableRenameCommand.scala     |  3 -
 .../command/table/CarbonDropTableCommand.scala     | 48 ++-----------
 .../sql/listeners/DropCacheEventListeners.scala    |  5 --
 .../sql/listeners/ShowCacheEventListeners.scala    | 14 ----
 13 files changed, 12 insertions(+), 294 deletions(-)

diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java b/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java
index f9ba679..34f3ec5 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java
@@ -312,16 +312,7 @@ public class ThriftWrapperSchemaConverterImpl implements SchemaConverter {
       TableInfo wrapperTableInfo, String dbName, String tableName) {
     org.apache.carbondata.format.TableSchema thriftFactTable =
         fromWrapperToExternalTableSchema(wrapperTableInfo.getFactTable());
-    org.apache.carbondata.format.TableInfo tableInfo =
-        new org.apache.carbondata.format.TableInfo(thriftFactTable,
-            new ArrayList<org.apache.carbondata.format.TableSchema>());
-    List<DataMapSchema> wrapperChildSchemaList = wrapperTableInfo.getDataMapSchemaList();
-    if (null != wrapperChildSchemaList) {
-      List<org.apache.carbondata.format.DataMapSchema> thriftChildSchemas =
-          fromWrapperToExternalChildSchemaList(wrapperChildSchemaList);
-      tableInfo.setDataMapSchemas(thriftChildSchemas);
-    }
-    return tableInfo;
+    return new org.apache.carbondata.format.TableInfo(thriftFactTable, new ArrayList<>());
   }
 
   private List<org.apache.carbondata.format.RelationIdentifier> fromWrapperToExternalRI(
@@ -657,10 +648,6 @@ public class ThriftWrapperSchemaConverterImpl implements SchemaConverter {
     wrapperTableInfo.setTableUniqueName(CarbonTable.buildUniqueName(dbName, tableName));
     wrapperTableInfo.setFactTable(
         fromExternalToWrapperTableSchema(externalTableInfo.getFact_table(), tableName));
-    if (null != externalTableInfo.getDataMapSchemas()) {
-      wrapperTableInfo.setDataMapSchemaList(
-          fromExternalToWrapperChildSchemaList(externalTableInfo.getDataMapSchemas()));
-    }
     wrapperTableInfo.setTablePath(tablePath);
     return wrapperTableInfo;
   }
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 f414e60..b61ba6c 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
@@ -126,9 +126,6 @@ public class CarbonTable implements Serializable, Writable {
   // The last index of the dimension column in all columns
   private int dimensionOrdinalMax;
 
-  // True if this table has datamap
-  private boolean hasDataMapSchema;
-
   // True if local dictionary is enabled for this table
   private boolean isLocalDictionaryEnabled;
 
@@ -151,32 +148,11 @@ public class CarbonTable implements Serializable, Writable {
    * This method will cast the same to the appropriate classes
    */
   private static void updateTableInfo(TableInfo tableInfo) {
-    List<DataMapSchema> dataMapSchemas = new ArrayList<>();
-    for (DataMapSchema dataMapSchema : tableInfo.getDataMapSchemaList()) {
-      DataMapSchema newDataMapSchema = DataMapSchemaFactory.INSTANCE
-          .getDataMapSchema(dataMapSchema.getDataMapName(), dataMapSchema.getProviderName());
-      newDataMapSchema.setChildSchema(dataMapSchema.getChildSchema());
-      newDataMapSchema.setProperties(dataMapSchema.getProperties());
-      newDataMapSchema.setRelationIdentifier(dataMapSchema.getRelationIdentifier());
-      dataMapSchemas.add(newDataMapSchema);
-    }
-    tableInfo.setDataMapSchemaList(dataMapSchemas);
     for (ColumnSchema columnSchema : tableInfo.getFactTable().getListOfColumns()) {
       columnSchema.setDataType(DataTypeUtil
           .valueOf(columnSchema.getDataType(), columnSchema.getPrecision(),
               columnSchema.getScale()));
     }
-    List<DataMapSchema> childSchema = tableInfo.getDataMapSchemaList();
-    for (DataMapSchema dataMapSchema : childSchema) {
-      if (dataMapSchema.childSchema != null
-          && dataMapSchema.childSchema.getListOfColumns().size() > 0) {
-        for (ColumnSchema columnSchema : dataMapSchema.childSchema.getListOfColumns()) {
-          columnSchema.setDataType(DataTypeUtil
-              .valueOf(columnSchema.getDataType(), columnSchema.getPrecision(),
-                  columnSchema.getScale()));
-        }
-      }
-    }
     if (tableInfo.getFactTable().getBucketingInfo() != null) {
       for (ColumnSchema columnSchema : tableInfo.getFactTable().getBucketingInfo()
           .getListOfColumns()) {
@@ -287,8 +263,6 @@ public class CarbonTable implements Serializable, Writable {
     if (tableInfo.getFactTable().getPartitionInfo() != null) {
       table.partition = tableInfo.getFactTable().getPartitionInfo();
     }
-    table.hasDataMapSchema =
-        null != tableInfo.getDataMapSchemaList() && tableInfo.getDataMapSchemaList().size() > 0;
     setLocalDictInfo(table, tableInfo);
   }
 
@@ -871,20 +845,6 @@ public class CarbonTable implements Serializable, Writable {
     return dimensionOrdinalMax;
   }
 
-  public boolean hasDataMapSchema() {
-    return hasDataMapSchema;
-  }
-
-  public DataMapSchema getDataMapSchema(String dataMapName) {
-    List<DataMapSchema> dataMaps = tableInfo.getDataMapSchemaList();
-    for (DataMapSchema dataMap : dataMaps) {
-      if (dataMap.getDataMapName().equalsIgnoreCase(dataMapName)) {
-        return dataMap;
-      }
-    }
-    return null;
-  }
-
   public boolean isChildTableForMV() {
     return null != tableInfo.getFactTable().getTableProperties()
         .get(CarbonCommonConstants.PARENT_TABLES) && !tableInfo.getFactTable().getTableProperties()
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableBuilder.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableBuilder.java
index a635033..ea6a3eb 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableBuilder.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableBuilder.java
@@ -17,7 +17,6 @@
 
 package org.apache.carbondata.core.metadata.schema.table;
 
-import java.util.ArrayList;
 import java.util.Objects;
 
 /**
@@ -72,7 +71,6 @@ public class CarbonTableBuilder {
     tableInfo.setTablePath(tablePath);
     tableInfo.setTransactionalTable(isTransactionalTable);
     tableInfo.setLastUpdatedTime(System.currentTimeMillis());
-    tableInfo.setDataMapSchemaList(new ArrayList<DataMapSchema>(0));
     return CarbonTable.buildFromTableInfo(tableInfo);
   }
 }
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java
index e8c786b..05a6186 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java
@@ -25,18 +25,13 @@ import java.io.DataOutput;
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
 import java.util.Map;
-import java.util.Set;
 
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
-import org.apache.carbondata.core.metadata.schema.table.column.ParentColumnTableRelation;
 
 import org.apache.log4j.Logger;
 
@@ -97,17 +92,12 @@ public class TableInfo implements Serializable, Writable {
   // this identifier is a lazy field which will be created when it is used first time
   private AbsoluteTableIdentifier identifier;
 
-  private List<DataMapSchema> dataMapSchemaList;
-
-  private List<RelationIdentifier> parentRelationIdentifiers;
-
   /**
    * flag to check whether any schema modification operation has happened after creation of table
    */
   private boolean isSchemaModified;
 
   public TableInfo() {
-    dataMapSchemaList = new ArrayList<>();
     isTransactionalTable = true;
   }
 
@@ -123,7 +113,6 @@ public class TableInfo implements Serializable, Writable {
    */
   public void setFactTable(TableSchema factTable) {
     this.factTable = factTable;
-    updateParentRelationIdentifier();
     updateIsSchemaModified();
     updateHasColumnDrift();
   }
@@ -139,22 +128,6 @@ public class TableInfo implements Serializable, Writable {
     }
   }
 
-  private void updateParentRelationIdentifier() {
-    Set<RelationIdentifier> parentRelationIdentifiers = new HashSet<>();
-    this.parentRelationIdentifiers = new ArrayList<>();
-    List<ColumnSchema> listOfColumns = this.factTable.getListOfColumns();
-    for (ColumnSchema columnSchema : listOfColumns) {
-      List<ParentColumnTableRelation> parentColumnTableRelations =
-          columnSchema.getParentColumnTableRelations();
-      if (null != parentColumnTableRelations) {
-        for (int i = 0; i < parentColumnTableRelations.size(); i++) {
-          parentRelationIdentifiers.add(parentColumnTableRelations.get(i).getRelationIdentifier());
-        }
-      }
-    }
-    this.parentRelationIdentifiers.addAll(parentRelationIdentifiers);
-  }
-
   /**
    * @return the databaseName
    */
@@ -201,14 +174,6 @@ public class TableInfo implements Serializable, Writable {
     this.tablePath = tablePath;
   }
 
-  public List<DataMapSchema> getDataMapSchemaList() {
-    return dataMapSchemaList;
-  }
-
-  public void setDataMapSchemaList(List<DataMapSchema> dataMapSchemaList) {
-    this.dataMapSchemaList = dataMapSchemaList;
-  }
-
   /**
    * to generate the hash code
    */
@@ -283,24 +248,6 @@ public class TableInfo implements Serializable, Writable {
     out.writeUTF(getOrCreateAbsoluteTableIdentifier().getTablePath());
     out.writeBoolean(isTransactionalTable);
     out.writeBoolean(hasColumnDrift);
-    boolean isChildSchemaExists =
-        null != dataMapSchemaList && dataMapSchemaList.size() > 0;
-    out.writeBoolean(isChildSchemaExists);
-    if (isChildSchemaExists) {
-      out.writeShort(dataMapSchemaList.size());
-      for (int i = 0; i < dataMapSchemaList.size(); i++) {
-        dataMapSchemaList.get(i).write(out);
-      }
-    }
-    boolean isParentTableRelationIndentifierExists =
-        null != parentRelationIdentifiers && parentRelationIdentifiers.size() > 0;
-    out.writeBoolean(isParentTableRelationIndentifierExists);
-    if (isParentTableRelationIndentifierExists) {
-      out.writeShort(parentRelationIdentifiers.size());
-      for (int i = 0; i < parentRelationIdentifiers.size(); i++) {
-        parentRelationIdentifiers.get(i).write(out);
-      }
-    }
     out.writeBoolean(isSchemaModified);
   }
 
@@ -314,31 +261,6 @@ public class TableInfo implements Serializable, Writable {
     this.tablePath = in.readUTF();
     this.isTransactionalTable = in.readBoolean();
     this.hasColumnDrift = in.readBoolean();
-    boolean isChildSchemaExists = in.readBoolean();
-    this.dataMapSchemaList = new ArrayList<>();
-    if (isChildSchemaExists) {
-      short numberOfChildTable = in.readShort();
-      for (int i = 0; i < numberOfChildTable; i++) {
-        DataMapSchema childSchema = new DataMapSchema();
-        childSchema.readFields(in);
-        DataMapSchema dataMapSchema = DataMapSchemaFactory.INSTANCE
-            .getDataMapSchema(childSchema.getDataMapName(), childSchema.getProviderName());
-        dataMapSchema.setChildSchema(childSchema.getChildSchema());
-        dataMapSchema.setRelationIdentifier(childSchema.getRelationIdentifier());
-        dataMapSchema.setProperties(childSchema.getProperties());
-        dataMapSchemaList.add(dataMapSchema);
-      }
-    }
-    boolean isParentTableRelationIdentifierExists = in.readBoolean();
-    if (isParentTableRelationIdentifierExists) {
-      short parentTableIdentifiersListSize = in.readShort();
-      this.parentRelationIdentifiers = new ArrayList<>();
-      for (int i = 0; i < parentTableIdentifiersListSize; i++) {
-        RelationIdentifier relationIdentifier = new RelationIdentifier(null, null, null);
-        relationIdentifier.readFields(in);
-        this.parentRelationIdentifiers.add(relationIdentifier);
-      }
-    }
     this.isSchemaModified = in.readBoolean();
   }
 
@@ -364,10 +286,6 @@ public class TableInfo implements Serializable, Writable {
     return tableInfo;
   }
 
-  public List<RelationIdentifier> getParentRelationIdentifiers() {
-    return parentRelationIdentifiers;
-  }
-
   public boolean isTransactionalTable() {
     return isTransactionalTable;
   }
diff --git a/core/src/test/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImplTest.java b/core/src/test/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImplTest.java
index d39dec9..ca0ef87 100644
--- a/core/src/test/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImplTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImplTest.java
@@ -1434,7 +1434,6 @@ public class ThriftWrapperSchemaConverterImplTest {
     org.apache.carbondata.format.TableInfo expectedResult =
         new org.apache.carbondata.format.TableInfo(thriftFactTable, new ArrayList<org.apache
             .carbondata.format.TableSchema>());
-    expectedResult.setDataMapSchemas(new ArrayList<DataMapSchema>());
     assertEquals(expectedResult, actualResult);
   }
 
diff --git a/core/src/test/java/org/apache/carbondata/core/metadata/schema/table/TableInfoTest.java b/core/src/test/java/org/apache/carbondata/core/metadata/schema/table/TableInfoTest.java
index 6341535..37d7c67 100644
--- a/core/src/test/java/org/apache/carbondata/core/metadata/schema/table/TableInfoTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/metadata/schema/table/TableInfoTest.java
@@ -18,7 +18,6 @@
 package org.apache.carbondata.core.metadata.schema.table;
 
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
 
 import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl;
@@ -87,8 +86,7 @@ public class TableInfoTest extends TestCase {
         + "\"tableProperties\":{\"sort_columns\":\"c1\",\"comment\":\"\","
         + "\"local_dictionary_enable\":\"true\"}},\"lastUpdatedTime\":1530534235537,"
         + "\"tablePath\":\"/store/carbonversion_1_1/testinttype1\","
-        + "\"isTransactionalTable\":true,\"dataMapSchemaList\":[],"
-        + "\"parentRelationIdentifiers\":[],\"isSchemaModified\":false}");
+        + "\"isTransactionalTable\":true,\"isSchemaModified\":false}");
     TableInfo tableInfo = CarbonUtil.convertGsonToTableInfo(properties);
     // the schema evolution should not be null
     assertTrue(null != tableInfo.getFactTable().getSchemaEvolution());
@@ -204,11 +202,8 @@ public class TableInfoTest extends TestCase {
         + "\"parentRelationIdentifiers\":[],\"isSchemaModified\":false}");
     TableInfo tableInfo = CarbonUtil.convertGsonToTableInfo(properties);
     // the schema evolution should not be null
-    assertTrue(null != tableInfo.getDataMapSchemaList());
-    List<DataMapSchema> dataMapSchemaList = tableInfo.getDataMapSchemaList();
-    for (DataMapSchema dataMapSchema : dataMapSchemaList) {
-      String providerName = dataMapSchema.getProviderName();
-      assertTrue("org.apache.carbondata.core.datamap.AggregateDataMap".equals(providerName));
-    }
+    assertTrue(null != tableInfo.getFactTable());
+    assertTrue(null != tableInfo.getFactTable().getListOfColumns());
+    assertTrue(null != tableInfo.getFactTable().getTableProperties());
   }
 }
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 38b25a5..5838b38 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
@@ -72,14 +72,6 @@ case class CarbonCreateDataMapCommand(
       throw new MalformedCarbonCommandException("Unsupported operation on non transactional table")
     }
 
-    if (mainTable != null && mainTable.getDataMapSchema(dataMapName) != null) {
-      if (!ifNotExistsSet) {
-        throw new MalformedDataMapCommandException(s"DataMap name '$dataMapName' already exist")
-      } else {
-        return Seq.empty
-      }
-    }
-
     if (mainTable != null && CarbonUtil.getFormatVersion(mainTable) != ColumnarFormatVersion.V3) {
       throw new MalformedCarbonCommandException(s"Unsupported operation on table with " +
                                                 s"V1 or V2 format data")
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 b23974a..68ee59d 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
@@ -32,7 +32,7 @@ import org.apache.spark.sql.types.StringType
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datamap.{DataMapStoreManager, DataMapUtil}
 import org.apache.carbondata.core.datamap.status.{DataMapSegmentStatusUtil, DataMapStatus, DataMapStatusManager}
-import org.apache.carbondata.core.metadata.schema.datamap.{DataMapClassProvider, DataMapProperty}
+import org.apache.carbondata.core.metadata.schema.datamap.DataMapProperty
 import org.apache.carbondata.core.metadata.schema.table.DataMapSchema
 import org.apache.carbondata.core.statusmanager.{SegmentStatus, SegmentStatusManager}
 import org.apache.carbondata.core.util.path.CarbonTablePath
@@ -68,9 +68,6 @@ case class CarbonDataMapShowCommand(tableIdentifier: Option[TableIdentifier])
         val carbonTable = CarbonEnv.getCarbonTable(table)(sparkSession)
         setAuditTable(carbonTable)
         Checker.validateTableExists(table.database, table.table, sparkSession)
-        if (carbonTable.hasDataMapSchema) {
-          dataMapSchemaList.addAll(carbonTable.getTableInfo.getDataMapSchemaList)
-        }
         val indexSchemas = DataMapStoreManager.getInstance().getDataMapSchemasOfTable(carbonTable)
         if (!indexSchemas.isEmpty) {
           dataMapSchemaList.addAll(indexSchemas)
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/RefreshCarbonTableCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/RefreshCarbonTableCommand.scala
index 18190e5..a55b477 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/RefreshCarbonTableCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/RefreshCarbonTableCommand.scala
@@ -17,8 +17,6 @@
 
 package org.apache.spark.sql.execution.command.management
 
-import java.util
-
 import scala.collection.JavaConverters._
 import scala.collection.mutable
 
@@ -37,7 +35,7 @@ import org.apache.carbondata.core.indexstore.PartitionSpec
 import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
 import org.apache.carbondata.core.metadata.schema.SchemaReader
 import org.apache.carbondata.core.metadata.schema.partition.PartitionType
-import org.apache.carbondata.core.metadata.schema.table.{DataMapSchema, TableInfo}
+import org.apache.carbondata.core.metadata.schema.table. TableInfo
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema
 import org.apache.carbondata.core.statusmanager.{SegmentStatus, SegmentStatusManager}
 import org.apache.carbondata.core.util.path.CarbonTablePath
@@ -88,24 +86,7 @@ case class RefreshCarbonTableCommand(
         // refresh the column schema in case of store before V3
         refreshColumnSchema(tableInfo)
 
-        // 2.2 register the table with the hive check if the table being registered has
-        // aggregate table then do the below steps
-        // 2.2.1 validate that all the aggregate tables are copied at the store location.
-        val dataMapSchemaList = tableInfo.getDataMapSchemaList
-        if (null != dataMapSchemaList && dataMapSchemaList.size() != 0) {
-          // validate all the aggregate tables are copied at the storeLocation
-          val allExists = validateAllAggregateTablePresent(databaseName,
-            dataMapSchemaList, sparkSession)
-          if (!allExists) {
-            // fail the register operation
-            val msg = s"Table registration with Database name [$databaseName] and Table name " +
-                      s"[$tableName] failed. All the aggregate Tables for table [$tableName] is" +
-                      s" not copied under database [$databaseName]"
-            throwMetadataException(databaseName, tableName, msg)
-          }
-          // 2.2.1 Register the aggregate tables to hive
-          registerAggregates(databaseName, dataMapSchemaList)(sparkSession)
-        }
+        // 2.2 register the table with the hive
         registerTableWithHive(databaseName, tableName, tableInfo, tablePath)(sparkSession)
         // Register partitions to hive metastore in case of hive partitioning carbon table
         if (tableInfo.getFactTable.getPartitionInfo != null &&
@@ -192,53 +173,6 @@ case class RefreshCarbonTableCommand(
   }
 
   /**
-   * The method validate that all the aggregate table are physically present
-   *
-   * @param dataMapSchemaList
-   * @param sparkSession
-   */
-  def validateAllAggregateTablePresent(dbName: String, dataMapSchemaList: util.List[DataMapSchema],
-      sparkSession: SparkSession): Boolean = {
-    var fileExist = false
-    dataMapSchemaList.asScala.foreach(dataMap => {
-      val tableName = dataMap.getChildSchema.getTableName
-      val tablePath = CarbonEnv.getTablePath(Some(dbName), tableName)(sparkSession)
-      val schemaFilePath = CarbonTablePath.getSchemaFilePath(tablePath)
-      try {
-        fileExist = FileFactory.isFileExist(schemaFilePath)
-      } catch {
-        case e: Exception =>
-          fileExist = false
-      }
-      if (!fileExist) {
-        return fileExist;
-      }
-    })
-    true
-  }
-
-  /**
-   * The method iterates over all the aggregate tables and register them to hive
-   *
-   * @param dataMapSchemaList
-   * @return
-   */
-  def registerAggregates(dbName: String,
-      dataMapSchemaList: util.List[DataMapSchema])(sparkSession: SparkSession): Any = {
-    dataMapSchemaList.asScala.foreach(dataMap => {
-      val tableName = dataMap.getChildSchema.getTableName
-      if (!sparkSession.sessionState.catalog.listTables(dbName)
-        .exists(_.table.equalsIgnoreCase(tableName))) {
-        val tablePath = CarbonEnv.getTablePath(Some(dbName), tableName)(sparkSession)
-        val absoluteTableIdentifier = AbsoluteTableIdentifier
-          .from(tablePath, dbName, tableName)
-        val tableInfo = SchemaReader.getTableInfo(absoluteTableIdentifier)
-        registerTableWithHive(dbName, tableName, tableInfo, tablePath)(sparkSession)
-      }
-    })
-  }
-
-  /**
    * Read all the partition information which is stored in each segment and add to
    * the hive metastore
    */
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 ff7931f..972be77 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
@@ -101,9 +101,6 @@ private[sql] case class CarbonAlterTableRenameCommand(
       }
       // get the old table all data map schema
       val dataMapSchemaList: util.List[DataMapSchema] = new util.ArrayList[DataMapSchema]()
-      if (carbonTable.hasDataMapSchema) {
-        dataMapSchemaList.addAll(carbonTable.getTableInfo.getDataMapSchemaList)
-      }
       val indexSchemas = DataMapStoreManager.getInstance().getDataMapSchemasOfTable(carbonTable)
       if (!indexSchemas.isEmpty) {
         dataMapSchemaList.addAll(indexSchemas)
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 bb687e3..e0c153c 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
@@ -87,19 +87,6 @@ case class CarbonDropTableCommand(
         // streaming table should acquire streaming.lock
         carbonLocks += CarbonLockUtil.getLockObject(identifier, LockUsage.STREAMING_LOCK)
       }
-      val relationIdentifiers = carbonTable.getTableInfo.getParentRelationIdentifiers
-      if (relationIdentifiers != null && !relationIdentifiers.isEmpty) {
-        var ignoreParentTableCheck = false
-        if (!ignoreParentTableCheck && !dropChildTable) {
-          if (!ifExistsSet) {
-            throwMetadataException(dbName, tableName,
-              "Child table which is associated with datamap cannot be dropped, " +
-              "use DROP DATAMAP command to drop")
-          } else {
-            return Seq.empty
-          }
-        }
-      }
       val operationContext = new OperationContext
       val dropTablePreEvent: DropTablePreEvent =
         DropTablePreEvent(
@@ -110,33 +97,10 @@ case class CarbonDropTableCommand(
 
       CarbonEnv.getInstance(sparkSession).carbonMetaStore.dropTable(identifier)(sparkSession)
 
-      if (carbonTable.hasDataMapSchema) {
-        // drop all child tables
-       val childSchemas = carbonTable.getTableInfo.getDataMapSchemaList
-
-        childDropCommands = childSchemas.asScala
-          .filter(_.getRelationIdentifier != null)
-          .map { childSchema =>
-            val childTable =
-              CarbonEnv.getCarbonTable(
-                TableIdentifier(childSchema.getRelationIdentifier.getTableName,
-                  Some(childSchema.getRelationIdentifier.getDatabaseName)))(sparkSession)
-            val dropCommand = CarbonDropTableCommand(
-              ifExistsSet = true,
-              Some(childSchema.getRelationIdentifier.getDatabaseName),
-              childSchema.getRelationIdentifier.getTableName,
-              dropChildTable = true
-            )
-            dropCommand.carbonTable = childTable
-            dropCommand
-          }
-        childDropCommands.foreach(_.processMetadata(sparkSession))
-      }
-      val indexDatamapSchemas =
-        DataMapStoreManager.getInstance().getDataMapSchemasOfTable(carbonTable)
-      LOGGER.info(s"Dropping DataMaps in table $tableName, size: ${indexDatamapSchemas.size()}")
-      if (!indexDatamapSchemas.isEmpty) {
-        childDropDataMapCommands = indexDatamapSchemas.asScala.map { schema =>
+      val dataMapSchemas = DataMapStoreManager.getInstance().getDataMapSchemasOfTable(carbonTable)
+      LOGGER.info(s"Dropping DataMaps in table $tableName, size: ${dataMapSchemas.size()}")
+      if (!dataMapSchemas.isEmpty) {
+        childDropDataMapCommands = dataMapSchemas.asScala.map { schema =>
           val command = CarbonDropDataMapCommand(schema.getDataMapName,
             ifExistsSet,
             Some(TableIdentifier(tableName, Some(dbName))),
@@ -214,10 +178,6 @@ case class CarbonDropTableCommand(
         val file = FileFactory.getCarbonFile(tableLockPath)
         CarbonUtil.deleteFoldersAndFilesSilent(file)
       }
-      if (carbonTable.hasDataMapSchema && childDropCommands.nonEmpty) {
-        // drop all child tables
-        childDropCommands.foreach(_.processData(sparkSession))
-      }
       childDropDataMapCommands.foreach(_.processData(sparkSession))
     }
     Seq.empty
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/listeners/DropCacheEventListeners.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/listeners/DropCacheEventListeners.scala
index 3be0606..91a8caa 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/listeners/DropCacheEventListeners.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/listeners/DropCacheEventListeners.scala
@@ -52,11 +52,6 @@ object DropCacheDataMapEventListener extends OperationEventListener {
           throw new UnsupportedOperationException("Operation not allowed on child table.")
         }
 
-        if (carbonTable.hasDataMapSchema) {
-          val childrenSchemas = carbonTable.getTableInfo.getDataMapSchemaList.asScala
-            .filter(_.getRelationIdentifier != null)
-          dropCacheForChildTables(sparkSession, childrenSchemas)
-        }
         if (DataMapUtil.hasMVDataMap(carbonTable)) {
           val childrenSchemas = DataMapStoreManager.getInstance
             .getDataMapSchemasOfTable(carbonTable).asScala
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/listeners/ShowCacheEventListeners.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/listeners/ShowCacheEventListeners.scala
index cfd88e5..e38ae79 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/listeners/ShowCacheEventListeners.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/listeners/ShowCacheEventListeners.scala
@@ -43,20 +43,6 @@ object ShowCachePreMVEventListener extends OperationEventListener {
         if (carbonTable.isChildTableForMV && !internalCall) {
           throw new UnsupportedOperationException("Operation not allowed on child table.")
         }
-
-        val childTables = operationContext.getProperty(carbonTable.getTableUniqueName)
-          .asInstanceOf[List[(String, String)]]
-
-        if (carbonTable.hasDataMapSchema) {
-          val childrenSchemas = carbonTable.getTableInfo.getDataMapSchemaList.asScala
-            .filter(_.getRelationIdentifier != null)
-          operationContext.setProperty(carbonTable.getTableUniqueName, childrenSchemas.collect {
-            case childSchema if childSchema.getRelationIdentifier != null =>
-              (s"${ childSchema.getRelationIdentifier.getDatabaseName }-${
-                childSchema.getRelationIdentifier.getTableName
-              }", childSchema.getProviderName, childSchema.getRelationIdentifier.getTableId)
-          }.toList ++ childTables)
-        }
     }
   }
 }