You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by ra...@apache.org on 2019/06/22 09:09:55 UTC

[carbondata] branch master updated: [CARBONDATA-3398] Handled show cache for index server and MV

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

ravipesala 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 f708efb  [CARBONDATA-3398] Handled show cache for index server and MV
f708efb is described below

commit f708efb183d0247f9d6a46f7dff6bb4507998f3f
Author: kunal642 <ku...@gmail.com>
AuthorDate: Tue May 28 15:30:49 2019 +0530

    [CARBONDATA-3398] Handled show cache for index server and MV
    
    Added support to show/drop metacahe information from index server.
    Added tableNotFoundException fix when dbName and tableName have '' in their names, while splitting using '' the dbName was extracted wrongly. Instead now dbname and tableName would be seperated by '-' internally for show cache
    
    This closes #3245
---
 .../core/datamap/dev/DataMapFactory.java           |   4 +
 .../core/indexstore/BlockletDetailsFetcher.java    |   2 -
 .../blockletindex/BlockletDataMapFactory.java      |  13 +-
 .../bloom/BloomCoarseGrainDataMapFactory.java      |  16 +
 .../hadoop/api/CarbonTableInputFormat.java         |   4 +-
 .../sql/commands/TestCarbonShowCacheCommand.scala  |  35 +-
 .../apache/carbondata/spark/util/CommonUtil.scala  |   9 +-
 .../carbondata/indexserver/DataMapJobs.scala       |   2 +-
 .../indexserver/DistributedShowCacheRDD.scala      |  32 +-
 .../carbondata/indexserver/IndexServer.scala       |   9 +-
 .../scala/org/apache/spark/sql/CarbonEnv.scala     |   2 +-
 .../command/cache/CarbonShowCacheCommand.scala     | 465 ++++++++++++++-------
 .../command/cache/ShowCacheEventListeners.scala    |  78 ++--
 .../scala/org/apache/spark/util/DataMapUtil.scala  |   2 +-
 14 files changed, 428 insertions(+), 245 deletions(-)

diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java
index 3fa7be6..1116525 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java
@@ -192,4 +192,8 @@ public abstract class DataMapFactory<T extends DataMap> {
   public boolean supportRebuild() {
     return false;
   }
+
+  public String getCacheSize() {
+    return null;
+  }
 }
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDetailsFetcher.java b/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDetailsFetcher.java
index 5eace3c..ae01e9e 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDetailsFetcher.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDetailsFetcher.java
@@ -60,6 +60,4 @@ public interface BlockletDetailsFetcher {
    * clears the datamap from cache and segmentMap from executor
    */
   void clear();
-
-  String getCacheSize() throws IOException ;
 }
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java
index cab1b8b..f928976 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java
@@ -302,14 +302,19 @@ public class BlockletDataMapFactory extends CoarseGrainDataMapFactory
     }
   }
 
