[GitHub] [carbondata] marchpure commented on a change in pull request #3922: [CARBONDATA-3983] SI compatability issue

2020-09-11 Thread GitBox


marchpure commented on a change in pull request #3922:
URL: https://github.com/apache/carbondata/pull/3922#discussion_r487350326



##
File path: 
core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/ImplicitExpression.java
##
@@ -60,12 +61,16 @@ public ImplicitExpression(Map> 
blockIdToBlockletIdMapping)
   private void addBlockEntry(String blockletPath) {
 String blockId =
 blockletPath.substring(0, 
blockletPath.lastIndexOf(CarbonCommonConstants.FILE_SEPARATOR));
+// Check if blockletPath contains old tuple id format, and convert it to 
compatible format.
+if (blockId.contains("batchno")) {
+  blockId = 
CarbonTablePath.getShortBlockId(blockId).substring(blockletPath.indexOf('/') + 
1);
+}
 Set blockletIds = blockIdToBlockletIdMapping.get(blockId);
 if (null == blockletIds) {
   blockletIds = new HashSet<>();
   blockIdToBlockletIdMapping.put(blockId, blockletIds);
 }
-blockletIds.add(Integer.parseInt(blockletPath.substring(blockId.length() + 
1)));
+
blockletIds.add(Integer.parseInt(blockletPath.substring(blockletPath.lastIndexOf('/')
 + 1)));

Review comment:
   use File.seperator instead of '/'

##
File path: 
core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/ImplicitExpression.java
##
@@ -60,12 +61,16 @@ public ImplicitExpression(Map> 
blockIdToBlockletIdMapping)
   private void addBlockEntry(String blockletPath) {
 String blockId =
 blockletPath.substring(0, 
blockletPath.lastIndexOf(CarbonCommonConstants.FILE_SEPARATOR));
+// Check if blockletPath contains old tuple id format, and convert it to 
compatible format.
+if (blockId.contains("batchno")) {
+  blockId = 
CarbonTablePath.getShortBlockId(blockId).substring(blockletPath.indexOf('/') + 
1);

Review comment:
   use File.seperator instead of '/'

##
File path: 
core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/ImplicitExpression.java
##
@@ -60,12 +61,16 @@ public ImplicitExpression(Map> 
blockIdToBlockletIdMapping)
   private void addBlockEntry(String blockletPath) {
 String blockId =
 blockletPath.substring(0, 
blockletPath.lastIndexOf(CarbonCommonConstants.FILE_SEPARATOR));

Review comment:
   use File.seperator instead of CarbonCommonConstants.FILE_SEPARATOR





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:
us...@infra.apache.org




[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3876: TestingCI

2020-09-11 Thread GitBox


CarbonDataQA1 commented on pull request #3876:
URL: https://github.com/apache/carbondata/pull/3876#issuecomment-691212938


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



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:
us...@infra.apache.org




[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3876: TestingCI

2020-09-11 Thread GitBox


CarbonDataQA1 commented on pull request #3876:
URL: https://github.com/apache/carbondata/pull/3876#issuecomment-691211755


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



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:
us...@infra.apache.org




[GitHub] [carbondata] Karan980 commented on pull request #3876: TestingCI

2020-09-11 Thread GitBox


Karan980 commented on pull request #3876:
URL: https://github.com/apache/carbondata/pull/3876#issuecomment-691152266


   retest this please



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:
us...@infra.apache.org




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

2020-09-11 Thread GitBox


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:
us...@infra.apache.org




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

2020-09-11 Thread GitBox


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:
us...@infra.apache.org




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

2020-09-11 Thread GitBox


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:
us...@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

2020-09-11 Thread GitBox


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:
us...@infra.apache.org




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

2020-09-11 Thread GitBox


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:
us...@infra.apache.org




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

2020-09-11 Thread GitBox


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:
us...@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

2020-09-11 Thread GitBox


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:
us...@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

2020-09-11 Thread GitBox


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:
us...@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

2020-09-11 Thread GitBox


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:
us...@infra.apache.org




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

2020-09-11 Thread GitBox


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:
us...@infra.apache.org




[jira] [Created] (CARBONDATA-3983) SI compatability issue

2020-09-11 Thread SHREELEKHYA GAMPA (Jira)
SHREELEKHYA GAMPA created CARBONDATA-3983:
-

 Summary: SI compatability issue
 Key: CARBONDATA-3983
 URL: https://issues.apache.org/jira/browse/CARBONDATA-3983
 Project: CarbonData
  Issue Type: Bug
Reporter: SHREELEKHYA GAMPA


Read from maintable having SI returns empty resultset when SI is stored with 
old tuple id storage format. 

Bug id: BUG2020090205414
PR link: https://github.com/apache/carbondata/pull/3922



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


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

2020-09-11 Thread GitBox


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:
us...@infra.apache.org




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

2020-09-11 Thread GitBox


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 PartitionCach

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

2020-09-11 Thread GitBox


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:
us...@infra.apache.org




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

2020-09-11 Thread GitBox


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:
us...@infra.apache.org




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

2020-09-11 Thread GitBox


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:
us...@infra.apache.org




[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3922: [WIP] SI compatability issue

2020-09-11 Thread GitBox


CarbonDataQA1 commented on pull request #3922:
URL: https://github.com/apache/carbondata/pull/3922#issuecomment-691049096


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



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:
us...@infra.apache.org




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

2020-09-11 Thread GitBox


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:
us...@infra.apache.org




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

2020-09-11 Thread GitBox


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:
us...@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

2020-09-11 Thread GitBox


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:
us...@infra.apache.org




[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3922: [WIP] SI compatability issue

2020-09-11 Thread GitBox


CarbonDataQA1 commented on pull request #3922:
URL: https://github.com/apache/carbondata/pull/3922#issuecomment-691048404


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



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:
us...@infra.apache.org




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

2020-09-11 Thread GitBox


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:
us...@infra.apache.org




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

2020-09-11 Thread GitBox


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:
us...@infra.apache.org




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

2020-09-11 Thread GitBox


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:
us...@infra.apache.org




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

2020-09-11 Thread GitBox


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:
us...@infra.apache.org




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

2020-09-11 Thread GitBox


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:
us...@infra.apache.org




[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3876: TestingCI

2020-09-11 Thread GitBox


CarbonDataQA1 commented on pull request #3876:
URL: https://github.com/apache/carbondata/pull/3876#issuecomment-691029288


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



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:
us...@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

2020-09-11 Thread GitBox


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:
us...@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

2020-09-11 Thread GitBox


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:
us...@infra.apache.org




[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3876: TestingCI

2020-09-11 Thread GitBox


CarbonDataQA1 commented on pull request #3876:
URL: https://github.com/apache/carbondata/pull/3876#issuecomment-691027495


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



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:
us...@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

2020-09-11 Thread GitBox


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:
us...@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

2020-09-11 Thread 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 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:
us...@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

2020-09-11 Thread 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:
us...@infra.apache.org




[GitHub] [carbondata] ShreelekhyaG opened a new pull request #3922: [WIP] SI compatability issue

2020-09-11 Thread GitBox


ShreelekhyaG opened a new pull request #3922:
URL: https://github.com/apache/carbondata/pull/3922


### Why is this PR needed?
   Read from maintable having SI returns empty resultset when SI has with old 
tuple id storage format. 

### What changes were proposed in this PR?
   Checked if blockletPath contains old tuple id format, and convert it to 
compatible format.
   
### 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:
us...@infra.apache.org




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

2020-09-11 Thread 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.Carb

[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3921: [CARBONDATA-3928] Removed records from exception message.

2020-09-11 Thread GitBox


CarbonDataQA1 commented on pull request #3921:
URL: https://github.com/apache/carbondata/pull/3921#issuecomment-690984549


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



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:
us...@infra.apache.org




[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3921: [CARBONDATA-3928] Removed records from exception message.

2020-09-11 Thread GitBox


CarbonDataQA1 commented on pull request #3921:
URL: https://github.com/apache/carbondata/pull/3921#issuecomment-690984968


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



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:
us...@infra.apache.org




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

2020-09-11 Thread 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:
us...@infra.apache.org




[GitHub] [carbondata] Karan-c980 commented on pull request #3876: TestingCI

2020-09-11 Thread GitBox


Karan-c980 commented on pull request #3876:
URL: https://github.com/apache/carbondata/pull/3876#issuecomment-690968014


   retest this please



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:
us...@infra.apache.org




[GitHub] [carbondata] nihal0107 opened a new pull request #3921: [CARBONDATA-3928] Removed records from exception message.

2020-09-11 Thread GitBox


nihal0107 opened a new pull request #3921:
URL: https://github.com/apache/carbondata/pull/3921


### Why is this PR needed?
   Currently, when the string length exceeds 32000 and bad record action as 
default then
   records are thrown in exception message which violates security concern.

### What changes were proposed in this PR?
Removed the records from the exception message and only printed in logger 
file and redirected csv file.
   
### Does this PR introduce any user interface change?
- No
   
### Is any new testcase added?
- 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:
us...@infra.apache.org