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

classic Classic list List threaded Threaded
74 messages Options
1234
Reply | Threaded
Open this post in threaded view
|

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

GitBox

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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

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

GitBox

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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

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

GitBox
In reply to this post by GitBox

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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

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

GitBox
In reply to this post by GitBox

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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

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

GitBox
In reply to this post by GitBox

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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

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

GitBox
In reply to this post by GitBox

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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

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

GitBox
In reply to this post by GitBox

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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

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

GitBox
In reply to this post by GitBox

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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

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

GitBox
In reply to this post by GitBox

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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

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

GitBox
In reply to this post by GitBox

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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

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

GitBox
In reply to this post by GitBox

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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

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

GitBox
In reply to this post by GitBox

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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

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

GitBox
In reply to this post by GitBox

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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

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

GitBox
In reply to this post by GitBox

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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

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

GitBox
In reply to this post by GitBox

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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

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

GitBox
In reply to this post by GitBox

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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

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

GitBox
In reply to this post by GitBox

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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

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

GitBox
In reply to this post by GitBox

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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

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

GitBox
In reply to this post by GitBox

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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

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

GitBox
In reply to this post by GitBox

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:
[hidden email]


1234