You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@carbondata.apache.org by GitBox <gi...@apache.org> on 2020/08/31 16:57:52 UTC

[GitHub] [carbondata] kunal642 opened a new pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

kunal642 opened a new pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908


    ### Why is this PR needed?
    
    
    ### What changes were proposed in this PR?
   
       
    ### Does this PR introduce any user interface change?
    - No
    - Yes. (please explain the change and update document)
   
    ### Is any new testcase added?
    - No
    - Yes
   
       
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#discussion_r487008223



##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       we have users with 200k segments. But the data is partitioned by date and time. so all queries will have partition columns filter. so they query-specific segments every time (without any degrade in query time) 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] marchpure commented on a change in pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
marchpure commented on a change in pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#discussion_r486976125



##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       agree with you. 
   also avoid to use getAbsolutePath function,which has high IO overhead

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       agree with you. 
   also avoid to use getAbsolutePath function,which has high IO overhead

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       agree with you. 
   also avoid to use getAbsolutePath function,which has high IO overhead




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] QiangCai commented on pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
QiangCai commented on pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#issuecomment-690981838






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] akashrn5 commented on a change in pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#discussion_r486982098



##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {

Review comment:
       As i can see from here, we are basically removing the cache if the segment file is updated. Actually same thing we do in `TableSegmentRefresher`, and use that for updating all cache, can move this piece of logic there also to unify for all? we use `TableSegmentRefresher` during query, but here we are doing in case of load, can you try if we can do that? 
   
   If not this PR, may be you can give a try in another, as it will be unified and good.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       As you mentioned first time it will take lot of time, but query time we will get benefit. You said matched partitions and since here for every load we are trying to cache, we wont be able to get any matched partitions right?
   
   It is something similar to our Indexes(MV, SI) etc where we load all segment at once in first time and from next time we do incremental, as @QiangCai mentioned. So here from subsequent loads, only that segment should be considered, which i think its being taken care.
   
   @ajantha-bhat are you trying to say to change to handle in query or load itself? Correct me, if i'm wrong. 

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableLoadingTestCase.scala
##########
@@ -585,6 +587,30 @@ class StandardPartitionTableLoadingTestCase extends QueryTest with BeforeAndAfte
     }
   }
 
+  test("test partition caching") {
+    CarbonProperties.getInstance().addProperty("carbon.read.partition.hive.direct", "false")
+    sql("drop table if exists partition_cache")
+    sql("create table partition_cache(a string) partitioned by(b int) stored as carbondata")
+    sql("insert into partition_cache select 'k',1")
+    sql("select * from partition_cache where b = 1").collect()

Review comment:
       can we add one assert to check the cache, after the load instead of query in one scenario?

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       and i also checked now, this caching happens in flow query too, so you can consider my reply specific to load.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {

Review comment:
       yes, you are right, i didn't observe that module level.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {

Review comment:
       As i can see from here, we are basically removing the cache if the segment file is updated. Actually same thing we do in `TableSegmentRefresher`, and use that for updating all cache, can move this piece of logic there also to unify for all? we use `TableSegmentRefresher` during query, but here we are doing in case of load, can you try if we can do that? 
   
   If not this PR, may be you can give a try in another, as it will be unified and good.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       As you mentioned first time it will take lot of time, but query time we will get benefit. You said matched partitions and since here for every load we are trying to cache, we wont be able to get any matched partitions right?
   
   It is something similar to our Indexes(MV, SI) etc where we load all segment at once in first time and from next time we do incremental, as @QiangCai mentioned. So here from subsequent loads, only that segment should be considered, which i think its being taken care.
   
   @ajantha-bhat are you trying to say to change to handle in query or load itself? Correct me, if i'm wrong. 

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableLoadingTestCase.scala
##########
@@ -585,6 +587,30 @@ class StandardPartitionTableLoadingTestCase extends QueryTest with BeforeAndAfte
     }
   }
 
+  test("test partition caching") {
+    CarbonProperties.getInstance().addProperty("carbon.read.partition.hive.direct", "false")
+    sql("drop table if exists partition_cache")
+    sql("create table partition_cache(a string) partitioned by(b int) stored as carbondata")
+    sql("insert into partition_cache select 'k',1")
+    sql("select * from partition_cache where b = 1").collect()

Review comment:
       can we add one assert to check the cache, after the load instead of query in one scenario?

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       and i also checked now, this caching happens in flow query too, so you can consider my reply specific to load.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {

Review comment:
       yes, you are right, i didn't observe that module level.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {

Review comment:
       As i can see from here, we are basically removing the cache if the segment file is updated. Actually same thing we do in `TableSegmentRefresher`, and use that for updating all cache, can move this piece of logic there also to unify for all? we use `TableSegmentRefresher` during query, but here we are doing in case of load, can you try if we can do that? 
   
   If not this PR, may be you can give a try in another, as it will be unified and good.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       As you mentioned first time it will take lot of time, but query time we will get benefit. You said matched partitions and since here for every load we are trying to cache, we wont be able to get any matched partitions right?
   
   It is something similar to our Indexes(MV, SI) etc where we load all segment at once in first time and from next time we do incremental, as @QiangCai mentioned. So here from subsequent loads, only that segment should be considered, which i think its being taken care.
   
   @ajantha-bhat are you trying to say to change to handle in query or load itself? Correct me, if i'm wrong. 

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableLoadingTestCase.scala
##########
@@ -585,6 +587,30 @@ class StandardPartitionTableLoadingTestCase extends QueryTest with BeforeAndAfte
     }
   }
 
+  test("test partition caching") {
+    CarbonProperties.getInstance().addProperty("carbon.read.partition.hive.direct", "false")
+    sql("drop table if exists partition_cache")
+    sql("create table partition_cache(a string) partitioned by(b int) stored as carbondata")
+    sql("insert into partition_cache select 'k',1")
+    sql("select * from partition_cache where b = 1").collect()

Review comment:
       can we add one assert to check the cache, after the load instead of query in one scenario?

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       and i also checked now, this caching happens in flow query too, so you can consider my reply specific to load.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {

Review comment:
       yes, you are right, i didn't observe that module level.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {

Review comment:
       As i can see from here, we are basically removing the cache if the segment file is updated. Actually same thing we do in `TableSegmentRefresher`, and use that for updating all cache, can move this piece of logic there also to unify for all? we use `TableSegmentRefresher` during query, but here we are doing in case of load, can you try if we can do that? 
   
   If not this PR, may be you can give a try in another, as it will be unified and good.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       As you mentioned first time it will take lot of time, but query time we will get benefit. You said matched partitions and since here for every load we are trying to cache, we wont be able to get any matched partitions right?
   
   It is something similar to our Indexes(MV, SI) etc where we load all segment at once in first time and from next time we do incremental, as @QiangCai mentioned. So here from subsequent loads, only that segment should be considered, which i think its being taken care.
   
   @ajantha-bhat are you trying to say to change to handle in query or load itself? Correct me, if i'm wrong. 

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableLoadingTestCase.scala
##########
@@ -585,6 +587,30 @@ class StandardPartitionTableLoadingTestCase extends QueryTest with BeforeAndAfte
     }
   }
 
+  test("test partition caching") {
+    CarbonProperties.getInstance().addProperty("carbon.read.partition.hive.direct", "false")
+    sql("drop table if exists partition_cache")
+    sql("create table partition_cache(a string) partitioned by(b int) stored as carbondata")
+    sql("insert into partition_cache select 'k',1")
+    sql("select * from partition_cache where b = 1").collect()

Review comment:
       can we add one assert to check the cache, after the load instead of query in one scenario?

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       and i also checked now, this caching happens in flow query too, so you can consider my reply specific to load.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {

Review comment:
       yes, you are right, i didn't observe that module level.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {

Review comment:
       As i can see from here, we are basically removing the cache if the segment file is updated. Actually same thing we do in `TableSegmentRefresher`, and use that for updating all cache, can move this piece of logic there also to unify for all? we use `TableSegmentRefresher` during query, but here we are doing in case of load, can you try if we can do that? 
   
   If not this PR, may be you can give a try in another, as it will be unified and good.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       As you mentioned first time it will take lot of time, but query time we will get benefit. You said matched partitions and since here for every load we are trying to cache, we wont be able to get any matched partitions right?
   
   It is something similar to our Indexes(MV, SI) etc where we load all segment at once in first time and from next time we do incremental, as @QiangCai mentioned. So here from subsequent loads, only that segment should be considered, which i think its being taken care.
   
   @ajantha-bhat are you trying to say to change to handle in query or load itself? Correct me, if i'm wrong. 

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableLoadingTestCase.scala
##########
@@ -585,6 +587,30 @@ class StandardPartitionTableLoadingTestCase extends QueryTest with BeforeAndAfte
     }
   }
 
+  test("test partition caching") {
+    CarbonProperties.getInstance().addProperty("carbon.read.partition.hive.direct", "false")
+    sql("drop table if exists partition_cache")
+    sql("create table partition_cache(a string) partitioned by(b int) stored as carbondata")
+    sql("insert into partition_cache select 'k',1")
+    sql("select * from partition_cache where b = 1").collect()

Review comment:
       can we add one assert to check the cache, after the load instead of query in one scenario?

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       and i also checked now, this caching happens in flow query too, so you can consider my reply specific to load.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {

Review comment:
       yes, you are right, i didn't observe that module level.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {

Review comment:
       As i can see from here, we are basically removing the cache if the segment file is updated. Actually same thing we do in `TableSegmentRefresher`, and use that for updating all cache, can move this piece of logic there also to unify for all? we use `TableSegmentRefresher` during query, but here we are doing in case of load, can you try if we can do that? 
   
   If not this PR, may be you can give a try in another, as it will be unified and good.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       As you mentioned first time it will take lot of time, but query time we will get benefit. You said matched partitions and since here for every load we are trying to cache, we wont be able to get any matched partitions right?
   
   It is something similar to our Indexes(MV, SI) etc where we load all segment at once in first time and from next time we do incremental, as @QiangCai mentioned. So here from subsequent loads, only that segment should be considered, which i think its being taken care.
   
   @ajantha-bhat are you trying to say to change to handle in query or load itself? Correct me, if i'm wrong. 

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableLoadingTestCase.scala
##########
@@ -585,6 +587,30 @@ class StandardPartitionTableLoadingTestCase extends QueryTest with BeforeAndAfte
     }
   }
 
+  test("test partition caching") {
+    CarbonProperties.getInstance().addProperty("carbon.read.partition.hive.direct", "false")
+    sql("drop table if exists partition_cache")
+    sql("create table partition_cache(a string) partitioned by(b int) stored as carbondata")
+    sql("insert into partition_cache select 'k',1")
+    sql("select * from partition_cache where b = 1").collect()

Review comment:
       can we add one assert to check the cache, after the load instead of query in one scenario?

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       and i also checked now, this caching happens in flow query too, so you can consider my reply specific to load.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {

Review comment:
       yes, you are right, i didn't observe that module level.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {

Review comment:
       As i can see from here, we are basically removing the cache if the segment file is updated. Actually same thing we do in `TableSegmentRefresher`, and use that for updating all cache, can move this piece of logic there also to unify for all? we use `TableSegmentRefresher` during query, but here we are doing in case of load, can you try if we can do that? 
   
   If not this PR, may be you can give a try in another, as it will be unified and good.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       As you mentioned first time it will take lot of time, but query time we will get benefit. You said matched partitions and since here for every load we are trying to cache, we wont be able to get any matched partitions right?
   
   It is something similar to our Indexes(MV, SI) etc where we load all segment at once in first time and from next time we do incremental, as @QiangCai mentioned. So here from subsequent loads, only that segment should be considered, which i think its being taken care.
   
   @ajantha-bhat are you trying to say to change to handle in query or load itself? Correct me, if i'm wrong. 

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableLoadingTestCase.scala
##########
@@ -585,6 +587,30 @@ class StandardPartitionTableLoadingTestCase extends QueryTest with BeforeAndAfte
     }
   }
 
+  test("test partition caching") {
+    CarbonProperties.getInstance().addProperty("carbon.read.partition.hive.direct", "false")
+    sql("drop table if exists partition_cache")
+    sql("create table partition_cache(a string) partitioned by(b int) stored as carbondata")
+    sql("insert into partition_cache select 'k',1")
+    sql("select * from partition_cache where b = 1").collect()

Review comment:
       can we add one assert to check the cache, after the load instead of query in one scenario?

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       and i also checked now, this caching happens in flow query too, so you can consider my reply specific to load.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {

Review comment:
       yes, you are right, i didn't observe that module level.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#issuecomment-690026266


   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/4029/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#issuecomment-691502233






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] kunal642 commented on a change in pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
kunal642 commented on a change in pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#discussion_r486996158



##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)

Review comment:
       this is to remove entry of any invalid segment(like deleted segment) from the cacheablePartitionSpecs.
   
   As cacheablePartitionSpecs is a map of (segmentNo -> PartitionSpecs), therefore i am removing all the invalid segments from the final cache.
   
   Does this address your comment? 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#discussion_r486970553



##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       readPartition reads the segment file. 
   so basically first time we read all the segment files of the table.
   
   I have seen the users with 200K segments in one table stored in cloud. query takes more than 5 hours for them if we do this.
   
   So, the current design fails for huge segments. need to change the key as partition spec or cache only matched partitions from hive metastore read
   
   @kumarvishal09 , @QiangCai @marchpure : what is your opinion on this ?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] akashrn5 commented on a change in pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#discussion_r486982098