-  @Override public String getCacheSize() throws IOException {
+  @Override
+  public String getCacheSize() {
     long sum = 0L;
     int numOfIndexFiles = 0;
     for (Map.Entry<String, Set<TableBlockIndexUniqueIdentifier>> entry : segmentMap.entrySet()) {
       for (TableBlockIndexUniqueIdentifier tableBlockIndexUniqueIdentifier : entry.getValue()) {
-        sum += cache.get(new TableBlockIndexUniqueIdentifierWrapper(tableBlockIndexUniqueIdentifier,
-            getCarbonTable())).getMemorySize();
-        numOfIndexFiles++;
+        BlockletDataMapIndexWrapper blockletDataMapIndexWrapper = cache.getIfPresent(
+            new TableBlockIndexUniqueIdentifierWrapper(tableBlockIndexUniqueIdentifier,
+                getCarbonTable()));
+        if (blockletDataMapIndexWrapper != null) {
+          sum += blockletDataMapIndexWrapper.getMemorySize();
+          numOfIndexFiles++;
+        }
       }
     }
     return numOfIndexFiles + ":" + sum;
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 03599a9..f261871 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
@@ -453,4 +453,20 @@ public class BloomCoarseGrainDataMapFactory extends DataMapFactory<CoarseGrainDa
   public DataMapLevel getDataMapLevel() {
     return DataMapLevel.CG;
   }
+
+  @Override
+  public String getCacheSize() {
+    long sum = 0L;
+    for (Map.Entry<String, Set<String>> entry : segmentMap.entrySet()) {
+      for (String shardName : entry.getValue()) {
+        for (CarbonColumn carbonColumn : dataMapMeta.getIndexedColumns()) {
+          BloomCacheKeyValue.CacheValue cacheValue = cache
+              .getIfPresent(new BloomCacheKeyValue.CacheKey(shardName, carbonColumn.getColName()));
+          if (cacheValue != null) {
+            sum += cacheValue.getMemorySize();
+          }
+        }
+      }
+    } return 0 + ":" + sum;
+  }
 }
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 6c99142..90ff520 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
@@ -565,7 +565,9 @@ public class CarbonTableInputFormat<T> extends CarbonInputFormat<T> {
                   allSegments.getInvalidSegments(), toBeCleanedSegments));
           for (InputSplit extendedBlocklet : extendedBlocklets) {
             CarbonInputSplit blocklet = (CarbonInputSplit) extendedBlocklet;
-            blockletToRowCountMap.put(blocklet.getSegmentId() + "," + blocklet.getFilePath(),
+            String filePath = blocklet.getFilePath().replace("\\", "/");
+            String blockName = filePath.substring(filePath.lastIndexOf("/") + 1);
+            blockletToRowCountMap.put(blocklet.getSegmentId() + "," + blockName,
                 (long) blocklet.getRowCount());
           }
         } catch (Exception e) {
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 35ac2e3..eb4b769 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
@@ -124,17 +124,17 @@ class TestCarbonShowCacheCommand extends QueryTest with BeforeAndAfterAll {
     sql(s"LOAD DATA INPATH '$resourcesPath/data.csv' INTO TABLE empTable")
     sql("select count(*) from empTable").show()
     var showCache = sql("SHOW METACACHE on table empTable").collect()
-    assert(showCache(0).get(2).toString.equalsIgnoreCase("3/3 index files cached"))
+    assert(showCache(1).get(2).toString.equalsIgnoreCase("3/3 index files cached"))
     sql("delete from table empTable where segment.id in(0)").show()
     // check whether count(*) query invalidates the cache for the invalid segments
     sql("select count(*) from empTable").show()
     showCache = sql("SHOW METACACHE on table empTable").collect()
-    assert(showCache(0).get(2).toString.equalsIgnoreCase("2/2 index files cached"))
+    assert(showCache(1).get(2).toString.equalsIgnoreCase("2/2 index files cached"))
     sql("delete from table empTable where segment.id in(1)").show()
     // check whether select * query invalidates the cache for the invalid segments
     sql("select * from empTable").show()
     showCache = sql("SHOW METACACHE on table empTable").collect()
-    assert(showCache(0).get(2).toString.equalsIgnoreCase("1/1 index files cached"))
+    assert(showCache(1).get(2).toString.equalsIgnoreCase("1/1 index files cached"))
   }
 
   test("test external table show cache") {
@@ -186,7 +186,7 @@ class TestCarbonShowCacheCommand extends QueryTest with BeforeAndAfterAll {
     // Database with 3 tables but only 2 are in cache
     sql("use cache_db").collect()
     val result2 = sql("show metacache").collect()
-    assertResult(4)(result2.length)
+    assertResult(5)(result2.length)
 
     // Make sure PreAgg tables are not in SHOW METADATA
     sql("use default").collect()
@@ -202,32 +202,33 @@ class TestCarbonShowCacheCommand extends QueryTest with BeforeAndAfterAll {
 
     // Table with Index, Dictionary & Bloom filter
     val result1 = sql("show metacache on table cache_1").collect()
-    assertResult(3)(result1.length)
-    assertResult("1/1 index files cached")(result1(0).getString(2))
-    assertResult("bloomfilter")(result1(2).getString(2))
+    assertResult(4)(result1.length)
+    assertResult("1/1 index files cached")(result1(1).getString(2))
+    assertResult("bloomfilter")(result1(3).getString(2))
 
     // Table with Index and Dictionary
     val result2 = sql("show metacache on table cache_db.cache_2").collect()
-    assertResult(2)(result2.length)
-    assertResult("2/2 index files cached")(result2(0).getString(2))
-    assertResult("0 B")(result2(1).getString(1))
+    assertResult(3)(result2.length)
+    assertResult("2/2 index files cached")(result2(1).getString(2))
+    assertResult("0 B")(result2(2).getString(1))
 
     // Table not in cache
     checkAnswer(sql("show metacache on table cache_db.cache_3"),
-      Seq(Row("Index", "0 B", "0/1 index files cached"), Row("Dictionary", "0 B", "")))
+      Seq(Row("DRIVER CACHE","",""), Row("Index", "0 B", "0/1 index files cached"),
+        Row("Dictionary", "0 B", "")))
 
     // 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))
+    assertResult(4)(result4.length)
+    assertResult("1/1 index files cached")(result4(1).getString(2))
+    assertResult("0 B")(result4(2).getString(1))
+    assertResult("preaggregate")(result4(3).getString(2))
 
     sql("use default").collect()
 
     // Table with 5 index files
     val result5 = sql("show metacache on table cache_5").collect()
-    assertResult(2)(result5.length)
-    assertResult("5/5 index files cached")(result5(0).getString(2))
+    assertResult(3)(result5.length)
+    assertResult("5/5 index files cached")(result5(1).getString(2))
   }
 }
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
index 1c89a0c..7015279 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
@@ -985,7 +985,14 @@ object CommonUtil {
     }
   }
 
-  def bytesToDisplaySize(size: Long): String = bytesToDisplaySize(BigDecimal.valueOf(size))
+  def bytesToDisplaySize(size: Long): String = {
+    try {
+      bytesToDisplaySize(BigDecimal.valueOf(size))
+    } catch {
+      case _: Exception =>
+        size.toString
+    }
+  }
 
   // This method converts the bytes count to display size upto 2 decimal places
   def bytesToDisplaySize(size: BigDecimal): String = {
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/indexserver/DataMapJobs.scala b/integration/spark2/src/main/scala/org/apache/carbondata/indexserver/DataMapJobs.scala
index 4acfc33..363558e 100644
--- a/integration/spark2/src/main/scala/org/apache/carbondata/indexserver/DataMapJobs.scala
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/indexserver/DataMapJobs.scala
@@ -111,7 +111,7 @@ class DistributedDataMapJob extends AbstractDataMapJob {
           asInstanceOf[BinaryExpression])
     }
     if (filterInf.isInstanceOf[RowLevelFilterResolverImpl] &&
-      filterInf.getFilterExecuterType == ExpressionType.UNKNOWN) {
+        filterInf.getFilterExpression.getFilterExpressionType == ExpressionType.UNKNOWN) {
       return filterProcessor.changeUnknownResloverToTrue(tableIdentifer)
     }
     return filterInf;
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/indexserver/DistributedShowCacheRDD.scala b/integration/spark2/src/main/scala/org/apache/carbondata/indexserver/DistributedShowCacheRDD.scala
index 172ea47..78b7e72 100644
--- a/integration/spark2/src/main/scala/org/apache/carbondata/indexserver/DistributedShowCacheRDD.scala
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/indexserver/DistributedShowCacheRDD.scala
@@ -16,20 +16,26 @@
  */
 package org.apache.carbondata.indexserver
 
+import scala.collection.JavaConverters._
+
 import org.apache.spark.{Partition, TaskContext}
 import org.apache.spark.sql.SparkSession
 import org.apache.spark.sql.hive.DistributionUtil
-import scala.collection.JavaConverters._
 
-import org.apache.carbondata.core.cache.CacheProvider
 import org.apache.carbondata.core.datamap.DataMapStoreManager
+import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataMapFactory
 import org.apache.carbondata.hadoop.CarbonInputSplit
 import org.apache.carbondata.spark.rdd.CarbonRDD
 
 class DistributedShowCacheRDD(@transient private val ss: SparkSession, tableName: String)
   extends CarbonRDD[String](ss, Nil) {
 
-  val executorsList: Array[String] = DistributionUtil.getNodeList(ss.sparkContext)
+  val executorsList: Array[String] = DistributionUtil.getExecutors(ss.sparkContext).flatMap {
+    case (host, executors) =>
+      executors.map {
+        executor => s"executor_${host}_$executor"
+      }
+  }.toArray
 
   override protected def internalGetPartitions: Array[Partition] = {
     executorsList.zipWithIndex.map {
@@ -43,15 +49,21 @@ class DistributedShowCacheRDD(@transient private val ss: SparkSession, tableName
 
   override def internalCompute(split: Partition, context: TaskContext): Iterator[String] = {
     val dataMaps = DataMapStoreManager.getInstance().getAllDataMaps.asScala
+    val tableList = tableName.split(",").map(_.replace("-", "_"))
     val iterator = dataMaps.collect {
-      case (table, tableDataMaps) if table.isEmpty ||
-                                     (tableName.nonEmpty && tableName.equalsIgnoreCase(table)) =>
+      case (table, tableDataMaps) if tableName.isEmpty || tableList.contains(table) =>
         val sizeAndIndexLengths = tableDataMaps.asScala
-          .map(_.getBlockletDetailsFetcher.getCacheSize)
-        // return tableName_indexFileLength_indexCachesize for each executor.
-        sizeAndIndexLengths.map {
-          x => s"$table:$x"
-        }
+          .map { dataMap =>
+            val dataMapName = if (dataMap.getDataMapFactory.isInstanceOf[BlockletDataMapFactory]) {
+              table
+            } else {
+              dataMap.getDataMapSchema.getRelationIdentifier.getDatabaseName + "_" + dataMap
+              .getDataMapSchema.getDataMapName
+            }
+            s"${ dataMapName }:${ dataMap.getDataMapFactory.getCacheSize }:${
+              dataMap.getDataMapSchema.getProviderName} "
+          }
+        sizeAndIndexLengths
     }.flatten.toIterator
     iterator
   }
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/indexserver/IndexServer.scala b/integration/spark2/src/main/scala/org/apache/carbondata/indexserver/IndexServer.scala
index 295ebe1..051b9de 100644
--- a/integration/spark2/src/main/scala/org/apache/carbondata/indexserver/IndexServer.scala
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/indexserver/IndexServer.scala
@@ -45,9 +45,9 @@ trait ServerInterface {
   def getSplits(request: DistributableDataMapFormat): ExtendedBlockletWrapperContainer
 
   /**
-   * Get the cache size for the specified table.
+   * Get the cache size for the specified tables.
    */
-  def showCache(tableName: String) : Array[String]
+  def showCache(tableNames: String) : Array[String]
 
   /**
    * Invalidate the cache for the specified segments only. Used in case of compaction/Update/Delete.
@@ -174,6 +174,7 @@ object IndexServer extends ServerInterface {
    * @return Return a new Client to communicate with the Index Server.
    */
   def getClient: ServerInterface = {
+    val configuration = SparkSQLUtil.sessionState(sparkSession).newHadoopConf()
     import org.apache.hadoop.ipc.RPC
     val indexServerUser = sparkSession.sparkContext.getConf
       .get("spark.carbon.indexserver.principal", "")
@@ -181,10 +182,10 @@ object IndexServer extends ServerInterface {
       .get("spark.carbon.indexserver.keytab", "")
     val ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(indexServerUser,
       indexServerKeyTab)
-    LOGGER.info("Login successful for user " + indexServerUser);
+    LOGGER.info("Login successful for user " + indexServerUser)
     RPC.getProxy(classOf[ServerInterface],
       RPC.getProtocolVersion(classOf[ServerInterface]),
       new InetSocketAddress(serverIp, serverPort), ugi,
-      FileFactory.getConfiguration, NetUtils.getDefaultSocketFactory(FileFactory.getConfiguration))
+      FileFactory.getConfiguration, NetUtils.getDefaultSocketFactory(configuration))
   }
 }
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 8d4994e..93a6010 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
@@ -194,7 +194,7 @@ object CarbonEnv {
       .addListener(classOf[DropTableCacheEvent], DropCachePreAggEventListener)
       .addListener(classOf[DropTableCacheEvent], DropCacheBloomEventListener)
       .addListener(classOf[ShowTableCacheEvent], ShowCachePreAggEventListener)
-      .addListener(classOf[ShowTableCacheEvent], ShowCacheBloomEventListener)
+      .addListener(classOf[ShowTableCacheEvent], ShowCacheDataMapEventListener)
       .addListener(classOf[DeleteSegmentByIdPreEvent], DataMapDeleteSegmentPreListener)
       .addListener(classOf[DeleteSegmentByDatePreEvent], DataMapDeleteSegmentPreListener)
       .addListener(classOf[AlterTableDropColumnPreEvent], DataMapDropColumnPreListener)
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 1c3af69..e5cda69 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
@@ -14,7 +14,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.spark.sql.execution.command.cache
 
 import scala.collection.mutable
@@ -30,10 +29,12 @@ import org.apache.spark.sql.types.StringType
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.cache.CacheProvider
 import org.apache.carbondata.core.cache.dictionary.AbstractColumnDictionaryInfo
+import org.apache.carbondata.core.datamap.DataMapStoreManager
 import org.apache.carbondata.core.indexstore.BlockletDataMapIndexWrapper
+import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataMapFactory
+import org.apache.carbondata.core.metadata.schema.datamap.DataMapClassProvider
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.datamap.bloom.BloomCacheKeyValue
 import org.apache.carbondata.events.{OperationContext, OperationListenerBus, ShowTableCacheEvent}
 import org.apache.carbondata.indexserver.IndexServer
 import org.apache.carbondata.spark.util.CarbonScalaUtil
@@ -44,6 +45,10 @@ case class CarbonShowCacheCommand(tableIdentifier: Option[TableIdentifier],
     internalCall: Boolean = false)
   extends MetadataCommand {
 
+  private lazy val cacheResult: Seq[(String, Int, Long, String)] = {
+    executeJobToGetCache(List())
+  }
+
   private val LOGGER = LogServiceFactory.getLogService(classOf[CarbonShowCacheCommand].getName)
 
   override def output: Seq[AttributeReference] = {
@@ -62,216 +67,366 @@ case class CarbonShowCacheCommand(tableIdentifier: Option[TableIdentifier],
     }
   }
 
-  override protected def opName: String = "SHOW CACHE"
+  override def processMetadata(sparkSession: SparkSession): Seq[Row] = {
+    if (tableIdentifier.isEmpty) {
+      /**
+       * Assemble result for database
+       */
+      getAllTablesCache(sparkSession)
+    } else {
+      /**
+       * Assemble result for table
+       */
+      val carbonTable = CarbonEnv.getCarbonTable(tableIdentifier.get)(sparkSession)
+      Checker
+        .validateTableExists(tableIdentifier.get.database, tableIdentifier.get.table, sparkSession)
+      val numberOfIndexFiles = CacheUtil.getAllIndexFiles(carbonTable).size
+      val driverRawResults = getTableCacheFromDriver(sparkSession, carbonTable, numberOfIndexFiles)
+      val indexRawResults = if (CarbonProperties.getInstance().isDistributedPruningEnabled
+      (tableIdentifier.get.database.getOrElse(sparkSession.catalog.currentDatabase),
+        tableIdentifier.get.table)) {
+        getTableCacheFromIndexServer(carbonTable, numberOfIndexFiles)(sparkSession)
+      } else { Seq() }
+      val result = driverRawResults.slice(0, 2) ++
+                   driverRawResults.drop(2).map { row =>
+                     Row(row.get(0), row.getLong(1) + row.getLong(2), row.get(3))
+                   }
+      val serverResults = indexRawResults.slice(0, 2) ++
+                          indexRawResults.drop(2).map { row =>
+                            Row(row.get(0), row.getLong(1) + row.getLong(2), row.get(3))
+                          }
+      Seq(Row("DRIVER CACHE", "", "")) ++ result.map {
+        row =>
+          Row(row.get(0), bytesToDisplaySize(row.getLong(1)), row.get(2))
+      } ++ (serverResults match {
+        case Nil => Seq()
+        case list =>
+          Seq(Row("-----------", "-----------", "-----------"), Row("INDEX CACHE", "", "")) ++
+          list.map {
+          row => Row(row.get(0), bytesToDisplaySize(row.getLong(1)), row.get(2))
+        }
+      })
+    }
+  }
 
   def getAllTablesCache(sparkSession: SparkSession): Seq[Row] = {
     val currentDatabase = sparkSession.sessionState.catalog.getCurrentDatabase
     val cache = CacheProvider.getInstance().getCarbonCache
-    if (cache == null) {
-      Seq(
-        Row("ALL", "ALL", 0L, 0L, 0L),
-        Row(currentDatabase, "ALL", 0L, 0L, 0L))
-    } else {
-      var carbonTables = mutable.ArrayBuffer[CarbonTable]()
-      sparkSession.sessionState.catalog.listTables(currentDatabase).foreach {
-        tableIdent =>
+    val isDistributedPruningEnabled = CarbonProperties.getInstance()
+      .isDistributedPruningEnabled("", "")
+    if (cache == null && !isDistributedPruningEnabled) {
+      return makeEmptyCacheRows(currentDatabase)
+    }
+    var carbonTables = mutable.ArrayBuffer[CarbonTable]()
+    sparkSession.sessionState.catalog.listTables(currentDatabase).foreach {
+      tableIdent =>
+        try {
+          val carbonTable = CarbonEnv.getCarbonTable(tableIdent)(sparkSession)
+          if (!carbonTable.isChildDataMap && !carbonTable.isChildTable) {
+            carbonTables += carbonTable
+          }
+        } catch {
+          case _: NoSuchTableException =>
+            LOGGER.debug("Ignoring non-carbon table " + tableIdent.table)
+        }
+    }
+    val indexServerRows = if (isDistributedPruningEnabled) {
+      carbonTables.flatMap {
+        mainTable =>
           try {
-            val carbonTable = CarbonEnv.getCarbonTable(tableIdent)(sparkSession)
-            if (!carbonTable.isChildDataMap) {
-              carbonTables += carbonTable
-            }
+            makeRows(getTableCacheFromIndexServer(mainTable)(sparkSession), mainTable)
           } catch {
-            case ex: NoSuchTableException =>
-              LOGGER.debug("Ignoring non-carbon table " + tableIdent.table)
+            case ex: UnsupportedOperationException => Seq()
           }
       }
+    } else { Seq() }
 
-      // All tables of current database
-      var (dbDatamapSize, dbDictSize) = (0L, 0L)
-      val tableList = carbonTables.flatMap {
+    val driverRows = if (cache != null) {
+      carbonTables.flatMap {
         carbonTable =>
           try {
-            val tableResult = getTableCache(sparkSession, carbonTable)
-            var (indexSize, datamapSize) = (tableResult(0).getLong(1), 0L)
-            tableResult.drop(2).foreach {
-              row =>
-                indexSize += row.getLong(1)
-                datamapSize += row.getLong(2)
-            }
-            val dictSize = tableResult(1).getLong(1)
-
-            dbDictSize += dictSize
-            dbDatamapSize += datamapSize
-
-            val tableName = if (!carbonTable.isTransactionalTable) {
-              carbonTable.getTableName + " (external table)"
-            }
-            else {
-              carbonTable.getTableName
-            }
-            Seq((currentDatabase, tableName, indexSize, datamapSize, dictSize))
+            makeRows(getTableCacheFromDriver(sparkSession, carbonTable), carbonTable)
           } catch {
-            case ex: UnsupportedOperationException =>
-              Seq.empty
+            case ex: UnsupportedOperationException => Seq()
           }
-      }.collect {
-        case (db, table, indexSize, datamapSize, dictSize) if !((indexSize == 0) &&
-                                                                (datamapSize == 0) &&
-                                                                (dictSize == 0)) =>
-          Row(db, table, indexSize, datamapSize, dictSize)
       }
+    } else { Seq() }
 
+    val (driverdbIndexSize, driverdbDatamapSize, driverdbDictSize) = calculateDBIndexAndDatamapSize(
+      driverRows)
+    val (indexdbIndexSize, indexdbDatamapSize, indexAllDictSize) = calculateDBIndexAndDatamapSize(
+      indexServerRows)
+    val (indexAllIndexSize, indexAllDatamapSize) = getIndexServerCacheSizeForCurrentDB
+
+    val driverDisplayRows = if (cache != null) {
       val tablePaths = carbonTables.map {
         carbonTable =>
           carbonTable.getTablePath
       }
-
-      // Scan whole cache and fill the entries for All-Database-All-Tables
-      // and Current-Database-All-Tables
-      var (allIndexSize, allDatamapSize, allDictSize) = (0L, 0L, 0L)
-      var dbIndexSize = 0L
-      cache.getCacheMap.asScala.foreach {
-        case (key, cacheable) =>
-          cacheable match {
-            case _: BlockletDataMapIndexWrapper =>
-              allIndexSize += cacheable.getMemorySize
-              if (tablePaths.exists { path => key.startsWith(path) }) {
-                dbIndexSize += cacheable.getMemorySize
-              }
-            case _: BloomCacheKeyValue.CacheValue =>
-              allDatamapSize += cacheable.getMemorySize
-            case _: AbstractColumnDictionaryInfo =>
-              allDictSize += cacheable.getMemorySize
-          }
+      val (driverIndexSize, driverDatamapSize, allDictSize) = getAllDriverCacheSize(tablePaths
+        .toList)
+      if (driverRows.nonEmpty) {
+        val rows = (Seq(
+          Row("ALL", "ALL", driverIndexSize, driverDatamapSize, allDictSize),
+          Row(currentDatabase, "ALL", driverdbIndexSize, driverdbDatamapSize, driverdbDictSize)
+        ) ++ driverRows).collect {
+          case row if row.getLong(2) != 0L || row.getLong(3) != 0L || row.getLong(4) != 0L =>
+            Row(row(0), row(1), bytesToDisplaySize(row.getLong(2)),
+              bytesToDisplaySize(row.getLong(3)), bytesToDisplaySize(row.getLong(4)))
+        }
+        Seq(Row("DRIVER CACHE", "", "", "", "")) ++ rows
+      } else {
+        makeEmptyCacheRows(currentDatabase)
       }
+    } else {
+      makeEmptyCacheRows(currentDatabase)
+    }
 
-      Seq(
-        Row("ALL", "ALL", allIndexSize, allDatamapSize, allDictSize),
-        Row(currentDatabase, "ALL", dbIndexSize, dbDatamapSize, dbDictSize)
-      ) ++ tableList
+    //      val (serverIndexSize, serverDataMapSize) = getAllIndexServerCacheSize
+    val indexDisplayRows = if (indexServerRows.nonEmpty) {
+      val rows = (Seq(
+        Row("ALL", "ALL", indexAllIndexSize, indexAllDatamapSize, indexAllDictSize),
+        Row(currentDatabase, "ALL", indexdbIndexSize, indexdbDatamapSize, driverdbDictSize)
+      ) ++ indexServerRows).collect {
+        case row if row.getLong(2) != 0L || row.getLong(3) != 0L || row.getLong(4) != 0L =>
+          Row(row.get(0), row.get(1), bytesToDisplaySize(row.getLong(2)),
+            bytesToDisplaySize(row.getLong(3)), bytesToDisplaySize(row.getLong(4)))
+      }
+      Seq(Row("INDEX SERVER CACHE", "", "", "", "")) ++ rows
+    } else {
+      Seq()
     }
+    driverDisplayRows ++ indexDisplayRows
   }
 
-  def getTableCache(sparkSession: SparkSession, carbonTable: CarbonTable): Seq[Row] = {
+  def getTableCacheFromDriver(sparkSession: SparkSession, carbonTable: CarbonTable,
+      numOfIndexFiles: Int = 0): Seq[Row] = {
     val cache = CacheProvider.getInstance().getCarbonCache
-    val allIndexFiles: List[String] = CacheUtil.getAllIndexFiles(carbonTable)
-    if (cache == null) {
-      var comments = 0 + "/" + allIndexFiles.size + " index files cached"
+    if (cache != null) {
+      val childTableList = getChildTableList(carbonTable)(sparkSession)
+      val (parentMetaCacheInfo, dataMapCacheInfo) = collectDriverMetaCacheInfo(carbonTable
+        .getTableUniqueName) match {
+        case list =>
+          val parentCache = list
+            .filter(_._4.equalsIgnoreCase(BlockletDataMapFactory.DATA_MAP_SCHEMA
+              .getProviderName)) match {
+            case Nil => ("", 0, 0L, "")
+            case head :: _ => head
+          }
+          val dataMapList = list
+            .filter(!_._4.equalsIgnoreCase(BlockletDataMapFactory.DATA_MAP_SCHEMA
+              .getProviderName))
+          (parentCache, dataMapList)
+        case Nil => (("", 0, 0L, ""), Nil)
+      }
+      val parentDictionary = getDictionarySize(carbonTable)(sparkSession)
+      val childMetaCacheInfos = childTableList.flatMap {
+        childTable =>
+          val tableArray = childTable._1.split("-")
+          val dbName = tableArray(0)
+          val tableName = tableArray(1)
+          val childMetaCacheInfo = collectDriverMetaCacheInfo(s"${dbName}_$tableName")
+          childMetaCacheInfo.map {
+            childMeta => Row(childMeta._1, childMeta._3, 0L, childTable._2)
+          }
+      } ++ (dataMapCacheInfo.map {
+        childMeta => Row(childMeta._1, childMeta._3, 0L, childMeta._4)
+      })
+      var comments = parentMetaCacheInfo._2 + s"/$numOfIndexFiles index files cached"
       if (!carbonTable.isTransactionalTable) {
         comments += " (external table)"
       }
-      return Seq(
-        Row("Index", 0L, comments),
+      Seq(
+        Row("Index", parentMetaCacheInfo._3, comments),
+        Row("Dictionary", parentDictionary, "")
+      ) ++ childMetaCacheInfos
+    } else {
+      Seq(
+        Row("Index", 0L, ""),
         Row("Dictionary", 0L, "")
       )
     }
+  }
 
-    val showTableCacheEvent = ShowTableCacheEvent(carbonTable, sparkSession, internalCall)
-    val operationContext = new OperationContext
-    // datamapName -> (datamapProviderName, indexSize, datamapSize)
-    val currentTableSizeMap = scala.collection.mutable.Map[String, (String, String, Long, Long)]()
-    operationContext.setProperty(carbonTable.getTableUniqueName, currentTableSizeMap)
-    OperationListenerBus.getInstance.fireEvent(showTableCacheEvent, operationContext)
+  override protected def opName: String = "SHOW CACHE"
 
-    // Get all Index files for the specified table in cache
-    val (indexFilesLength, size) = if (CarbonProperties.getInstance()
-        .isDistributedPruningEnabled(carbonTable.getDatabaseName, carbonTable.getTableName)) {
-      getTableCache(carbonTable.getTableUniqueName)
-    } else {
-      val memorySizeForEachIndexFile: List[Long] = allIndexFiles.collect {
-        case indexFile if cache.get(indexFile) != null =>
-          cache.get(indexFile).getMemorySize
-      }
-      (memorySizeForEachIndexFile.length, memorySizeForEachIndexFile.sum)
+  private def makeEmptyCacheRows(currentDatabase: String) = {
+    Seq(
+      Row("ALL", "ALL", bytesToDisplaySize(0), bytesToDisplaySize(0), bytesToDisplaySize(0)),
+      Row(currentDatabase, "ALL", bytesToDisplaySize(0), bytesToDisplaySize(0),
+        bytesToDisplaySize(0)))
+  }
+
+  private def calculateDBIndexAndDatamapSize(rows: Seq[Row]): (Long, Long, Long) = {
+    rows.map {
+      row =>
+        (row(2).asInstanceOf[Long], row(3).asInstanceOf[Long], row.get(4).asInstanceOf[Long])
+    }.fold((0L, 0L, 0L)) {
+      case (a, b) =>
+        (a._1 + b._1, a._2 + b._2, a._3 + b._3)
     }
+  }
 
-    // Extract dictionary keys for the table and create cache keys from those
-    val dictKeys = CacheUtil.getAllDictCacheKeys(carbonTable)
-    val sizeOfDictInCache = dictKeys.collect {
-      case dictKey if cache.get(dictKey) != null =>
-        cache.get(dictKey).getMemorySize
-    }.sum
+  private def makeRows(tableResult: Seq[Row], carbonTable: CarbonTable) = {
+    var (indexSize, datamapSize) = (tableResult(0).getLong(1), 0L)
+    tableResult.drop(2).foreach {
+      row =>
+        indexSize += row.getLong(1)
+        datamapSize += row.getLong(2)
+    }
+    val dictSize = tableResult(1).getLong(1)
+    Seq(Row(carbonTable.getDatabaseName, carbonTable.getTableName,
+      indexSize,
+      datamapSize,
+      dictSize))
+  }
 
-    // Assemble result for all the datamaps for the table
-    val otherDatamaps = operationContext.getProperty(carbonTable.getTableUniqueName)
-      .asInstanceOf[mutable.Map[String, (String, Long, Long)]]
-    val otherDatamapsResults: Seq[Row] = otherDatamaps.map {
-      case (name, (provider, indexSize, dmSize)) =>
-        Row(name, indexSize, dmSize, provider)
-    }.toSeq
-    var comments = indexFilesLength + "/" + allIndexFiles.size + " index files cached"
-    if (!carbonTable.isTransactionalTable) {
+  private def getTableCacheFromIndexServer(mainTable: CarbonTable, numberOfIndexFiles: Int = 0)
+    (sparkSession: SparkSession): Seq[Row] = {
+    val childTables = getChildTableList(mainTable)(sparkSession)
+    val cache = if (tableIdentifier.nonEmpty) {
+      executeJobToGetCache(childTables.map(_._1) ++ List(mainTable.getTableUniqueName))
+    } else {
+      cacheResult
+    }
+    val (mainTableFiles, mainTableCache) = getTableCache(cache, mainTable.getTableUniqueName)
+    val childMetaCacheInfos = childTables.flatMap {
+      childTable =>
+        val tableName = childTable._1.replace("-", "_")
+        if (childTable._2
+          .equalsIgnoreCase(DataMapClassProvider.BLOOMFILTER.getShortName)) {
+          Seq(Row(tableName, 0L, getTableCache(cache, tableName)._2, childTable._2))
+        } else {
+          val childCache = getTableCache(cache, tableName)._2
+          Seq(Row(tableName, childCache, 0L, childTable._2))
+        }
+    }
+    var comments = mainTableFiles + s"/$numberOfIndexFiles index files cached"
+    if (!mainTable.isTransactionalTable) {
       comments += " (external table)"
     }
     Seq(
-      Row("Index", size, comments),
-      Row("Dictionary", sizeOfDictInCache, "")
-    ) ++ otherDatamapsResults
+      Row("Index", mainTableCache, comments),
+      Row("Dictionary", getDictionarySize(mainTable)(sparkSession), "")
+    ) ++ childMetaCacheInfos
+
   }
 
-  private lazy val cacheResult: Seq[(String, Int, Long)] = {
-    val tableUniqueName = tableIdentifier match {
-      case Some(identifier) => s"${
-        identifier.database.getOrElse(SparkSession.getActiveSession
-          .get.catalog.currentDatabase)
-      }_${ identifier.table }"
-      case None => ""
-    }
-    val (result, time) = CarbonScalaUtil.logTime {
-      try {
-        IndexServer.getClient.showCache(tableUniqueName).map(_.split(":"))
+  private def executeJobToGetCache(tableUniqueNames: List[String]): Seq[(String, Int, Long,
+    String)] = {
+    try {
+      val (result, time) = CarbonScalaUtil.logTime {
+        IndexServer.getClient.showCache(tableUniqueNames.mkString(",")).map(_.split(":"))
           .groupBy(_.head).map { t =>
           var sum = 0L
           var length = 0
+          var provider = ""
           t._2.foreach {
             arr =>
               sum += arr(2).toLong
               length += arr(1).toInt
+              provider = arr(3)
           }
-          (t._1, length, sum)
+          (t._1, length, sum, provider)
         }
-      } catch {
-        case e: Exception =>
-          throw new RuntimeException("Failed to get Cache Information. ", e)
       }
+      LOGGER.info(s"Time taken to get cache results from Index Server is $time ms")
+      result.toList
+    } catch {
+      case ex: Exception =>
+        LOGGER.error("Error while getting cache details from index server", ex)
+        Seq()
     }
-    LOGGER.info(s"Time taken to get cache results from Index Server is $time ms")
-    result.toList
   }
 
-  private def getTableCache(tableName: String): (Int, Long) = {
-    val (_, indexFileLength, cacheSize) = cacheResult.find(_._1 == tableName).getOrElse(("", 0, 0L))
+  private def getTableCache(cache: Seq[(String, Int, Long, String)], tableName: String) = {
+    val (_, indexFileLength, cacheSize, _) = cache.find(_._1 == tableName)
+      .getOrElse(("", 0, 0L, ""))
     (indexFileLength, cacheSize)
   }
 
-  override def processMetadata(sparkSession: SparkSession): Seq[Row] = {
-    if (tableIdentifier.isEmpty) {
-      /**
-       * Assemble result for database
-       */
-      val result = getAllTablesCache(sparkSession)
-      result.map {
-        row =>
-          Row(row.get(0), row.get(1), bytesToDisplaySize(row.getLong(2)),
-            bytesToDisplaySize(row.getLong(3)), bytesToDisplaySize(row.getLong(4)))
-      }
-    } else {
-      /**
-       * Assemble result for table
-       */
-      val carbonTable = CarbonEnv.getCarbonTable(tableIdentifier.get)(sparkSession)
-      Checker
-        .validateTableExists(tableIdentifier.get.database, tableIdentifier.get.table, sparkSession)
-      val rawResult = getTableCache(sparkSession, carbonTable)
-      val result = rawResult.slice(0, 2) ++
-                   rawResult.drop(2).map {
-                     row =>
-                       Row(row.get(0), row.getLong(1) + row.getLong(2), row.get(3))
-                   }
-      result.map {
-        row =>
-          Row(row.get(0), bytesToDisplaySize(row.getLong(1)), row.get(2))
-      }
+  private def getChildTableList(carbonTable: CarbonTable)
+    (sparkSession: SparkSession): List[(String, String)] = {
+    val showTableCacheEvent = ShowTableCacheEvent(carbonTable, sparkSession, internalCall)
+    val operationContext = new OperationContext
+    // datamapName -> (datamapProviderName, indexSize, datamapSize)
+    operationContext.setProperty(carbonTable.getTableUniqueName, List())
+    OperationListenerBus.getInstance.fireEvent(showTableCacheEvent, operationContext)
+    operationContext.getProperty(carbonTable.getTableUniqueName)
+      .asInstanceOf[List[(String, String)]]
+  }
+
+  private def getDictionarySize(carbonTable: CarbonTable)(sparkSession: SparkSession): Long = {
+    val dictKeys = CacheUtil.getAllDictCacheKeys(carbonTable)
+    val cache = CacheProvider.getInstance().getCarbonCache
+    dictKeys.collect {
+      case dictKey if cache != null && cache.get(dictKey) != null =>
+        cache.get(dictKey).getMemorySize
+    }.sum
+  }
+
+  private def getAllDriverCacheSize(tablePaths: List[String]) = {
+    val cache = CacheProvider.getInstance().getCarbonCache
+    // Scan whole cache and fill the entries for All-Database-All-Tables
+    // and Current-Database-All-Tables
+    var (allIndexSize, allDatamapSize, allDictSize) = (0L, 0L, 0L)
+    var dbIndexSize = 0L
+    cache.getCacheMap.asScala.foreach {
+      case (key, cacheable) =>
+        cacheable match {
+          case _: BlockletDataMapIndexWrapper =>
+            allIndexSize += cacheable.getMemorySize
+            if (tablePaths.exists { path => key.startsWith(path) }) {
+              dbIndexSize += cacheable.getMemorySize
+            }
+          case _: AbstractColumnDictionaryInfo =>
+            allDictSize += cacheable.getMemorySize
+            // consider eveything else as a datamap.
+          case _ =>
+            allDatamapSize += cacheable.getMemorySize
+        }
     }
+    (allIndexSize, allDatamapSize, allDictSize)
   }
+
+  private def collectDriverMetaCacheInfo(tableName: String): List[(String, Int, Long, String)] = {
+    val dataMaps = DataMapStoreManager.getInstance().getAllDataMaps.asScala
+    dataMaps.collect {
+      case (table, tableDataMaps) if table.isEmpty ||
+                                     (tableName.nonEmpty && tableName.equalsIgnoreCase(table)) =>
+        val sizeAndIndexLengths = tableDataMaps.asScala
+          .map { dataMap =>
+            if (dataMap.getDataMapFactory.isInstanceOf[BlockletDataMapFactory]) {
+              s"$table:${ dataMap.getDataMapFactory.getCacheSize }:${
+                dataMap.getDataMapSchema.getProviderName}"
+            } else {
+              s"${ dataMap.getDataMapSchema.getDataMapName }:${
+                dataMap.getDataMapFactory.getCacheSize
+              }:${ dataMap.getDataMapSchema.getProviderName }"
+            }
+          }
+        sizeAndIndexLengths.map {
+          sizeAndLength =>
+            val array = sizeAndLength.split(":")
+            (array(0), array(1).toInt, array(2).toLong, array(3))
+        }
+    }.flatten.toList
+  }
+
+  private def getIndexServerCacheSizeForCurrentDB: (Long, Long) = {
+    var (allIndexSize, allDatamapSize) = (0L, 0L)
+    val bloomFilterIdentifier = DataMapClassProvider.BLOOMFILTER.getShortName
+    cacheResult.foreach {
+      case (_, _, sum, provider) =>
+        provider.toLowerCase match {
+          case `bloomFilterIdentifier` =>
+            allIndexSize += sum
+          case _ =>
+            allDatamapSize += sum
+        }
+    }
+    (allIndexSize, allDatamapSize)
+  }
+
 }
+
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/execution/command/cache/ShowCacheEventListeners.scala
index 3eab81a..55d8313 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/execution/command/cache/ShowCacheEventListeners.scala
@@ -17,24 +17,13 @@
 
 package org.apache.spark.sql.execution.command.cache
 
-import java.util
-import java.util.{HashSet, Set}
-
 import scala.collection.JavaConverters._
-import scala.collection.mutable
-
-import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.CarbonEnv
 
 import org.apache.carbondata.common.logging.LogServiceFactory
-import org.apache.carbondata.core.cache.CacheProvider
 import org.apache.carbondata.core.datamap.DataMapStoreManager
-import org.apache.carbondata.core.indexstore.BlockletDataMapIndexWrapper
 import org.apache.carbondata.core.metadata.schema.datamap.DataMapClassProvider
-import org.apache.carbondata.core.util.path.CarbonTablePath
-import org.apache.carbondata.datamap.bloom.{BloomCacheKeyValue, BloomCoarseGrainDataMapFactory}
+import org.apache.carbondata.core.metadata.schema.table.DataMapSchema
 import org.apache.carbondata.events._
-import org.apache.carbondata.processing.merger.CarbonDataMergerUtil
 
 object ShowCachePreAggEventListener extends OperationEventListener {
 
@@ -50,37 +39,30 @@ object ShowCachePreAggEventListener extends OperationEventListener {
     event match {
       case showTableCacheEvent: ShowTableCacheEvent =>
         val carbonTable = showTableCacheEvent.carbonTable
-        val sparkSession = showTableCacheEvent.sparkSession
         val internalCall = showTableCacheEvent.internalCall
         if ((carbonTable.isChildDataMap || carbonTable.isChildTable) && !internalCall) {
           throw new UnsupportedOperationException("Operation not allowed on child table.")
         }
 
-        val currentTableSizeMap = operationContext.getProperty(carbonTable.getTableUniqueName)
-          .asInstanceOf[mutable.Map[String, (String, Long, Long)]]
+        val childTables = operationContext.getProperty(carbonTable.getTableUniqueName)
+          .asInstanceOf[List[(String, String)]]
 
         if (carbonTable.hasDataMapSchema) {
           val childrenSchemas = carbonTable.getTableInfo.getDataMapSchemaList.asScala
             .filter(_.getRelationIdentifier != null)
-          for (childSchema <- childrenSchemas) {
-            val datamapName = childSchema.getDataMapName
-            val datamapProvider = childSchema.getProviderName
-            val childCarbonTable = CarbonEnv.getCarbonTable(
-              TableIdentifier(childSchema.getRelationIdentifier.getTableName,
-                Some(carbonTable.getDatabaseName)))(sparkSession)
-
-            val resultForChild = CarbonShowCacheCommand(None, true)
-              .getTableCache(sparkSession, childCarbonTable)
-            val datamapSize = resultForChild.head.getLong(1)
-            currentTableSizeMap.put(datamapName, (datamapProvider, datamapSize, 0L))
-          }
+          operationContext.setProperty(carbonTable.getTableUniqueName, childrenSchemas.collect {
+            case childSchema if childSchema.getRelationIdentifier != null =>
+              (s"${ childSchema.getRelationIdentifier.getDatabaseName }-${
+                childSchema.getRelationIdentifier.getTableName
+              }", childSchema.getProviderName)
+          }.toList ++ childTables)
         }
     }
   }
 }
 
 
-object ShowCacheBloomEventListener extends OperationEventListener {
+object ShowCacheDataMapEventListener extends OperationEventListener {
 
   val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
 
@@ -94,32 +76,32 @@ object ShowCacheBloomEventListener extends OperationEventListener {
     event match {
       case showTableCacheEvent: ShowTableCacheEvent =>
         val carbonTable = showTableCacheEvent.carbonTable
-        val cache = CacheProvider.getInstance().getCarbonCache
-        val currentTableSizeMap = operationContext.getProperty(carbonTable.getTableUniqueName)
-          .asInstanceOf[mutable.Map[String, (String, Long, Long)]]
+        val childTables = operationContext.getProperty(carbonTable.getTableUniqueName)
+          .asInstanceOf[List[(String, String)]]
 
         // Extract all datamaps for the table
         val datamaps = DataMapStoreManager.getInstance().getDataMapSchemasOfTable(carbonTable)
-          .asScala
-
-        datamaps.foreach {
-          case datamap if datamap.getProviderName
-            .equalsIgnoreCase(DataMapClassProvider.BLOOMFILTER.getShortName) =>
+          .asScala.toList
 
-            // Get datamap keys
-            val datamapKeys = CacheUtil.getBloomCacheKeys(carbonTable, datamap)
+        val bloomDataMaps = filterDataMaps(datamaps, DataMapClassProvider.BLOOMFILTER.getShortName)
 
-            // calculate the memory size if key exists in cache
-            val datamapSize = datamapKeys.collect {
-              case key if cache.get(key) != null =>
-                cache.get(key).getMemorySize
-            }.sum
-
-            // put the datmap size into main table's map.
-            currentTableSizeMap
-              .put(datamap.getDataMapName, (datamap.getProviderName, 0L, datamapSize))
+        val mvDataMaps = filterDataMaps(datamaps, DataMapClassProvider.MV.getShortName)
+        operationContext
+          .setProperty(carbonTable.getTableUniqueName, childTables ++ bloomDataMaps ++ mvDataMaps)
+    }
+  }
 
-          case _ =>
+  private def filterDataMaps(dataMaps: List[DataMapSchema],
+      filter: String): List[(String, String)] = {
+    dataMaps.collect {
+      case dataMap if dataMap.getProviderName
+        .equalsIgnoreCase(filter) =>
+        if (filter.equalsIgnoreCase(DataMapClassProvider.BLOOMFILTER.getShortName)) {
+          (s"${ dataMap.getRelationIdentifier.getDatabaseName }-${
+            dataMap.getDataMapName}", dataMap.getProviderName)
+        } else {
+          (s"${ dataMap.getRelationIdentifier.getDatabaseName }-${
+            dataMap.getRelationIdentifier.getTableName}", dataMap.getProviderName)
         }
     }
   }
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 7925798..9ee0da3 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
@@ -184,7 +184,7 @@ object DataMapUtil {
       fields.collect {
         case col if fieldRelationMap(col).aggregateFunction.isEmpty &&
                     fieldRelationMap(col).columnTableRelationList.size == 1 &&
-                    parentcol.equals(fieldRelationMap(col).
+                    parentcol.equalsIgnoreCase(fieldRelationMap(col).
                       columnTableRelationList.get.head.parentColumnName) =>
           col.column
       })