##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {

Review comment:
       As i can see from here, we are basically removing the cache if the segment file is updated. Actually same thing we do in `TableSegmentRefresher`, and use that for updating all cache, can move this piece of logic there also to unify for all? we use `TableSegmentRefresher` during query, but here we are doing in case of load, can you try if we can do that? 
   
   If not this PR, may be you can give a try in another, as it will be unified and good.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] akashrn5 commented on a change in pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#discussion_r486987021



##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableLoadingTestCase.scala
##########
@@ -585,6 +587,30 @@ class StandardPartitionTableLoadingTestCase extends QueryTest with BeforeAndAfte
     }
   }
 
+  test("test partition caching") {
+    CarbonProperties.getInstance().addProperty("carbon.read.partition.hive.direct", "false")
+    sql("drop table if exists partition_cache")
+    sql("create table partition_cache(a string) partitioned by(b int) stored as carbondata")
+    sql("insert into partition_cache select 'k',1")
+    sql("select * from partition_cache where b = 1").collect()

Review comment:
       can we add one assert to check the cache, after the load instead of query in one scenario?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] kunal642 commented on a change in pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
kunal642 commented on a change in pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#discussion_r487005938



##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       @ajantha-bhat 
   
   `sparkSession.sessionState.catalog.listPartitionsByFilter(identifier, partitionFilters)` is already being used for to filter the partitions based on the filter, but in concurrent queries as apark has to take a lock on metastore before accessing it, therefore the query performance was being degraded.
   
   And no user will have 200k segment, they need to anyways compact otherwise the query performance would be degraded




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] QiangCai commented on a change in pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
QiangCai commented on a change in pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#discussion_r483458848



##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,143 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.metadata.SegmentFileStore
+import org.apache.carbondata.core.statusmanager.SegmentStatusManager
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object  PartitionCacheManager extends Cache[PartitionCacheKey, CacheablePartitionSpec] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): CacheablePartitionSpec = {
+    val cacheablePartitionSpec =
+      CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+    val tableStatusModifiedTime = FileFactory
+      .getCarbonFile(CarbonTablePath.getTableStatusFilePath(identifier.tablePath))
+      .getLastModifiedTime
+    if (cacheablePartitionSpec != null) {
+      if (tableStatusModifiedTime > cacheablePartitionSpec.timestamp) {
+        readPartitions(identifier, tableStatusModifiedTime)
+      } else {
+        cacheablePartitionSpec
+      }
+    } else {
+      readPartitions(identifier, tableStatusModifiedTime)
+    }
+  }
+
+  override def getAll(keys: util.List[PartitionCacheKey]):
+  util.List[CacheablePartitionSpec] = {
+    keys.asScala.map(get).toList.asJava
+  }
+
+  override def getIfPresent(key: PartitionCacheKey): CacheablePartitionSpec = {
+    CACHE.get(key.tableId).asInstanceOf[CacheablePartitionSpec]
+  }
+
+  override def invalidate(partitionCacheKey: PartitionCacheKey): Unit = {
+    CACHE.remove(partitionCacheKey.tableId)
+  }
+
+  private def readPartitions(identifier: PartitionCacheKey, tableStatusModifiedTime: Long) = {

Review comment:
       please check query flow. it also use incremental loading index.
   here maybe can reuse the segment info cache.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] kunal642 commented on a change in pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
kunal642 commented on a change in pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#discussion_r486994915



##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       @ajantha-bhat This is disabled by default, so if the user feels that hive metastore getPartitions is slow, can enable this to improve performance.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#discussion_r486993959



##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       @akashrn5 : yeah, my problem scenario is first time query or upgrade, taking huge time (near to 5 hours for first time query) is still not ok.  In this scenario, I feel reading from hive metastore itself is very fast compared to number of IO from the cloud storage
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] asfgit closed pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] kunal642 commented on a change in pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
kunal642 commented on a change in pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#discussion_r483419069



##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,143 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.metadata.SegmentFileStore
+import org.apache.carbondata.core.statusmanager.SegmentStatusManager
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object  PartitionCacheManager extends Cache[PartitionCacheKey, CacheablePartitionSpec] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): CacheablePartitionSpec = {
+    val cacheablePartitionSpec =
+      CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+    val tableStatusModifiedTime = FileFactory
+      .getCarbonFile(CarbonTablePath.getTableStatusFilePath(identifier.tablePath))
+      .getLastModifiedTime
+    if (cacheablePartitionSpec != null) {
+      if (tableStatusModifiedTime > cacheablePartitionSpec.timestamp) {
+        readPartitions(identifier, tableStatusModifiedTime)
+      } else {
+        cacheablePartitionSpec
+      }
+    } else {
+      readPartitions(identifier, tableStatusModifiedTime)
+    }
+  }
+
+  override def getAll(keys: util.List[PartitionCacheKey]):
+  util.List[CacheablePartitionSpec] = {
+    keys.asScala.map(get).toList.asJava
+  }
+
+  override def getIfPresent(key: PartitionCacheKey): CacheablePartitionSpec = {
+    CACHE.get(key.tableId).asInstanceOf[CacheablePartitionSpec]
+  }
+
+  override def invalidate(partitionCacheKey: PartitionCacheKey): Unit = {
+    CACHE.remove(partitionCacheKey.tableId)
+  }
+
+  private def readPartitions(identifier: PartitionCacheKey, tableStatusModifiedTime: Long) = {

Review comment:
       Do you mean caching of partitions after load?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#discussion_r486965218



##########
File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
##########
@@ -1273,6 +1273,13 @@ private CarbonCommonConstants() {
   @CarbonProperty
   public static final String CARBON_MAX_DRIVER_LRU_CACHE_SIZE = "carbon.max.driver.lru.cache.size";
 
+  /**
+   * max driver lru cache size upto which partition lru cache will be loaded in memory
+   */
+  @CarbonProperty

Review comment:
       please also need to update the document

##########
File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
##########
@@ -1273,6 +1273,13 @@ private CarbonCommonConstants() {
   @CarbonProperty
   public static final String CARBON_MAX_DRIVER_LRU_CACHE_SIZE = "carbon.max.driver.lru.cache.size";
 
+  /**
+   * max driver lru cache size upto which partition lru cache will be loaded in memory
+   */
+  @CarbonProperty

Review comment:
       please also update the document

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       readPartition reads the segment file. 
   so basically first time we read all the segment files of the table.
   
   I have seen the users with 200K segments in one table stored in cloud. query takes more than 5 hours for them if we do this.
   
   So, need to change the key as partition spec and cache only matched partitions
   
   @kumarvishal09 , @QiangCai @marchpure : what is your opinion on this ?

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       readPartition reads the segment file. 
   so basically first time we read all the segment files of the table.
   
   I have seen the users with 200K segments in one table stored in cloud. query takes more than 5 hours for them if we do this.
   
   **So, need to change the key as partition spec and cache only matched partitions**
   
   @kumarvishal09 , @QiangCai @marchpure : what is your opinion on this ?

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       readPartition reads the segment file. 
   so basically first time we read all the segment files of the table.
   
   I have seen the users with 200K segments in one table stored in cloud. query takes more than 5 hours for them if we do this.
   
   **So, the current design fails. need to change the key as partition spec and cache only matched partitions**
   
   @kumarvishal09 , @QiangCai @marchpure : what is your opinion on this ?

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       @akashrn5 : yeah, my problem scenario is first time query or upgrade, taking huge time (near to 5 hours for first time query) is still not ok.  In this scenario, I feel reading from hive metastore itself is very fast compared to number of IO from the cloud storage
   

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       ok. If the user feels hive metastore getPartitions is slow, still he cannot enable this in cloud scenario if he had the huge segments as it will be slower than hive metastore. 
   
   As the feature is disabled by default. Ok for me.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       ok. If the user feels hive metastore getPartitions is slow, still he cannot enable this in cloud scenario if he had the huge number of segments (as IO is very slow) as it will be slower than hive metastore. 
   
   As the feature is disabled by default. Ok for me.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       readPartition reads the segment file. 
   so basically first time we read all the segment files of the table.
   
   I have seen the users with 200K segments in one table stored in cloud. query takes more than 5 hours for them if we do this.
   
   So, the current design fails for huge segments. need to change the key as partition spec or cache only matched partitions from hive metastore read
   
   @kumarvishal09 , @QiangCai @marchpure : what is your opinion on this ?

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       ok. If the user feels hive metastore getPartitions is slow, still he cannot enable this in cloud scenario if he had the huge number of segments (as IO is very slow) as it will be slower than hive metastore. 
   
   Also if the hive metastore supports filter pushdown, they can prune the partitions faster and give us the results using some metadata. (this might already be there) instead of we pruning manually by doing IO of all segment file first.
   
   **As the feature is disabled by default. Ok for me.**

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       we have users with 200k segments. But the data is partitioned by date and time. so all queries will have partition columns filter. so they query-specific segments every time (without any degrade in query time) 

##########
File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
##########
@@ -1273,6 +1273,13 @@ private CarbonCommonConstants() {
   @CarbonProperty
   public static final String CARBON_MAX_DRIVER_LRU_CACHE_SIZE = "carbon.max.driver.lru.cache.size";
 
+  /**
+   * max driver lru cache size upto which partition lru cache will be loaded in memory
+   */
+  @CarbonProperty

Review comment:
       please also need to update the document

##########
File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
##########
@@ -1273,6 +1273,13 @@ private CarbonCommonConstants() {
   @CarbonProperty
   public static final String CARBON_MAX_DRIVER_LRU_CACHE_SIZE = "carbon.max.driver.lru.cache.size";
 
+  /**
+   * max driver lru cache size upto which partition lru cache will be loaded in memory
+   */
+  @CarbonProperty

Review comment:
       please also update the document

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       readPartition reads the segment file. 
   so basically first time we read all the segment files of the table.
   
   I have seen the users with 200K segments in one table stored in cloud. query takes more than 5 hours for them if we do this.
   
   So, need to change the key as partition spec and cache only matched partitions
   
   @kumarvishal09 , @QiangCai @marchpure : what is your opinion on this ?

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       readPartition reads the segment file. 
   so basically first time we read all the segment files of the table.
   
   I have seen the users with 200K segments in one table stored in cloud. query takes more than 5 hours for them if we do this.
   
   **So, need to change the key as partition spec and cache only matched partitions**
   
   @kumarvishal09 , @QiangCai @marchpure : what is your opinion on this ?

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       readPartition reads the segment file. 
   so basically first time we read all the segment files of the table.
   
   I have seen the users with 200K segments in one table stored in cloud. query takes more than 5 hours for them if we do this.
   
   **So, the current design fails. need to change the key as partition spec and cache only matched partitions**
   
   @kumarvishal09 , @QiangCai @marchpure : what is your opinion on this ?

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       @akashrn5 : yeah, my problem scenario is first time query or upgrade, taking huge time (near to 5 hours for first time query) is still not ok.  In this scenario, I feel reading from hive metastore itself is very fast compared to number of IO from the cloud storage
   

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       ok. If the user feels hive metastore getPartitions is slow, still he cannot enable this in cloud scenario if he had the huge segments as it will be slower than hive metastore. 
   
   As the feature is disabled by default. Ok for me.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       ok. If the user feels hive metastore getPartitions is slow, still he cannot enable this in cloud scenario if he had the huge number of segments (as IO is very slow) as it will be slower than hive metastore. 
   
   As the feature is disabled by default. Ok for me.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       readPartition reads the segment file. 
   so basically first time we read all the segment files of the table.
   
   I have seen the users with 200K segments in one table stored in cloud. query takes more than 5 hours for them if we do this.
   
   So, the current design fails for huge segments. need to change the key as partition spec or cache only matched partitions from hive metastore read
   
   @kumarvishal09 , @QiangCai @marchpure : what is your opinion on this ?

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       ok. If the user feels hive metastore getPartitions is slow, still he cannot enable this in cloud scenario if he had the huge number of segments (as IO is very slow) as it will be slower than hive metastore. 
   
   Also if the hive metastore supports filter pushdown, they can prune the partitions faster and give us the results using some metadata. (this might already be there) instead of we pruning manually by doing IO of all segment file first.
   
   **As the feature is disabled by default. Ok for me.**

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       we have users with 200k segments. But the data is partitioned by date and time. so all queries will have partition columns filter. so they query-specific segments every time (without any degrade in query time) 

##########
File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
##########
@@ -1273,6 +1273,13 @@ private CarbonCommonConstants() {
   @CarbonProperty
   public static final String CARBON_MAX_DRIVER_LRU_CACHE_SIZE = "carbon.max.driver.lru.cache.size";
 
+  /**
+   * max driver lru cache size upto which partition lru cache will be loaded in memory
+   */
+  @CarbonProperty

Review comment:
       please also need to update the document

##########
File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
##########
@@ -1273,6 +1273,13 @@ private CarbonCommonConstants() {
   @CarbonProperty
   public static final String CARBON_MAX_DRIVER_LRU_CACHE_SIZE = "carbon.max.driver.lru.cache.size";
 
+  /**
+   * max driver lru cache size upto which partition lru cache will be loaded in memory
+   */
+  @CarbonProperty

Review comment:
       please also update the document

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       readPartition reads the segment file. 
   so basically first time we read all the segment files of the table.
   
   I have seen the users with 200K segments in one table stored in cloud. query takes more than 5 hours for them if we do this.
   
   So, need to change the key as partition spec and cache only matched partitions
   
   @kumarvishal09 , @QiangCai @marchpure : what is your opinion on this ?

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       readPartition reads the segment file. 
   so basically first time we read all the segment files of the table.
   
   I have seen the users with 200K segments in one table stored in cloud. query takes more than 5 hours for them if we do this.
   
   **So, need to change the key as partition spec and cache only matched partitions**
   
   @kumarvishal09 , @QiangCai @marchpure : what is your opinion on this ?

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       readPartition reads the segment file. 
   so basically first time we read all the segment files of the table.
   
   I have seen the users with 200K segments in one table stored in cloud. query takes more than 5 hours for them if we do this.
   
   **So, the current design fails. need to change the key as partition spec and cache only matched partitions**
   
   @kumarvishal09 , @QiangCai @marchpure : what is your opinion on this ?

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       @akashrn5 : yeah, my problem scenario is first time query or upgrade, taking huge time (near to 5 hours for first time query) is still not ok.  In this scenario, I feel reading from hive metastore itself is very fast compared to number of IO from the cloud storage
   

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       ok. If the user feels hive metastore getPartitions is slow, still he cannot enable this in cloud scenario if he had the huge segments as it will be slower than hive metastore. 
   
   As the feature is disabled by default. Ok for me.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       ok. If the user feels hive metastore getPartitions is slow, still he cannot enable this in cloud scenario if he had the huge number of segments (as IO is very slow) as it will be slower than hive metastore. 
   
   As the feature is disabled by default. Ok for me.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       readPartition reads the segment file. 
   so basically first time we read all the segment files of the table.
   
   I have seen the users with 200K segments in one table stored in cloud. query takes more than 5 hours for them if we do this.
   
   So, the current design fails for huge segments. need to change the key as partition spec or cache only matched partitions from hive metastore read
   
   @kumarvishal09 , @QiangCai @marchpure : what is your opinion on this ?

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       ok. If the user feels hive metastore getPartitions is slow, still he cannot enable this in cloud scenario if he had the huge number of segments (as IO is very slow) as it will be slower than hive metastore. 
   
   Also if the hive metastore supports filter pushdown, they can prune the partitions faster and give us the results using some metadata. (this might already be there) instead of we pruning manually by doing IO of all segment file first.
   
   **As the feature is disabled by default. Ok for me.**

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       we have users with 200k segments. But the data is partitioned by date and time. so all queries will have partition columns filter. so they query-specific segments every time (without any degrade in query time) 

##########
File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
##########
@@ -1273,6 +1273,13 @@ private CarbonCommonConstants() {
   @CarbonProperty
   public static final String CARBON_MAX_DRIVER_LRU_CACHE_SIZE = "carbon.max.driver.lru.cache.size";
 
+  /**
+   * max driver lru cache size upto which partition lru cache will be loaded in memory
+   */
+  @CarbonProperty

Review comment:
       please also need to update the document

##########
File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
##########
@@ -1273,6 +1273,13 @@ private CarbonCommonConstants() {
   @CarbonProperty
   public static final String CARBON_MAX_DRIVER_LRU_CACHE_SIZE = "carbon.max.driver.lru.cache.size";
 
+  /**
+   * max driver lru cache size upto which partition lru cache will be loaded in memory
+   */
+  @CarbonProperty

Review comment:
       please also update the document

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       readPartition reads the segment file. 
   so basically first time we read all the segment files of the table.
   
   I have seen the users with 200K segments in one table stored in cloud. query takes more than 5 hours for them if we do this.
   
   So, need to change the key as partition spec and cache only matched partitions
   
   @kumarvishal09 , @QiangCai @marchpure : what is your opinion on this ?

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       readPartition reads the segment file. 
   so basically first time we read all the segment files of the table.
   
   I have seen the users with 200K segments in one table stored in cloud. query takes more than 5 hours for them if we do this.
   
   **So, need to change the key as partition spec and cache only matched partitions**
   
   @kumarvishal09 , @QiangCai @marchpure : what is your opinion on this ?

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       readPartition reads the segment file. 
   so basically first time we read all the segment files of the table.
   
   I have seen the users with 200K segments in one table stored in cloud. query takes more than 5 hours for them if we do this.
   
   **So, the current design fails. need to change the key as partition spec and cache only matched partitions**
   
   @kumarvishal09 , @QiangCai @marchpure : what is your opinion on this ?

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       @akashrn5 : yeah, my problem scenario is first time query or upgrade, taking huge time (near to 5 hours for first time query) is still not ok.  In this scenario, I feel reading from hive metastore itself is very fast compared to number of IO from the cloud storage
   

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       ok. If the user feels hive metastore getPartitions is slow, still he cannot enable this in cloud scenario if he had the huge segments as it will be slower than hive metastore. 
   
   As the feature is disabled by default. Ok for me.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       ok. If the user feels hive metastore getPartitions is slow, still he cannot enable this in cloud scenario if he had the huge number of segments (as IO is very slow) as it will be slower than hive metastore. 
   
   As the feature is disabled by default. Ok for me.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       readPartition reads the segment file. 
   so basically first time we read all the segment files of the table.
   
   I have seen the users with 200K segments in one table stored in cloud. query takes more than 5 hours for them if we do this.
   
   So, the current design fails for huge segments. need to change the key as partition spec or cache only matched partitions from hive metastore read
   
   @kumarvishal09 , @QiangCai @marchpure : what is your opinion on this ?

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       ok. If the user feels hive metastore getPartitions is slow, still he cannot enable this in cloud scenario if he had the huge number of segments (as IO is very slow) as it will be slower than hive metastore. 
   
   Also if the hive metastore supports filter pushdown, they can prune the partitions faster and give us the results using some metadata. (this might already be there) instead of we pruning manually by doing IO of all segment file first.
   
   **As the feature is disabled by default. Ok for me.**

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       we have users with 200k segments. But the data is partitioned by date and time. so all queries will have partition columns filter. so they query-specific segments every time (without any degrade in query time) 

##########
File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
##########
@@ -1273,6 +1273,13 @@ private CarbonCommonConstants() {
   @CarbonProperty
   public static final String CARBON_MAX_DRIVER_LRU_CACHE_SIZE = "carbon.max.driver.lru.cache.size";
 
+  /**
+   * max driver lru cache size upto which partition lru cache will be loaded in memory
+   */
+  @CarbonProperty

Review comment:
       please also need to update the document

##########
File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
##########
@@ -1273,6 +1273,13 @@ private CarbonCommonConstants() {
   @CarbonProperty
   public static final String CARBON_MAX_DRIVER_LRU_CACHE_SIZE = "carbon.max.driver.lru.cache.size";
 
+  /**
+   * max driver lru cache size upto which partition lru cache will be loaded in memory
+   */
+  @CarbonProperty

Review comment:
       please also update the document

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       readPartition reads the segment file. 
   so basically first time we read all the segment files of the table.
   
   I have seen the users with 200K segments in one table stored in cloud. query takes more than 5 hours for them if we do this.
   
   So, need to change the key as partition spec and cache only matched partitions
   
   @kumarvishal09 , @QiangCai @marchpure : what is your opinion on this ?

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       readPartition reads the segment file. 
   so basically first time we read all the segment files of the table.
   
   I have seen the users with 200K segments in one table stored in cloud. query takes more than 5 hours for them if we do this.
   
   **So, need to change the key as partition spec and cache only matched partitions**
   
   @kumarvishal09 , @QiangCai @marchpure : what is your opinion on this ?

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       readPartition reads the segment file. 
   so basically first time we read all the segment files of the table.
   
   I have seen the users with 200K segments in one table stored in cloud. query takes more than 5 hours for them if we do this.
   
   **So, the current design fails. need to change the key as partition spec and cache only matched partitions**
   
   @kumarvishal09 , @QiangCai @marchpure : what is your opinion on this ?

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       @akashrn5 : yeah, my problem scenario is first time query or upgrade, taking huge time (near to 5 hours for first time query) is still not ok.  In this scenario, I feel reading from hive metastore itself is very fast compared to number of IO from the cloud storage
   

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       ok. If the user feels hive metastore getPartitions is slow, still he cannot enable this in cloud scenario if he had the huge segments as it will be slower than hive metastore. 
   
   As the feature is disabled by default. Ok for me.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       ok. If the user feels hive metastore getPartitions is slow, still he cannot enable this in cloud scenario if he had the huge number of segments (as IO is very slow) as it will be slower than hive metastore. 
   
   As the feature is disabled by default. Ok for me.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       readPartition reads the segment file. 
   so basically first time we read all the segment files of the table.
   
   I have seen the users with 200K segments in one table stored in cloud. query takes more than 5 hours for them if we do this.
   
   So, the current design fails for huge segments. need to change the key as partition spec or cache only matched partitions from hive metastore read
   
   @kumarvishal09 , @QiangCai @marchpure : what is your opinion on this ?

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       ok. If the user feels hive metastore getPartitions is slow, still he cannot enable this in cloud scenario if he had the huge number of segments (as IO is very slow) as it will be slower than hive metastore. 
   
   Also if the hive metastore supports filter pushdown, they can prune the partitions faster and give us the results using some metadata. (this might already be there) instead of we pruning manually by doing IO of all segment file first.
   
   **As the feature is disabled by default. Ok for me.**

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       we have users with 200k segments. But the data is partitioned by date and time. so all queries will have partition columns filter. so they query-specific segments every time (without any degrade in query time) 

##########
File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
##########
@@ -1273,6 +1273,13 @@ private CarbonCommonConstants() {
   @CarbonProperty
   public static final String CARBON_MAX_DRIVER_LRU_CACHE_SIZE = "carbon.max.driver.lru.cache.size";
 
+  /**
+   * max driver lru cache size upto which partition lru cache will be loaded in memory
+   */
+  @CarbonProperty

Review comment:
       please also need to update the document

##########
File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
##########
@@ -1273,6 +1273,13 @@ private CarbonCommonConstants() {
   @CarbonProperty
   public static final String CARBON_MAX_DRIVER_LRU_CACHE_SIZE = "carbon.max.driver.lru.cache.size";
 
+  /**
+   * max driver lru cache size upto which partition lru cache will be loaded in memory
+   */
+  @CarbonProperty

Review comment:
       please also update the document

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       readPartition reads the segment file. 
   so basically first time we read all the segment files of the table.
   
   I have seen the users with 200K segments in one table stored in cloud. query takes more than 5 hours for them if we do this.
   
   So, need to change the key as partition spec and cache only matched partitions
   
   @kumarvishal09 , @QiangCai @marchpure : what is your opinion on this ?

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       readPartition reads the segment file. 
   so basically first time we read all the segment files of the table.
   
   I have seen the users with 200K segments in one table stored in cloud. query takes more than 5 hours for them if we do this.
   
   **So, need to change the key as partition spec and cache only matched partitions**
   
   @kumarvishal09 , @QiangCai @marchpure : what is your opinion on this ?

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       readPartition reads the segment file. 
   so basically first time we read all the segment files of the table.
   
   I have seen the users with 200K segments in one table stored in cloud. query takes more than 5 hours for them if we do this.
   
   **So, the current design fails. need to change the key as partition spec and cache only matched partitions**
   
   @kumarvishal09 , @QiangCai @marchpure : what is your opinion on this ?

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       @akashrn5 : yeah, my problem scenario is first time query or upgrade, taking huge time (near to 5 hours for first time query) is still not ok.  In this scenario, I feel reading from hive metastore itself is very fast compared to number of IO from the cloud storage
   

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       ok. If the user feels hive metastore getPartitions is slow, still he cannot enable this in cloud scenario if he had the huge segments as it will be slower than hive metastore. 
   
   As the feature is disabled by default. Ok for me.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       ok. If the user feels hive metastore getPartitions is slow, still he cannot enable this in cloud scenario if he had the huge number of segments (as IO is very slow) as it will be slower than hive metastore. 
   
   As the feature is disabled by default. Ok for me.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       readPartition reads the segment file. 
   so basically first time we read all the segment files of the table.
   
   I have seen the users with 200K segments in one table stored in cloud. query takes more than 5 hours for them if we do this.
   
   So, the current design fails for huge segments. need to change the key as partition spec or cache only matched partitions from hive metastore read
   
   @kumarvishal09 , @QiangCai @marchpure : what is your opinion on this ?

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       ok. If the user feels hive metastore getPartitions is slow, still he cannot enable this in cloud scenario if he had the huge number of segments (as IO is very slow) as it will be slower than hive metastore. 
   
   Also if the hive metastore supports filter pushdown, they can prune the partitions faster and give us the results using some metadata. (this might already be there) instead of we pruning manually by doing IO of all segment file first.
   
   **As the feature is disabled by default. Ok for me.**

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       we have users with 200k segments. But the data is partitioned by date and time. so all queries will have partition columns filter. so they query-specific segments every time (without any degrade in query time) 

##########
File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
##########
@@ -1273,6 +1273,13 @@ private CarbonCommonConstants() {
   @CarbonProperty
   public static final String CARBON_MAX_DRIVER_LRU_CACHE_SIZE = "carbon.max.driver.lru.cache.size";
 
+  /**
+   * max driver lru cache size upto which partition lru cache will be loaded in memory
+   */
+  @CarbonProperty

Review comment:
       please also need to update the document

##########
File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
##########
@@ -1273,6 +1273,13 @@ private CarbonCommonConstants() {
   @CarbonProperty
   public static final String CARBON_MAX_DRIVER_LRU_CACHE_SIZE = "carbon.max.driver.lru.cache.size";
 
+  /**
+   * max driver lru cache size upto which partition lru cache will be loaded in memory
+   */
+  @CarbonProperty

Review comment:
       please also update the document

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       readPartition reads the segment file. 
   so basically first time we read all the segment files of the table.
   
   I have seen the users with 200K segments in one table stored in cloud. query takes more than 5 hours for them if we do this.
   
   So, need to change the key as partition spec and cache only matched partitions
   
   @kumarvishal09 , @QiangCai @marchpure : what is your opinion on this ?

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       readPartition reads the segment file. 
   so basically first time we read all the segment files of the table.
   
   I have seen the users with 200K segments in one table stored in cloud. query takes more than 5 hours for them if we do this.
   
   **So, need to change the key as partition spec and cache only matched partitions**
   
   @kumarvishal09 , @QiangCai @marchpure : what is your opinion on this ?

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       readPartition reads the segment file. 
   so basically first time we read all the segment files of the table.
   
   I have seen the users with 200K segments in one table stored in cloud. query takes more than 5 hours for them if we do this.
   
   **So, the current design fails. need to change the key as partition spec and cache only matched partitions**
   
   @kumarvishal09 , @QiangCai @marchpure : what is your opinion on this ?

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       @akashrn5 : yeah, my problem scenario is first time query or upgrade, taking huge time (near to 5 hours for first time query) is still not ok.  In this scenario, I feel reading from hive metastore itself is very fast compared to number of IO from the cloud storage
   

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       ok. If the user feels hive metastore getPartitions is slow, still he cannot enable this in cloud scenario if he had the huge segments as it will be slower than hive metastore. 
   
   As the feature is disabled by default. Ok for me.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       ok. If the user feels hive metastore getPartitions is slow, still he cannot enable this in cloud scenario if he had the huge number of segments (as IO is very slow) as it will be slower than hive metastore. 
   
   As the feature is disabled by default. Ok for me.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       readPartition reads the segment file. 
   so basically first time we read all the segment files of the table.
   
   I have seen the users with 200K segments in one table stored in cloud. query takes more than 5 hours for them if we do this.
   
   So, the current design fails for huge segments. need to change the key as partition spec or cache only matched partitions from hive metastore read
   
   @kumarvishal09 , @QiangCai @marchpure : what is your opinion on this ?

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       ok. If the user feels hive metastore getPartitions is slow, still he cannot enable this in cloud scenario if he had the huge number of segments (as IO is very slow) as it will be slower than hive metastore. 
   
   Also if the hive metastore supports filter pushdown, they can prune the partitions faster and give us the results using some metadata. (this might already be there) instead of we pruning manually by doing IO of all segment file first.
   
   **As the feature is disabled by default. Ok for me.**

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       we have users with 200k segments. But the data is partitioned by date and time. so all queries will have partition columns filter. so they query-specific segments every time (without any degrade in query time) 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#issuecomment-683966337


   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/2193/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#discussion_r486965218



##########
File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
##########
@@ -1273,6 +1273,13 @@ private CarbonCommonConstants() {
   @CarbonProperty
   public static final String CARBON_MAX_DRIVER_LRU_CACHE_SIZE = "carbon.max.driver.lru.cache.size";
 
+  /**
+   * max driver lru cache size upto which partition lru cache will be loaded in memory
+   */
+  @CarbonProperty

Review comment:
       please also need to update the document

##########
File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
##########
@@ -1273,6 +1273,13 @@ private CarbonCommonConstants() {
   @CarbonProperty
   public static final String CARBON_MAX_DRIVER_LRU_CACHE_SIZE = "carbon.max.driver.lru.cache.size";
 
+  /**
+   * max driver lru cache size upto which partition lru cache will be loaded in memory
+   */
+  @CarbonProperty

Review comment:
       please also update the document

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       readPartition reads the segment file. 
   so basically first time we read all the segment files of the table.
   
   I have seen the users with 200K segments in one table stored in cloud. query takes more than 5 hours for them if we do this.
   
   So, need to change the key as partition spec and cache only matched partitions
   
   @kumarvishal09 , @QiangCai @marchpure : what is your opinion on this ?

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       readPartition reads the segment file. 
   so basically first time we read all the segment files of the table.
   
   I have seen the users with 200K segments in one table stored in cloud. query takes more than 5 hours for them if we do this.
   
   **So, need to change the key as partition spec and cache only matched partitions**
   
   @kumarvishal09 , @QiangCai @marchpure : what is your opinion on this ?

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       readPartition reads the segment file. 
   so basically first time we read all the segment files of the table.
   
   I have seen the users with 200K segments in one table stored in cloud. query takes more than 5 hours for them if we do this.
   
   **So, the current design fails. need to change the key as partition spec and cache only matched partitions**
   
   @kumarvishal09 , @QiangCai @marchpure : what is your opinion on this ?

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       @akashrn5 : yeah, my problem scenario is first time query or upgrade, taking huge time (near to 5 hours for first time query) is still not ok.  In this scenario, I feel reading from hive metastore itself is very fast compared to number of IO from the cloud storage
   

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       ok. If the user feels hive metastore getPartitions is slow, still he cannot enable this in cloud scenario if he had the huge segments as it will be slower than hive metastore. 
   
   As the feature is disabled by default. Ok for me.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       ok. If the user feels hive metastore getPartitions is slow, still he cannot enable this in cloud scenario if he had the huge number of segments (as IO is very slow) as it will be slower than hive metastore. 
   
   As the feature is disabled by default. Ok for me.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       readPartition reads the segment file. 
   so basically first time we read all the segment files of the table.
   
   I have seen the users with 200K segments in one table stored in cloud. query takes more than 5 hours for them if we do this.
   
   So, the current design fails for huge segments. need to change the key as partition spec or cache only matched partitions from hive metastore read
   
   @kumarvishal09 , @QiangCai @marchpure : what is your opinion on this ?

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       ok. If the user feels hive metastore getPartitions is slow, still he cannot enable this in cloud scenario if he had the huge number of segments (as IO is very slow) as it will be slower than hive metastore. 
   
   Also if the hive metastore supports filter pushdown, they can prune the partitions faster and give us the results using some metadata. (this might already be there) instead of we pruning manually by doing IO of all segment file first.
   
   **As the feature is disabled by default. Ok for me.**

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       we have users with 200k segments. But the data is partitioned by date and time. so all queries will have partition columns filter. so they query-specific segments every time (without any degrade in query time) 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] QiangCai commented on a change in pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
QiangCai commented on a change in pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#discussion_r486038703



##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {
+      // remove the existing cache as new cache values may be added.
+      // CarbonLRUCache does not allow cache updation until time is expired.
+      // TODO: Need to fix!!
+      CACHE.remove(identifier.tableId)
+      CACHE.put(identifier.tableId,
+        cacheObject,
+        cacheObject.getMemorySize,
+        identifier.expirationTime)
+    }
+    finalCache.values.flatMap(_._1).toList.asJava
+  }
+
+  override def getAll(keys: util.List[PartitionCacheKey]):
+  util.List[util.List[CatalogTablePartition]] = {
+    keys.asScala.toList.map(get).asJava
+  }
+
+  override def getIfPresent(key: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    CACHE.get(key.tableId).asInstanceOf[CacheablePartitionSpec].partitionSpecs.values.flatMap(_._1)
+      .toList.asJava
+  }
+
+  override def invalidate(partitionCacheKey: PartitionCacheKey): Unit = {
+    CACHE.remove(partitionCacheKey.tableId)
+  }
+
+  private def readPartition(identifier: PartitionCacheKey, segmentFilePath: String) = {
+    val segmentFile = SegmentFileStore.readSegmentFile(segmentFilePath)
+    segmentFile.getLocationMap.values().asScala
+      .flatMap(_.getPartitions.asScala).toSet.map { uniquePartition: String =>
+      val partitionSplit = uniquePartition.split("=")
+      val storageFormat = CatalogStorageFormat(
+        Some(new URI(identifier.tablePath + "/" + uniquePartition)),
+        None, None, None, compressed = false, Map())
+      CatalogTablePartition(Map(partitionSplit(0) -> partitionSplit(1)), storageFormat)
+    }.toSeq
+  }
+
+  override def put(key: PartitionCacheKey,
+      value: java.util.List[CatalogTablePartition]): Unit = {
+
+  }
+
+  override def clearAccessCount(keys: util.List[PartitionCacheKey]): Unit = {
+
+  }
+}
+
+case class PartitionCacheKey(tableId: String, tablePath: String, expirationTime: Long)
+
+case class CacheablePartitionSpec(partitionSpecs: Map[String, (Seq[CatalogTablePartition], Long)])

Review comment:
       please add a comment about partitionSpecs




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#issuecomment-691132772


   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/4049/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#discussion_r486970553



##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       readPartition reads the segment file. 
   so basically first time we read all the segment files of the table.
   
   I have seen the users with 200K segments in one table stored in cloud. query takes more than 5 hours for them if we do this.
   
   **So, the current design fails. need to change the key as partition spec and cache only matched partitions**
   
   @kumarvishal09 , @QiangCai @marchpure : what is your opinion on this ?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#discussion_r486965218



##########
File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
##########
@@ -1273,6 +1273,13 @@ private CarbonCommonConstants() {
   @CarbonProperty
   public static final String CARBON_MAX_DRIVER_LRU_CACHE_SIZE = "carbon.max.driver.lru.cache.size";
 
+  /**
+   * max driver lru cache size upto which partition lru cache will be loaded in memory
+   */
+  @CarbonProperty

Review comment:
       please also need to update the document




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] kunal642 commented on a change in pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
kunal642 commented on a change in pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#discussion_r486995129



##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {

Review comment:
       TableSegmentRefresher is in core module, cannot add there




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#issuecomment-688848438


   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/4002/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] QiangCai commented on a change in pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
QiangCai commented on a change in pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#discussion_r486039868



##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)

Review comment:
       move to the outside of the map function

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]

Review comment:
       move to the outside of the map function

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)

Review comment:
       1. cacheablePartitionSpecs doesn't contain an invalid segment.
   2. use partitionSpecs of CACHE - cacheablePartitionSpecs
   

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {

Review comment:
       if finalCache empty, should remove cache if exists

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {

Review comment:
       this always not empty. if the cache exists an invalid segment, it should update the cache.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {

Review comment:
       I mean it should has "else" branch to remove cache

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val existingCache = CACHE.get(identifier.tableId)
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      if (existingCache != null) {
+        val segmentCache = existingCache.asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --

Review comment:
       substraction of different type

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val existingCache = CACHE.get(identifier.tableId)
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      if (existingCache != null) {
+        val segmentCache = existingCache.asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {
+      // remove the existing cache as new cache values may be added.
+      // CarbonLRUCache does not allow cache updation until time is expired.
+      // TODO: Need to fix!!
+      CACHE.remove(identifier.tableId)

Review comment:
       for concurrent query,  line 53 will return null and it will lead to read all partitions.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)

Review comment:
       move to the outside of the map function

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]

Review comment:
       move to the outside of the map function

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)

Review comment:
       1. cacheablePartitionSpecs doesn't contain an invalid segment.
   2. use partitionSpecs of CACHE - cacheablePartitionSpecs
   

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {

Review comment:
       if finalCache empty, should remove cache if exists

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {

Review comment:
       this always not empty. if the cache exists an invalid segment, it should update the cache.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {

Review comment:
       I mean it should has "else" branch to remove cache

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val existingCache = CACHE.get(identifier.tableId)
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      if (existingCache != null) {
+        val segmentCache = existingCache.asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --

Review comment:
       substraction of different type

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val existingCache = CACHE.get(identifier.tableId)
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      if (existingCache != null) {
+        val segmentCache = existingCache.asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {
+      // remove the existing cache as new cache values may be added.
+      // CarbonLRUCache does not allow cache updation until time is expired.
+      // TODO: Need to fix!!
+      CACHE.remove(identifier.tableId)

Review comment:
       for concurrent query,  line 53 will return null and it will lead to read all partitions.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] kunal642 commented on a change in pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
kunal642 commented on a change in pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#discussion_r487003270



##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       @ajantha-bhat 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] kunal642 commented on a change in pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
kunal642 commented on a change in pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#discussion_r484941895



##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,143 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.metadata.SegmentFileStore
+import org.apache.carbondata.core.statusmanager.SegmentStatusManager
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object  PartitionCacheManager extends Cache[PartitionCacheKey, CacheablePartitionSpec] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): CacheablePartitionSpec = {
+    val cacheablePartitionSpec =
+      CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+    val tableStatusModifiedTime = FileFactory
+      .getCarbonFile(CarbonTablePath.getTableStatusFilePath(identifier.tablePath))
+      .getLastModifiedTime
+    if (cacheablePartitionSpec != null) {
+      if (tableStatusModifiedTime > cacheablePartitionSpec.timestamp) {
+        readPartitions(identifier, tableStatusModifiedTime)
+      } else {
+        cacheablePartitionSpec
+      }
+    } else {
+      readPartitions(identifier, tableStatusModifiedTime)
+    }
+  }
+
+  override def getAll(keys: util.List[PartitionCacheKey]):
+  util.List[CacheablePartitionSpec] = {
+    keys.asScala.map(get).toList.asJava
+  }
+
+  override def getIfPresent(key: PartitionCacheKey): CacheablePartitionSpec = {
+    CACHE.get(key.tableId).asInstanceOf[CacheablePartitionSpec]
+  }
+
+  override def invalidate(partitionCacheKey: PartitionCacheKey): Unit = {
+    CACHE.remove(partitionCacheKey.tableId)
+  }
+
+  private def readPartitions(identifier: PartitionCacheKey, tableStatusModifiedTime: Long) = {

Review comment:
       @QiangCai each load or query would be loading the already success segments, so it will now solve the problem you mentioned
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#discussion_r486998066



##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       ok. If the user feels hive metastore getPartitions is slow, still he cannot enable this in cloud scenario if he had the huge number of segments (as IO is very slow) as it will be slower than hive metastore. 
   
   Also if the hive metastore supports filter pushdown, they can prune the partitions faster and give us the results using some metadata. (this might already be there) instead of we pruning manually by doing IO of all segment file first.
   
   **As the feature is disabled by default. Ok for me.**




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] kunal642 commented on a change in pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
kunal642 commented on a change in pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#discussion_r486993964



##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {

Review comment:
       Can be empty during first load. in that case empty partitions would be stored in the cache

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {

Review comment:
       this check is to avoid caching empty cache map

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       @ajantha-bhat This is disabled by default, so if the user feels that hive metastore getPartitions is slow, can enable this to improve performance.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {

Review comment:
       TableSegmentRefresher is in core module, cannot add there

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)

Review comment:
       this is to remove entry of any invalid segment(like deleted segment) from the cacheablePartitionSpecs.
   
   As cacheablePartitionSpecs is a map of (segmentNo -> PartitionSpecs), therefore i am removing all the invalid segments from the final cache.
   
   Does this address your comment? 

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)

Review comment:
       done

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]

Review comment:
       done

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       @ajantha-bhat 

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       @ajantha-bhat 
   
   `sparkSession.sessionState.catalog.listPartitionsByFilter(identifier, partitionFilters)` is already being used for to filter the partitions based on the filter, but in concurrent queries as apark has to take a lock on metastore before accessing it, therefore the query performance was being degraded.
   
   And no user will have 200k segment, they need to anyways compact otherwise the query performance would be degraded

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       Yeah, because they are using presto, where compaction is not yet supported.
   
   If they are using hivemetastore surely they will face degrade in concurrent queries

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableLoadingTestCase.scala
##########
@@ -585,6 +587,30 @@ class StandardPartitionTableLoadingTestCase extends QueryTest with BeforeAndAfte
     }
   }
 
+  test("test partition caching") {
+    CarbonProperties.getInstance().addProperty("carbon.read.partition.hive.direct", "false")
+    sql("drop table if exists partition_cache")
+    sql("create table partition_cache(a string) partitioned by(b int) stored as carbondata")
+    sql("insert into partition_cache select 'k',1")
+    sql("select * from partition_cache where b = 1").collect()

Review comment:
       done

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {

Review comment:
       Can be empty during first load. in that case empty partitions would be stored in the cache

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {

Review comment:
       this check is to avoid caching empty cache map

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       @ajantha-bhat This is disabled by default, so if the user feels that hive metastore getPartitions is slow, can enable this to improve performance.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {

Review comment:
       TableSegmentRefresher is in core module, cannot add there

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)

Review comment:
       this is to remove entry of any invalid segment(like deleted segment) from the cacheablePartitionSpecs.
   
   As cacheablePartitionSpecs is a map of (segmentNo -> PartitionSpecs), therefore i am removing all the invalid segments from the final cache.
   
   Does this address your comment? 

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)

Review comment:
       done

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]

Review comment:
       done

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       @ajantha-bhat 

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       @ajantha-bhat 
   
   `sparkSession.sessionState.catalog.listPartitionsByFilter(identifier, partitionFilters)` is already being used for to filter the partitions based on the filter, but in concurrent queries as apark has to take a lock on metastore before accessing it, therefore the query performance was being degraded.
   
   And no user will have 200k segment, they need to anyways compact otherwise the query performance would be degraded

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       Yeah, because they are using presto, where compaction is not yet supported.
   
   If they are using hivemetastore surely they will face degrade in concurrent queries

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableLoadingTestCase.scala
##########
@@ -585,6 +587,30 @@ class StandardPartitionTableLoadingTestCase extends QueryTest with BeforeAndAfte
     }
   }
 
+  test("test partition caching") {
+    CarbonProperties.getInstance().addProperty("carbon.read.partition.hive.direct", "false")
+    sql("drop table if exists partition_cache")
+    sql("create table partition_cache(a string) partitioned by(b int) stored as carbondata")
+    sql("insert into partition_cache select 'k',1")
+    sql("select * from partition_cache where b = 1").collect()

Review comment:
       done

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {

Review comment:
       Can be empty during first load. in that case empty partitions would be stored in the cache

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {

Review comment:
       this check is to avoid caching empty cache map

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       @ajantha-bhat This is disabled by default, so if the user feels that hive metastore getPartitions is slow, can enable this to improve performance.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {

Review comment:
       TableSegmentRefresher is in core module, cannot add there

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)

Review comment:
       this is to remove entry of any invalid segment(like deleted segment) from the cacheablePartitionSpecs.
   
   As cacheablePartitionSpecs is a map of (segmentNo -> PartitionSpecs), therefore i am removing all the invalid segments from the final cache.
   
   Does this address your comment? 

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)

Review comment:
       done

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]

Review comment:
       done

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       @ajantha-bhat 

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       @ajantha-bhat 
   
   `sparkSession.sessionState.catalog.listPartitionsByFilter(identifier, partitionFilters)` is already being used for to filter the partitions based on the filter, but in concurrent queries as apark has to take a lock on metastore before accessing it, therefore the query performance was being degraded.
   
   And no user will have 200k segment, they need to anyways compact otherwise the query performance would be degraded

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       Yeah, because they are using presto, where compaction is not yet supported.
   
   If they are using hivemetastore surely they will face degrade in concurrent queries

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableLoadingTestCase.scala
##########
@@ -585,6 +587,30 @@ class StandardPartitionTableLoadingTestCase extends QueryTest with BeforeAndAfte
     }
   }
 
+  test("test partition caching") {
+    CarbonProperties.getInstance().addProperty("carbon.read.partition.hive.direct", "false")
+    sql("drop table if exists partition_cache")
+    sql("create table partition_cache(a string) partitioned by(b int) stored as carbondata")
+    sql("insert into partition_cache select 'k',1")
+    sql("select * from partition_cache where b = 1").collect()

Review comment:
       done

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {

Review comment:
       Can be empty during first load. in that case empty partitions would be stored in the cache

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {

Review comment:
       this check is to avoid caching empty cache map

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       @ajantha-bhat This is disabled by default, so if the user feels that hive metastore getPartitions is slow, can enable this to improve performance.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {

Review comment:
       TableSegmentRefresher is in core module, cannot add there

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)

Review comment:
       this is to remove entry of any invalid segment(like deleted segment) from the cacheablePartitionSpecs.
   
   As cacheablePartitionSpecs is a map of (segmentNo -> PartitionSpecs), therefore i am removing all the invalid segments from the final cache.
   
   Does this address your comment? 

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)

Review comment:
       done

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]

Review comment:
       done

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       @ajantha-bhat 

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       @ajantha-bhat 
   
   `sparkSession.sessionState.catalog.listPartitionsByFilter(identifier, partitionFilters)` is already being used for to filter the partitions based on the filter, but in concurrent queries as apark has to take a lock on metastore before accessing it, therefore the query performance was being degraded.
   
   And no user will have 200k segment, they need to anyways compact otherwise the query performance would be degraded

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       Yeah, because they are using presto, where compaction is not yet supported.
   
   If they are using hivemetastore surely they will face degrade in concurrent queries

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableLoadingTestCase.scala
##########
@@ -585,6 +587,30 @@ class StandardPartitionTableLoadingTestCase extends QueryTest with BeforeAndAfte
     }
   }
 
+  test("test partition caching") {
+    CarbonProperties.getInstance().addProperty("carbon.read.partition.hive.direct", "false")
+    sql("drop table if exists partition_cache")
+    sql("create table partition_cache(a string) partitioned by(b int) stored as carbondata")
+    sql("insert into partition_cache select 'k',1")
+    sql("select * from partition_cache where b = 1").collect()

Review comment:
       done

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {

Review comment:
       Can be empty during first load. in that case empty partitions would be stored in the cache

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {

Review comment:
       this check is to avoid caching empty cache map

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       @ajantha-bhat This is disabled by default, so if the user feels that hive metastore getPartitions is slow, can enable this to improve performance.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {

Review comment:
       TableSegmentRefresher is in core module, cannot add there

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)

Review comment:
       this is to remove entry of any invalid segment(like deleted segment) from the cacheablePartitionSpecs.
   
   As cacheablePartitionSpecs is a map of (segmentNo -> PartitionSpecs), therefore i am removing all the invalid segments from the final cache.
   
   Does this address your comment? 

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)

Review comment:
       done

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]

Review comment:
       done

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       @ajantha-bhat 

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       @ajantha-bhat 
   
   `sparkSession.sessionState.catalog.listPartitionsByFilter(identifier, partitionFilters)` is already being used for to filter the partitions based on the filter, but in concurrent queries as apark has to take a lock on metastore before accessing it, therefore the query performance was being degraded.
   
   And no user will have 200k segment, they need to anyways compact otherwise the query performance would be degraded

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       Yeah, because they are using presto, where compaction is not yet supported.
   
   If they are using hivemetastore surely they will face degrade in concurrent queries

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableLoadingTestCase.scala
##########
@@ -585,6 +587,30 @@ class StandardPartitionTableLoadingTestCase extends QueryTest with BeforeAndAfte
     }
   }
 
+  test("test partition caching") {
+    CarbonProperties.getInstance().addProperty("carbon.read.partition.hive.direct", "false")
+    sql("drop table if exists partition_cache")
+    sql("create table partition_cache(a string) partitioned by(b int) stored as carbondata")
+    sql("insert into partition_cache select 'k',1")
+    sql("select * from partition_cache where b = 1").collect()

Review comment:
       done




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#issuecomment-691502233






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#issuecomment-683969864


   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3933/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] kunal642 commented on a change in pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
kunal642 commented on a change in pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#discussion_r483445661



##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,143 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.metadata.SegmentFileStore
+import org.apache.carbondata.core.statusmanager.SegmentStatusManager
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object  PartitionCacheManager extends Cache[PartitionCacheKey, CacheablePartitionSpec] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): CacheablePartitionSpec = {
+    val cacheablePartitionSpec =
+      CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+    val tableStatusModifiedTime = FileFactory
+      .getCarbonFile(CarbonTablePath.getTableStatusFilePath(identifier.tablePath))
+      .getLastModifiedTime
+    if (cacheablePartitionSpec != null) {
+      if (tableStatusModifiedTime > cacheablePartitionSpec.timestamp) {
+        readPartitions(identifier, tableStatusModifiedTime)
+      } else {
+        cacheablePartitionSpec
+      }
+    } else {
+      readPartitions(identifier, tableStatusModifiedTime)
+    }
+  }
+
+  override def getAll(keys: util.List[PartitionCacheKey]):
+  util.List[CacheablePartitionSpec] = {
+    keys.asScala.map(get).toList.asJava
+  }
+
+  override def getIfPresent(key: PartitionCacheKey): CacheablePartitionSpec = {
+    CACHE.get(key.tableId).asInstanceOf[CacheablePartitionSpec]
+  }
+
+  override def invalidate(partitionCacheKey: PartitionCacheKey): Unit = {
+    CACHE.remove(partitionCacheKey.tableId)
+  }
+
+  private def readPartitions(identifier: PartitionCacheKey, tableStatusModifiedTime: Long) = {

Review comment:
       ok got it
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] QiangCai commented on pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
QiangCai commented on pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#issuecomment-690981838






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] QiangCai commented on a change in pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
QiangCai commented on a change in pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#discussion_r483443382



##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,143 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.metadata.SegmentFileStore
+import org.apache.carbondata.core.statusmanager.SegmentStatusManager
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object  PartitionCacheManager extends Cache[PartitionCacheKey, CacheablePartitionSpec] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): CacheablePartitionSpec = {
+    val cacheablePartitionSpec =
+      CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+    val tableStatusModifiedTime = FileFactory
+      .getCarbonFile(CarbonTablePath.getTableStatusFilePath(identifier.tablePath))
+      .getLastModifiedTime
+    if (cacheablePartitionSpec != null) {
+      if (tableStatusModifiedTime > cacheablePartitionSpec.timestamp) {
+        readPartitions(identifier, tableStatusModifiedTime)
+      } else {
+        cacheablePartitionSpec
+      }
+    } else {
+      readPartitions(identifier, tableStatusModifiedTime)
+    }
+  }
+
+  override def getAll(keys: util.List[PartitionCacheKey]):
+  util.List[CacheablePartitionSpec] = {
+    keys.asScala.map(get).toList.asJava
+  }
+
+  override def getIfPresent(key: PartitionCacheKey): CacheablePartitionSpec = {
+    CACHE.get(key.tableId).asInstanceOf[CacheablePartitionSpec]
+  }
+
+  override def invalidate(partitionCacheKey: PartitionCacheKey): Unit = {
+    CACHE.remove(partitionCacheKey.tableId)
+  }
+
+  private def readPartitions(identifier: PartitionCacheKey, tableStatusModifiedTime: Long) = {

Review comment:
       "readPartitions" will read all .segment files after loading. better to load the new .segment file only.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] kunal642 commented on a change in pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
kunal642 commented on a change in pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#discussion_r486993964



##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {

Review comment:
       Can be empty during first load. in that case empty partitions would be stored in the cache




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#discussion_r486970553



##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       readPartition reads the segment file. 
   so basically first time we read all the segment files of the table.
   
   I have seen the users with 200K segments in one table stored in cloud. query takes more than 5 hours for them if we do this.
   
   **So, need to change the key as partition spec and cache only matched partitions**
   
   @kumarvishal09 , @QiangCai @marchpure : what is your opinion on this ?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] kunal642 commented on pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
kunal642 commented on pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#issuecomment-690123684


   @QiangCai build passed


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] QiangCai commented on a change in pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
QiangCai commented on a change in pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#discussion_r486039868



##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)

Review comment:
       move to the outside of the map function

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]

Review comment:
       move to the outside of the map function

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)

Review comment:
       1. cacheablePartitionSpecs doesn't contain an invalid segment.
   2. use partitionSpecs of CACHE - cacheablePartitionSpecs
   

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {

Review comment:
       if finalCache empty, should remove cache if exists

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {

Review comment:
       this always not empty. if the cache exists an invalid segment, it should update the cache.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {

Review comment:
       I mean it should has "else" branch to remove cache

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val existingCache = CACHE.get(identifier.tableId)
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      if (existingCache != null) {
+        val segmentCache = existingCache.asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --

Review comment:
       substraction of different type

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val existingCache = CACHE.get(identifier.tableId)
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      if (existingCache != null) {
+        val segmentCache = existingCache.asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {
+      // remove the existing cache as new cache values may be added.
+      // CarbonLRUCache does not allow cache updation until time is expired.
+      // TODO: Need to fix!!
+      CACHE.remove(identifier.tableId)

Review comment:
       for concurrent query,  line 53 will return null and it will lead to read all partitions.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)

Review comment:
       move to the outside of the map function

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]

Review comment:
       move to the outside of the map function

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)

Review comment:
       1. cacheablePartitionSpecs doesn't contain an invalid segment.
   2. use partitionSpecs of CACHE - cacheablePartitionSpecs
   

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {

Review comment:
       if finalCache empty, should remove cache if exists

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {

Review comment:
       this always not empty. if the cache exists an invalid segment, it should update the cache.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {

Review comment:
       I mean it should has "else" branch to remove cache

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val existingCache = CACHE.get(identifier.tableId)
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      if (existingCache != null) {
+        val segmentCache = existingCache.asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --

Review comment:
       substraction of different type

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val existingCache = CACHE.get(identifier.tableId)
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      if (existingCache != null) {
+        val segmentCache = existingCache.asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {
+      // remove the existing cache as new cache values may be added.
+      // CarbonLRUCache does not allow cache updation until time is expired.
+      // TODO: Need to fix!!
+      CACHE.remove(identifier.tableId)

Review comment:
       for concurrent query,  line 53 will return null and it will lead to read all partitions.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)

Review comment:
       move to the outside of the map function

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]

Review comment:
       move to the outside of the map function

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)

Review comment:
       1. cacheablePartitionSpecs doesn't contain an invalid segment.
   2. use partitionSpecs of CACHE - cacheablePartitionSpecs
   

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {

Review comment:
       if finalCache empty, should remove cache if exists

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {

Review comment:
       this always not empty. if the cache exists an invalid segment, it should update the cache.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {

Review comment:
       I mean it should has "else" branch to remove cache

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val existingCache = CACHE.get(identifier.tableId)
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      if (existingCache != null) {
+        val segmentCache = existingCache.asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --

Review comment:
       substraction of different type

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val existingCache = CACHE.get(identifier.tableId)
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      if (existingCache != null) {
+        val segmentCache = existingCache.asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {
+      // remove the existing cache as new cache values may be added.
+      // CarbonLRUCache does not allow cache updation until time is expired.
+      // TODO: Need to fix!!
+      CACHE.remove(identifier.tableId)

Review comment:
       for concurrent query,  line 53 will return null and it will lead to read all partitions.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)

Review comment:
       move to the outside of the map function

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]

Review comment:
       move to the outside of the map function

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)

Review comment:
       1. cacheablePartitionSpecs doesn't contain an invalid segment.
   2. use partitionSpecs of CACHE - cacheablePartitionSpecs
   

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {

Review comment:
       if finalCache empty, should remove cache if exists

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {

Review comment:
       this always not empty. if the cache exists an invalid segment, it should update the cache.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {

Review comment:
       I mean it should has "else" branch to remove cache

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val existingCache = CACHE.get(identifier.tableId)
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      if (existingCache != null) {
+        val segmentCache = existingCache.asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --

Review comment:
       substraction of different type

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val existingCache = CACHE.get(identifier.tableId)
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      if (existingCache != null) {
+        val segmentCache = existingCache.asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {
+      // remove the existing cache as new cache values may be added.
+      // CarbonLRUCache does not allow cache updation until time is expired.
+      // TODO: Need to fix!!
+      CACHE.remove(identifier.tableId)

Review comment:
       for concurrent query,  line 53 will return null and it will lead to read all partitions.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)

Review comment:
       move to the outside of the map function

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]

Review comment:
       move to the outside of the map function

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)

Review comment:
       1. cacheablePartitionSpecs doesn't contain an invalid segment.
   2. use partitionSpecs of CACHE - cacheablePartitionSpecs
   

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {

Review comment:
       if finalCache empty, should remove cache if exists

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {

Review comment:
       this always not empty. if the cache exists an invalid segment, it should update the cache.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {

Review comment:
       I mean it should has "else" branch to remove cache

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val existingCache = CACHE.get(identifier.tableId)
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      if (existingCache != null) {
+        val segmentCache = existingCache.asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --

Review comment:
       substraction of different type

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val existingCache = CACHE.get(identifier.tableId)
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      if (existingCache != null) {
+        val segmentCache = existingCache.asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {
+      // remove the existing cache as new cache values may be added.
+      // CarbonLRUCache does not allow cache updation until time is expired.
+      // TODO: Need to fix!!
+      CACHE.remove(identifier.tableId)

Review comment:
       for concurrent query,  line 53 will return null and it will lead to read all partitions.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)

Review comment:
       move to the outside of the map function

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]

Review comment:
       move to the outside of the map function

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)

Review comment:
       1. cacheablePartitionSpecs doesn't contain an invalid segment.
   2. use partitionSpecs of CACHE - cacheablePartitionSpecs
   

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {

Review comment:
       if finalCache empty, should remove cache if exists

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {

Review comment:
       this always not empty. if the cache exists an invalid segment, it should update the cache.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {

Review comment:
       I mean it should has "else" branch to remove cache

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val existingCache = CACHE.get(identifier.tableId)
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      if (existingCache != null) {
+        val segmentCache = existingCache.asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --

Review comment:
       substraction of different type

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val existingCache = CACHE.get(identifier.tableId)
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      if (existingCache != null) {
+        val segmentCache = existingCache.asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {
+      // remove the existing cache as new cache values may be added.
+      // CarbonLRUCache does not allow cache updation until time is expired.
+      // TODO: Need to fix!!
+      CACHE.remove(identifier.tableId)

Review comment:
       for concurrent query,  line 53 will return null and it will lead to read all partitions.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)

Review comment:
       move to the outside of the map function

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]

Review comment:
       move to the outside of the map function

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)

Review comment:
       1. cacheablePartitionSpecs doesn't contain an invalid segment.
   2. use partitionSpecs of CACHE - cacheablePartitionSpecs
   

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {

Review comment:
       if finalCache empty, should remove cache if exists

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {

Review comment:
       this always not empty. if the cache exists an invalid segment, it should update the cache.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {

Review comment:
       I mean it should has "else" branch to remove cache

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val existingCache = CACHE.get(identifier.tableId)
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      if (existingCache != null) {
+        val segmentCache = existingCache.asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --

Review comment:
       substraction of different type

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val existingCache = CACHE.get(identifier.tableId)
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      if (existingCache != null) {
+        val segmentCache = existingCache.asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {
+      // remove the existing cache as new cache values may be added.
+      // CarbonLRUCache does not allow cache updation until time is expired.
+      // TODO: Need to fix!!
+      CACHE.remove(identifier.tableId)

Review comment:
       for concurrent query,  line 53 will return null and it will lead to read all partitions.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] QiangCai commented on a change in pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
QiangCai commented on a change in pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#discussion_r486039868



##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)

Review comment:
       move to the outside of the map function

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]

Review comment:
       move to the outside of the map function

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)

Review comment:
       1. cacheablePartitionSpecs doesn't contain an invalid segment.
   2. use partitionSpecs of CACHE - cacheablePartitionSpecs
   

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {

Review comment:
       if finalCache empty, should remove cache if exists

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {

Review comment:
       this always not empty. if the cache exists an invalid segment, it should update the cache.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#discussion_r486970553



##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       readPartition reads the segment file. 
   so basically first time we read all the segment files of the table.
   
   I have seen the users with 200K segments in one table stored in cloud. query takes more than 5 hours for them if we do this.
   
   So, need to change the key as partition spec and cache only matched partitions
   
   @kumarvishal09 , @QiangCai @marchpure : what is your opinion on this ?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] QiangCai commented on pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
QiangCai commented on pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#issuecomment-690981838






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#discussion_r486998066



##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       ok. If the user feels hive metastore getPartitions is slow, still he cannot enable this in cloud scenario if he had the huge number of segments (as IO is very slow) as it will be slower than hive metastore. 
   
   As the feature is disabled by default. Ok for me.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] asfgit closed pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] akashrn5 commented on a change in pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#discussion_r486982098



##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {

Review comment:
       As i can see from here, we are basically removing the cache if the segment file is updated. Actually same thing we do in `TableSegmentRefresher`, and use that for updating all cache, can move this piece of logic there also to unify for all? we use `TableSegmentRefresher` during query, but here we are doing in case of load, can you try if we can do that? 
   
   If not this PR, may be you can give a try in another, as it will be unified and good.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       As you mentioned first time it will take lot of time, but query time we will get benefit. You said matched partitions and since here for every load we are trying to cache, we wont be able to get any matched partitions right?
   
   It is something similar to our Indexes(MV, SI) etc where we load all segment at once in first time and from next time we do incremental, as @QiangCai mentioned. So here from subsequent loads, only that segment should be considered, which i think its being taken care.
   
   @ajantha-bhat are you trying to say to change to handle in query or load itself? Correct me, if i'm wrong. 

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableLoadingTestCase.scala
##########
@@ -585,6 +587,30 @@ class StandardPartitionTableLoadingTestCase extends QueryTest with BeforeAndAfte
     }
   }
 
+  test("test partition caching") {
+    CarbonProperties.getInstance().addProperty("carbon.read.partition.hive.direct", "false")
+    sql("drop table if exists partition_cache")
+    sql("create table partition_cache(a string) partitioned by(b int) stored as carbondata")
+    sql("insert into partition_cache select 'k',1")
+    sql("select * from partition_cache where b = 1").collect()

Review comment:
       can we add one assert to check the cache, after the load instead of query in one scenario?

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       and i also checked now, this caching happens in flow query too, so you can consider my reply specific to load.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {

Review comment:
       yes, you are right, i didn't observe that module level.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] kunal642 commented on a change in pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
kunal642 commented on a change in pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#discussion_r486993964



##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {

Review comment:
       Can be empty during first load. in that case empty partitions would be stored in the cache

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {

Review comment:
       this check is to avoid caching empty cache map

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       @ajantha-bhat This is disabled by default, so if the user feels that hive metastore getPartitions is slow, can enable this to improve performance.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {

Review comment:
       TableSegmentRefresher is in core module, cannot add there

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)

Review comment:
       this is to remove entry of any invalid segment(like deleted segment) from the cacheablePartitionSpecs.
   
   As cacheablePartitionSpecs is a map of (segmentNo -> PartitionSpecs), therefore i am removing all the invalid segments from the final cache.
   
   Does this address your comment? 

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)

Review comment:
       done

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]

Review comment:
       done

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       @ajantha-bhat 

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       @ajantha-bhat 
   
   `sparkSession.sessionState.catalog.listPartitionsByFilter(identifier, partitionFilters)` is already being used for to filter the partitions based on the filter, but in concurrent queries as apark has to take a lock on metastore before accessing it, therefore the query performance was being degraded.
   
   And no user will have 200k segment, they need to anyways compact otherwise the query performance would be degraded

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       Yeah, because they are using presto, where compaction is not yet supported.
   
   If they are using hivemetastore surely they will face degrade in concurrent queries

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableLoadingTestCase.scala
##########
@@ -585,6 +587,30 @@ class StandardPartitionTableLoadingTestCase extends QueryTest with BeforeAndAfte
     }
   }
 
+  test("test partition caching") {
+    CarbonProperties.getInstance().addProperty("carbon.read.partition.hive.direct", "false")
+    sql("drop table if exists partition_cache")
+    sql("create table partition_cache(a string) partitioned by(b int) stored as carbondata")
+    sql("insert into partition_cache select 'k',1")
+    sql("select * from partition_cache where b = 1").collect()

Review comment:
       done

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {

Review comment:
       Can be empty during first load. in that case empty partitions would be stored in the cache

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {

Review comment:
       this check is to avoid caching empty cache map

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       @ajantha-bhat This is disabled by default, so if the user feels that hive metastore getPartitions is slow, can enable this to improve performance.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {

Review comment:
       TableSegmentRefresher is in core module, cannot add there

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)

Review comment:
       this is to remove entry of any invalid segment(like deleted segment) from the cacheablePartitionSpecs.
   
   As cacheablePartitionSpecs is a map of (segmentNo -> PartitionSpecs), therefore i am removing all the invalid segments from the final cache.
   
   Does this address your comment? 

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)

Review comment:
       done

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]

Review comment:
       done

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       @ajantha-bhat 

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       @ajantha-bhat 
   
   `sparkSession.sessionState.catalog.listPartitionsByFilter(identifier, partitionFilters)` is already being used for to filter the partitions based on the filter, but in concurrent queries as apark has to take a lock on metastore before accessing it, therefore the query performance was being degraded.
   
   And no user will have 200k segment, they need to anyways compact otherwise the query performance would be degraded

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       Yeah, because they are using presto, where compaction is not yet supported.
   
   If they are using hivemetastore surely they will face degrade in concurrent queries

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableLoadingTestCase.scala
##########
@@ -585,6 +587,30 @@ class StandardPartitionTableLoadingTestCase extends QueryTest with BeforeAndAfte
     }
   }
 
+  test("test partition caching") {
+    CarbonProperties.getInstance().addProperty("carbon.read.partition.hive.direct", "false")
+    sql("drop table if exists partition_cache")
+    sql("create table partition_cache(a string) partitioned by(b int) stored as carbondata")
+    sql("insert into partition_cache select 'k',1")
+    sql("select * from partition_cache where b = 1").collect()

Review comment:
       done

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {

Review comment:
       Can be empty during first load. in that case empty partitions would be stored in the cache

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {

Review comment:
       this check is to avoid caching empty cache map

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       @ajantha-bhat This is disabled by default, so if the user feels that hive metastore getPartitions is slow, can enable this to improve performance.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {

Review comment:
       TableSegmentRefresher is in core module, cannot add there

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)

Review comment:
       this is to remove entry of any invalid segment(like deleted segment) from the cacheablePartitionSpecs.
   
   As cacheablePartitionSpecs is a map of (segmentNo -> PartitionSpecs), therefore i am removing all the invalid segments from the final cache.
   
   Does this address your comment? 

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)

Review comment:
       done

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]

Review comment:
       done

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       @ajantha-bhat 

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       @ajantha-bhat 
   
   `sparkSession.sessionState.catalog.listPartitionsByFilter(identifier, partitionFilters)` is already being used for to filter the partitions based on the filter, but in concurrent queries as apark has to take a lock on metastore before accessing it, therefore the query performance was being degraded.
   
   And no user will have 200k segment, they need to anyways compact otherwise the query performance would be degraded

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       Yeah, because they are using presto, where compaction is not yet supported.
   
   If they are using hivemetastore surely they will face degrade in concurrent queries

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableLoadingTestCase.scala
##########
@@ -585,6 +587,30 @@ class StandardPartitionTableLoadingTestCase extends QueryTest with BeforeAndAfte
     }
   }
 
+  test("test partition caching") {
+    CarbonProperties.getInstance().addProperty("carbon.read.partition.hive.direct", "false")
+    sql("drop table if exists partition_cache")
+    sql("create table partition_cache(a string) partitioned by(b int) stored as carbondata")
+    sql("insert into partition_cache select 'k',1")
+    sql("select * from partition_cache where b = 1").collect()

Review comment:
       done




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] akashrn5 commented on a change in pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#discussion_r486982098



##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {

Review comment:
       As i can see from here, we are basically removing the cache if the segment file is updated. Actually same thing we do in `TableSegmentRefresher`, and use that for updating all cache, can move this piece of logic there also to unify for all? we use `TableSegmentRefresher` during query, but here we are doing in case of load, can you try if we can do that? 
   
   If not this PR, may be you can give a try in another, as it will be unified and good.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       As you mentioned first time it will take lot of time, but query time we will get benefit. You said matched partitions and since here for every load we are trying to cache, we wont be able to get any matched partitions right?
   
   It is something similar to our Indexes(MV, SI) etc where we load all segment at once in first time and from next time we do incremental, as @QiangCai mentioned. So here from subsequent loads, only that segment should be considered, which i think its being taken care.
   
   @ajantha-bhat are you trying to say to change to handle in query or load itself? Correct me, if i'm wrong. 

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableLoadingTestCase.scala
##########
@@ -585,6 +587,30 @@ class StandardPartitionTableLoadingTestCase extends QueryTest with BeforeAndAfte
     }
   }
 
+  test("test partition caching") {
+    CarbonProperties.getInstance().addProperty("carbon.read.partition.hive.direct", "false")
+    sql("drop table if exists partition_cache")
+    sql("create table partition_cache(a string) partitioned by(b int) stored as carbondata")
+    sql("insert into partition_cache select 'k',1")
+    sql("select * from partition_cache where b = 1").collect()

Review comment:
       can we add one assert to check the cache, after the load instead of query in one scenario?

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       and i also checked now, this caching happens in flow query too, so you can consider my reply specific to load.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {

Review comment:
       yes, you are right, i didn't observe that module level.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] kunal642 commented on a change in pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
kunal642 commented on a change in pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#discussion_r486993964



##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {

Review comment:
       Can be empty during first load. in that case empty partitions would be stored in the cache

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {

Review comment:
       this check is to avoid caching empty cache map

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       @ajantha-bhat This is disabled by default, so if the user feels that hive metastore getPartitions is slow, can enable this to improve performance.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {

Review comment:
       TableSegmentRefresher is in core module, cannot add there

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)

Review comment:
       this is to remove entry of any invalid segment(like deleted segment) from the cacheablePartitionSpecs.
   
   As cacheablePartitionSpecs is a map of (segmentNo -> PartitionSpecs), therefore i am removing all the invalid segments from the final cache.
   
   Does this address your comment? 

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)

Review comment:
       done

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]

Review comment:
       done

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       @ajantha-bhat 

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       @ajantha-bhat 
   
   `sparkSession.sessionState.catalog.listPartitionsByFilter(identifier, partitionFilters)` is already being used for to filter the partitions based on the filter, but in concurrent queries as apark has to take a lock on metastore before accessing it, therefore the query performance was being degraded.
   
   And no user will have 200k segment, they need to anyways compact otherwise the query performance would be degraded

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       Yeah, because they are using presto, where compaction is not yet supported.
   
   If they are using hivemetastore surely they will face degrade in concurrent queries

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableLoadingTestCase.scala
##########
@@ -585,6 +587,30 @@ class StandardPartitionTableLoadingTestCase extends QueryTest with BeforeAndAfte
     }
   }
 
+  test("test partition caching") {
+    CarbonProperties.getInstance().addProperty("carbon.read.partition.hive.direct", "false")
+    sql("drop table if exists partition_cache")
+    sql("create table partition_cache(a string) partitioned by(b int) stored as carbondata")
+    sql("insert into partition_cache select 'k',1")
+    sql("select * from partition_cache where b = 1").collect()

Review comment:
       done




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#issuecomment-688852427


   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/2262/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#discussion_r486998066



##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       ok. If the user feels hive metastore getPartitions is slow, still he cannot enable this in cloud scenario if he had the huge segments as it will be slower than hive metastore. 
   
   As the feature is disabled by default. Ok for me.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#issuecomment-691131796






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] kunal642 commented on a change in pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
kunal642 commented on a change in pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#discussion_r486994128



##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {

Review comment:
       this check is to avoid caching empty cache map




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#issuecomment-690020844


   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/2290/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] akashrn5 commented on a change in pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#discussion_r486985758



##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       As you mentioned first time it will take lot of time, but query time we will get benefit. You said matched partitions and since here for every load we are trying to cache, we wont be able to get any matched partitions right?
   
   It is something similar to our Indexes(MV, SI) etc where we load all segment at once in first time and from next time we do incremental, as @QiangCai mentioned. So here from subsequent loads, only that segment should be considered, which i think its being taken care.
   
   @ajantha-bhat are you trying to say to change to handle in query or load itself? Correct me, if i'm wrong. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] QiangCai commented on a change in pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
QiangCai commented on a change in pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#discussion_r482855055



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonSessionUtil.scala
##########
@@ -103,19 +107,61 @@ object CarbonSessionUtil {
    *
    * @param partitionFilters
    * @param sparkSession
-   * @param identifier
+   * @param carbonTable
    * @return
    */
-  def prunePartitionsByFilter(partitionFilters: Seq[Expression],
+  def pruneAndCachePartitionsByFilters(partitionFilters: Seq[Expression],
       sparkSession: SparkSession,
-      identifier: TableIdentifier): Seq[CatalogTablePartition] = {
-    val allPartitions = sparkSession.sessionState.catalog.listPartitions(identifier)
+      carbonTable: CarbonTable): Seq[CatalogTablePartition] = {
+    val allPartitions = PartitionCacheManager.get(PartitionCacheKey(carbonTable.getTableId,
+      carbonTable.getTablePath, CarbonUtil.getExpiration_time(carbonTable)))
     ExternalCatalogUtils.prunePartitionsByFilter(
-      sparkSession.sessionState.catalog.getTableMetadata(identifier),
+      sparkSession.sessionState.catalog.getTableMetadata(TableIdentifier(carbonTable.getTableName,
+        Some(carbonTable.getDatabaseName))),
+      allPartitions.partitionSpecs,
+      partitionFilters,
+      sparkSession.sessionState.conf.sessionLocalTimeZone
+    )
+  }
+
+  def prunePartitionsByFilter(partitionFilters: Seq[Expression],

Review comment:
       can we remove this method?

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,143 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.metadata.SegmentFileStore
+import org.apache.carbondata.core.statusmanager.SegmentStatusManager
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object  PartitionCacheManager extends Cache[PartitionCacheKey, CacheablePartitionSpec] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): CacheablePartitionSpec = {
+    val cacheablePartitionSpec =
+      CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+    val tableStatusModifiedTime = FileFactory
+      .getCarbonFile(CarbonTablePath.getTableStatusFilePath(identifier.tablePath))
+      .getLastModifiedTime
+    if (cacheablePartitionSpec != null) {
+      if (tableStatusModifiedTime > cacheablePartitionSpec.timestamp) {
+        readPartitions(identifier, tableStatusModifiedTime)
+      } else {
+        cacheablePartitionSpec
+      }
+    } else {
+      readPartitions(identifier, tableStatusModifiedTime)
+    }
+  }
+
+  override def getAll(keys: util.List[PartitionCacheKey]):
+  util.List[CacheablePartitionSpec] = {
+    keys.asScala.map(get).toList.asJava
+  }
+
+  override def getIfPresent(key: PartitionCacheKey): CacheablePartitionSpec = {
+    CACHE.get(key.tableId).asInstanceOf[CacheablePartitionSpec]
+  }
+
+  override def invalidate(partitionCacheKey: PartitionCacheKey): Unit = {
+    CACHE.remove(partitionCacheKey.tableId)
+  }
+
+  private def readPartitions(identifier: PartitionCacheKey, tableStatusModifiedTime: Long) = {

Review comment:
       now it will reload all .segment files again. can we support incremental loading?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] marchpure commented on a change in pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
marchpure commented on a change in pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#discussion_r486976125



##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       agree with you. 
   also avoid to use getAbsolutePath function,which has high IO overhead




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#discussion_r486965218



##########
File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
##########
@@ -1273,6 +1273,13 @@ private CarbonCommonConstants() {
   @CarbonProperty
   public static final String CARBON_MAX_DRIVER_LRU_CACHE_SIZE = "carbon.max.driver.lru.cache.size";
 
+  /**
+   * max driver lru cache size upto which partition lru cache will be loaded in memory
+   */
+  @CarbonProperty

Review comment:
       please also need to update the document

##########
File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
##########
@@ -1273,6 +1273,13 @@ private CarbonCommonConstants() {
   @CarbonProperty
   public static final String CARBON_MAX_DRIVER_LRU_CACHE_SIZE = "carbon.max.driver.lru.cache.size";
 
+  /**
+   * max driver lru cache size upto which partition lru cache will be loaded in memory
+   */
+  @CarbonProperty

Review comment:
       please also update the document

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       readPartition reads the segment file. 
   so basically first time we read all the segment files of the table.
   
   I have seen the users with 200K segments in one table stored in cloud. query takes more than 5 hours for them if we do this.
   
   So, need to change the key as partition spec and cache only matched partitions
   
   @kumarvishal09 , @QiangCai @marchpure : what is your opinion on this ?

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       readPartition reads the segment file. 
   so basically first time we read all the segment files of the table.
   
   I have seen the users with 200K segments in one table stored in cloud. query takes more than 5 hours for them if we do this.
   
   **So, need to change the key as partition spec and cache only matched partitions**
   
   @kumarvishal09 , @QiangCai @marchpure : what is your opinion on this ?

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       readPartition reads the segment file. 
   so basically first time we read all the segment files of the table.
   
   I have seen the users with 200K segments in one table stored in cloud. query takes more than 5 hours for them if we do this.
   
   **So, the current design fails. need to change the key as partition spec and cache only matched partitions**
   
   @kumarvishal09 , @QiangCai @marchpure : what is your opinion on this ?

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       @akashrn5 : yeah, my problem scenario is first time query or upgrade, taking huge time (near to 5 hours for first time query) is still not ok.  In this scenario, I feel reading from hive metastore itself is very fast compared to number of IO from the cloud storage
   

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       ok. If the user feels hive metastore getPartitions is slow, still he cannot enable this in cloud scenario if he had the huge segments as it will be slower than hive metastore. 
   
   As the feature is disabled by default. Ok for me.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       ok. If the user feels hive metastore getPartitions is slow, still he cannot enable this in cloud scenario if he had the huge number of segments (as IO is very slow) as it will be slower than hive metastore. 
   
   As the feature is disabled by default. Ok for me.

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       readPartition reads the segment file. 
   so basically first time we read all the segment files of the table.
   
   I have seen the users with 200K segments in one table stored in cloud. query takes more than 5 hours for them if we do this.
   
   So, the current design fails for huge segments. need to change the key as partition spec or cache only matched partitions from hive metastore read
   
   @kumarvishal09 , @QiangCai @marchpure : what is your opinion on this ?

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       ok. If the user feels hive metastore getPartitions is slow, still he cannot enable this in cloud scenario if he had the huge number of segments (as IO is very slow) as it will be slower than hive metastore. 
   
   Also if the hive metastore supports filter pushdown, they can prune the partitions faster and give us the results using some metadata. (this might already be there) instead of we pruning manually by doing IO of all segment file first.
   
   **As the feature is disabled by default. Ok for me.**

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       we have users with 200k segments. But the data is partitioned by date and time. so all queries will have partition columns filter. so they query-specific segments every time (without any degrade in query time) 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] kunal642 commented on a change in pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
kunal642 commented on a change in pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#discussion_r486996230



##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)

Review comment:
       done

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]

Review comment:
       done




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] asfgit closed pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] akashrn5 commented on a change in pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#discussion_r486998077



##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {

Review comment:
       yes, you are right, i didn't observe that module level.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#issuecomment-691502233


   Agree with @QiangCai. Merging the PR now


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#issuecomment-691131796






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#discussion_r486965218



##########
File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
##########
@@ -1273,6 +1273,13 @@ private CarbonCommonConstants() {
   @CarbonProperty
   public static final String CARBON_MAX_DRIVER_LRU_CACHE_SIZE = "carbon.max.driver.lru.cache.size";
 
+  /**
+   * max driver lru cache size upto which partition lru cache will be loaded in memory
+   */
+  @CarbonProperty

Review comment:
       please also update the document




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] kunal642 commented on a change in pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
kunal642 commented on a change in pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#discussion_r487010034



##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       Yeah, because they are using presto, where compaction is not yet supported.
   
   If they are using hivemetastore surely they will face degrade in concurrent queries




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] akashrn5 commented on a change in pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#discussion_r486992144



##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       and i also checked now, this caching happens in flow query too, so you can consider my reply specific to load.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] kunal642 commented on a change in pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
kunal642 commented on a change in pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#discussion_r484838229



##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,143 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.metadata.SegmentFileStore
+import org.apache.carbondata.core.statusmanager.SegmentStatusManager
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object  PartitionCacheManager extends Cache[PartitionCacheKey, CacheablePartitionSpec] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): CacheablePartitionSpec = {
+    val cacheablePartitionSpec =
+      CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+    val tableStatusModifiedTime = FileFactory
+      .getCarbonFile(CarbonTablePath.getTableStatusFilePath(identifier.tablePath))
+      .getLastModifiedTime
+    if (cacheablePartitionSpec != null) {
+      if (tableStatusModifiedTime > cacheablePartitionSpec.timestamp) {
+        readPartitions(identifier, tableStatusModifiedTime)
+      } else {
+        cacheablePartitionSpec
+      }
+    } else {
+      readPartitions(identifier, tableStatusModifiedTime)
+    }
+  }
+
+  override def getAll(keys: util.List[PartitionCacheKey]):
+  util.List[CacheablePartitionSpec] = {
+    keys.asScala.map(get).toList.asJava
+  }
+
+  override def getIfPresent(key: PartitionCacheKey): CacheablePartitionSpec = {
+    CACHE.get(key.tableId).asInstanceOf[CacheablePartitionSpec]
+  }
+
+  override def invalidate(partitionCacheKey: PartitionCacheKey): Unit = {
+    CACHE.remove(partitionCacheKey.tableId)
+  }
+
+  private def readPartitions(identifier: PartitionCacheKey, tableStatusModifiedTime: Long) = {

Review comment:
       added per segment modification check...now only the updated/new segments would be loaded




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] kunal642 commented on a change in pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
kunal642 commented on a change in pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#discussion_r486057999



##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManger.scala
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {
+      // remove the existing cache as new cache values may be added.
+      // CarbonLRUCache does not allow cache updation until time is expired.
+      // TODO: Need to fix!!
+      CACHE.remove(identifier.tableId)
+      CACHE.put(identifier.tableId,
+        cacheObject,
+        cacheObject.getMemorySize,
+        identifier.expirationTime)
+    }
+    finalCache.values.flatMap(_._1).toList.asJava
+  }
+
+  override def getAll(keys: util.List[PartitionCacheKey]):
+  util.List[util.List[CatalogTablePartition]] = {
+    keys.asScala.toList.map(get).asJava
+  }
+
+  override def getIfPresent(key: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    CACHE.get(key.tableId).asInstanceOf[CacheablePartitionSpec].partitionSpecs.values.flatMap(_._1)
+      .toList.asJava
+  }
+
+  override def invalidate(partitionCacheKey: PartitionCacheKey): Unit = {
+    CACHE.remove(partitionCacheKey.tableId)
+  }
+
+  private def readPartition(identifier: PartitionCacheKey, segmentFilePath: String) = {
+    val segmentFile = SegmentFileStore.readSegmentFile(segmentFilePath)
+    segmentFile.getLocationMap.values().asScala
+      .flatMap(_.getPartitions.asScala).toSet.map { uniquePartition: String =>
+      val partitionSplit = uniquePartition.split("=")
+      val storageFormat = CatalogStorageFormat(
+        Some(new URI(identifier.tablePath + "/" + uniquePartition)),
+        None, None, None, compressed = false, Map())
+      CatalogTablePartition(Map(partitionSplit(0) -> partitionSplit(1)), storageFormat)
+    }.toSeq
+  }
+
+  override def put(key: PartitionCacheKey,
+      value: java.util.List[CatalogTablePartition]): Unit = {
+
+  }
+
+  override def clearAccessCount(keys: util.List[PartitionCacheKey]): Unit = {
+
+  }
+}
+
+case class PartitionCacheKey(tableId: String, tablePath: String, expirationTime: Long)
+
+case class CacheablePartitionSpec(partitionSpecs: Map[String, (Seq[CatalogTablePartition], Long)])

Review comment:
       added




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#issuecomment-691131796


   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/2311/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] marchpure commented on a change in pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
marchpure commented on a change in pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#discussion_r486976125



##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       agree with you. 
   also avoid to use getAbsolutePath function,which has high IO overhead

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       agree with you. 
   also avoid to use getAbsolutePath function,which has high IO overhead

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       agree with you. 
   also avoid to use getAbsolutePath function,which has high IO overhead

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       agree with you. 
   also avoid to use getAbsolutePath function,which has high IO overhead

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       agree with you. 
   also avoid to use getAbsolutePath function,which has high IO overhead

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,

Review comment:
       agree with you. 
   also avoid to use getAbsolutePath function,which has high IO overhead




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] QiangCai commented on pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
QiangCai commented on pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#issuecomment-690981838


   please check the comment


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] QiangCai commented on pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
QiangCai commented on pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#issuecomment-691502093


   @kunal642 please fix the comments in another PR.
   
   LGTM


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] kunal642 commented on a change in pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
kunal642 commented on a change in pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#discussion_r487383241



##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableLoadingTestCase.scala
##########
@@ -585,6 +587,30 @@ class StandardPartitionTableLoadingTestCase extends QueryTest with BeforeAndAfte
     }
   }
 
+  test("test partition caching") {
+    CarbonProperties.getInstance().addProperty("carbon.read.partition.hive.direct", "false")
+    sql("drop table if exists partition_cache")
+    sql("create table partition_cache(a string) partitioned by(b int) stored as carbondata")
+    sql("insert into partition_cache select 'k',1")
+    sql("select * from partition_cache where b = 1").collect()

Review comment:
       done




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [carbondata] QiangCai commented on a change in pull request #3908: [CARBONDATA-3967] cache partition on select to enable faster pruning

Posted by GitBox <gi...@apache.org>.
QiangCai commented on a change in pull request #3908:
URL: https://github.com/apache/carbondata/pull/3908#discussion_r487415994



##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      val existingCache = CACHE.get(identifier.tableId)
+      if (existingCache != null) {
+        val segmentCache = CACHE.get(identifier.tableId).asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {

Review comment:
       I mean it should has "else" branch to remove cache

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val existingCache = CACHE.get(identifier.tableId)
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      if (existingCache != null) {
+        val segmentCache = existingCache.asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --

Review comment:
       substraction of different type

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/PartitionCacheManager.scala
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.util
+
+import java.net.URI
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.{Cache, Cacheable, CarbonLRUCache}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.index.Segment
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFileStore}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+object PartitionCacheManager extends Cache[PartitionCacheKey,
+  java.util.List[CatalogTablePartition]] {
+
+  private val CACHE = new CarbonLRUCache(
+    CarbonCommonConstants.CARBON_PARTITION_MAX_DRIVER_LRU_CACHE_SIZE,
+    CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def get(identifier: PartitionCacheKey): java.util.List[CatalogTablePartition] = {
+    LOGGER.info("Reading partition values from store")
+    // read the tableStatus file to get valid and invalid segments
+    val validInvalidSegments = new SegmentStatusManager(AbsoluteTableIdentifier.from(
+      identifier.tablePath, null, null, identifier.tableId))
+      .getValidAndInvalidSegments
+    val existingCache = CACHE.get(identifier.tableId)
+    val cacheablePartitionSpecs = validInvalidSegments.getValidSegments.asScala.map { segment =>
+      val segmentFileName = segment.getSegmentFileName
+      val segmentFilePath = FileFactory.getCarbonFile(
+        CarbonTablePath.getSegmentFilePath(identifier.tablePath, segmentFileName))
+      // read the last modified time
+      val segmentFileModifiedTime = segmentFilePath.getLastModifiedTime
+      if (existingCache != null) {
+        val segmentCache = existingCache.asInstanceOf[CacheablePartitionSpec]
+          .partitionSpecs.get(segment.getSegmentNo)
+        segmentCache match {
+          case Some(c) =>
+            // check if cache needs to be updated
+            if (segmentFileModifiedTime > c._2) {
+              (segment.getSegmentNo, (readPartition(identifier,
+                segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+            } else {
+              (segment.getSegmentNo, c)
+            }
+          case None =>
+            (segment.getSegmentNo, (readPartition(identifier,
+              segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+        }
+      } else {
+        // read the partitions if not available in cache.
+        (segment.getSegmentNo, (readPartition(identifier,
+          segmentFilePath.getAbsolutePath), segmentFileModifiedTime))
+      }
+    }.toMap
+    // remove all invalid segment entries from cache
+    val finalCache = cacheablePartitionSpecs --
+                     validInvalidSegments.getInvalidSegments.asScala.map(_.getSegmentNo)
+    val cacheObject = CacheablePartitionSpec(finalCache)
+    if (finalCache.nonEmpty) {
+      // remove the existing cache as new cache values may be added.
+      // CarbonLRUCache does not allow cache updation until time is expired.
+      // TODO: Need to fix!!
+      CACHE.remove(identifier.tableId)

Review comment:
       for concurrent query,  line 53 will return null and it will lead to read all partitions.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org