[GitHub] [carbondata] ShreelekhyaG opened a new pull request #4107: [CARBONDATA-4149] Query with SI after add partition based on location on partition table gives incorrect results

classic Classic list List threaded Threaded
34 messages Options
12
Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] ShreelekhyaG opened a new pull request #4107: [CARBONDATA-4149] Query with SI after add partition based on location on partition table gives incorrect results

GitBox

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


    ### Why is this PR needed?
    Query with SI after add partition based on location on partition table gives incorrect results.
   While pruning, if it's an external segment, it should use `ExternalSegmentResolver `, and no need to use `ImplicitIncludeFilterExecutor `as an external segment is not added in the SI table.
   
    ### What changes were proposed in this PR?
   Based on `isRelative `path, set `isExternalSegment `value for partition segment.
   In `getBlockId `method, when segment id is not present, setting from block name.
       
    ### 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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4107: [CARBONDATA-4149] Query with SI after add partition based on location on partition table gives incorrect results

GitBox

CarbonDataQA2 commented on pull request #4107:
URL: https://github.com/apache/carbondata/pull/4107#issuecomment-799446392


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


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

For queries about this service, please contact Infrastructure at:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4107: [CARBONDATA-4149] Query with SI after add partition based on location on partition table gives incorrect results

GitBox
In reply to this post by GitBox

CarbonDataQA2 commented on pull request #4107:
URL: https://github.com/apache/carbondata/pull/4107#issuecomment-799453029


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


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

For queries about this service, please contact Infrastructure at:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4107: [WIP] Query with SI after add partition based on location on partition table gives incorrect results

GitBox
In reply to this post by GitBox

CarbonDataQA2 commented on pull request #4107:
URL: https://github.com/apache/carbondata/pull/4107#issuecomment-801458583


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


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

For queries about this service, please contact Infrastructure at:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4107: [WIP] Query with SI after add partition based on location on partition table gives incorrect results

GitBox
In reply to this post by GitBox

CarbonDataQA2 commented on pull request #4107:
URL: https://github.com/apache/carbondata/pull/4107#issuecomment-801460677


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12444/job/ApacheCarbonPRBuilder2.3/5577/
   


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

For queries about this service, please contact Infrastructure at:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4107: [CARBONDATA-4149] Query with SI after add partition based on location on partition table gives incorrect results

GitBox
In reply to this post by GitBox

CarbonDataQA2 commented on pull request #4107:
URL: https://github.com/apache/carbondata/pull/4107#issuecomment-801779576


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


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

For queries about this service, please contact Infrastructure at:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4107: [CARBONDATA-4149] Query with SI after add partition based on location on partition table gives incorrect results

GitBox
In reply to this post by GitBox

CarbonDataQA2 commented on pull request #4107:
URL: https://github.com/apache/carbondata/pull/4107#issuecomment-801781581


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


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

For queries about this service, please contact Infrastructure at:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #4107: [CARBONDATA-4149] Query with SI after add partition based on location on partition table gives incorrect results

GitBox
In reply to this post by GitBox

Indhumathi27 commented on a change in pull request #4107:
URL: https://github.com/apache/carbondata/pull/4107#discussion_r596686311



##########
File path: core/src/main/java/org/apache/carbondata/core/indexstore/ExtendedBlockletWrapper.java
##########
@@ -121,8 +121,9 @@ public ExtendedBlockletWrapper(List<ExtendedBlocklet> extendedBlockletList, Stri
     DataOutputStream stream = new DataOutputStream(bos);
     try {
       for (ExtendedBlocklet extendedBlocklet : extendedBlockletList) {
+        boolean isExternalPath = !extendedBlocklet.getFilePath().contains(tablePath);

Review comment:
       please check and remove, as discussed

##########
File path: core/src/main/java/org/apache/carbondata/core/readcommitter/TableStatusReadCommittedScope.java
##########
@@ -86,6 +87,13 @@ public TableStatusReadCommittedScope(AbsoluteTableIdentifier identifier,
     } else {
       SegmentFileStore fileStore =
           new SegmentFileStore(identifier.getTablePath(), segment.getSegmentFileName());
+      Optional<SegmentFileStore.FolderDetails>

Review comment:
       please add a comment, about this scenario

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableAddHivePartitionCommand.scala
##########
@@ -137,6 +153,19 @@ case class CarbonAlterTableAddHivePartitionCommand(
         CarbonUtil.checkAndCreateFolderWithPermission(segmentsLoc)
         val segmentPath = segmentsLoc + CarbonCommonConstants.FILE_SEPARATOR + segmentFileName
         SegmentFileStore.writeSegmentFile(segmentFile, segmentPath)
+        CarbonLoaderUtil
+          .recordNewLoadMetadata(newMetaEntry, loadModel, false, false)
+        operationContext.setProperty(table.getTableUniqueName + "_Segment", loadModel.getSegmentId)

Review comment:
       We no need to fire events for SI, since we are not going to load data to SI




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

For queries about this service, please contact Infrastructure at:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #4107: [CARBONDATA-4149] Query with SI after add partition based on location on partition table gives incorrect results

GitBox
In reply to this post by GitBox

Indhumathi27 commented on a change in pull request #4107:
URL: https://github.com/apache/carbondata/pull/4107#discussion_r596714740



##########
File path: core/src/main/java/org/apache/carbondata/core/index/Segment.java
##########
@@ -417,4 +417,8 @@ public void setSegmentMetaDataInfo(SegmentMetaDataInfo segmentMetaDataInfo) {
   public boolean isExternalSegment() {
     return isExternalSegment;
   }
+
+  public void setIsExternalSegment(boolean isExternalSegment) {

Review comment:
       check, if you can add Path in LoadMetadetail for external partition also




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

For queries about this service, please contact Infrastructure at:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #4107: [CARBONDATA-4149] Query with SI after add partition based on location on partition table gives incorrect results

GitBox
In reply to this post by GitBox

ShreelekhyaG commented on a change in pull request #4107:
URL: https://github.com/apache/carbondata/pull/4107#discussion_r596992325



##########
File path: core/src/main/java/org/apache/carbondata/core/index/Segment.java
##########
@@ -417,4 +417,8 @@ public void setSegmentMetaDataInfo(SegmentMetaDataInfo segmentMetaDataInfo) {
   public boolean isExternalSegment() {
     return isExternalSegment;
   }
+
+  public void setIsExternalSegment(boolean isExternalSegment) {

Review comment:
       added path in loadMetadetail.




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

For queries about this service, please contact Infrastructure at:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #4107: [CARBONDATA-4149] Query with SI after add partition based on location on partition table gives incorrect results

GitBox
In reply to this post by GitBox

ShreelekhyaG commented on a change in pull request #4107:
URL: https://github.com/apache/carbondata/pull/4107#discussion_r596994614



##########
File path: core/src/main/java/org/apache/carbondata/core/indexstore/ExtendedBlockletWrapper.java
##########
@@ -121,8 +121,9 @@ public ExtendedBlockletWrapper(List<ExtendedBlocklet> extendedBlockletList, Stri
     DataOutputStream stream = new DataOutputStream(bos);
     try {
       for (ExtendedBlocklet extendedBlocklet : extendedBlockletList) {
+        boolean isExternalPath = !extendedBlocklet.getFilePath().contains(tablePath);

Review comment:
       done




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

For queries about this service, please contact Infrastructure at:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #4107: [CARBONDATA-4149] Query with SI after add partition based on location on partition table gives incorrect results

GitBox
In reply to this post by GitBox

ShreelekhyaG commented on a change in pull request #4107:
URL: https://github.com/apache/carbondata/pull/4107#discussion_r596995121



##########
File path: core/src/main/java/org/apache/carbondata/core/readcommitter/TableStatusReadCommittedScope.java
##########
@@ -86,6 +87,13 @@ public TableStatusReadCommittedScope(AbsoluteTableIdentifier identifier,
     } else {
       SegmentFileStore fileStore =
           new SegmentFileStore(identifier.getTablePath(), segment.getSegmentFileName());
+      Optional<SegmentFileStore.FolderDetails>

Review comment:
       removed this part, as we are setting the path in loadMetaDataDetails, it's no longer required.




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

For queries about this service, please contact Infrastructure at:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #4107: [CARBONDATA-4149] Query with SI after add partition based on location on partition table gives incorrect results

GitBox
In reply to this post by GitBox

ShreelekhyaG commented on a change in pull request #4107:
URL: https://github.com/apache/carbondata/pull/4107#discussion_r596995330



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableAddHivePartitionCommand.scala
##########
@@ -137,6 +153,19 @@ case class CarbonAlterTableAddHivePartitionCommand(
         CarbonUtil.checkAndCreateFolderWithPermission(segmentsLoc)
         val segmentPath = segmentsLoc + CarbonCommonConstants.FILE_SEPARATOR + segmentFileName
         SegmentFileStore.writeSegmentFile(segmentFile, segmentPath)
+        CarbonLoaderUtil
+          .recordNewLoadMetadata(newMetaEntry, loadModel, false, false)
+        operationContext.setProperty(table.getTableUniqueName + "_Segment", loadModel.getSegmentId)

Review comment:
       ok, removed.




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

For queries about this service, please contact Infrastructure at:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #4107: [CARBONDATA-4149] Query with SI after add partition based on location on partition table gives incorrect results

GitBox
In reply to this post by GitBox

Indhumathi27 commented on a change in pull request #4107:
URL: https://github.com/apache/carbondata/pull/4107#discussion_r597099633



##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonTableCompactor.scala
##########
@@ -276,7 +290,25 @@ class CarbonTableCompactor(
           segmentMetaDataAccumulator)
       } else {
         if (mergeRDD != null) {
-          mergeRDD.collect
+          val result = mergeRDD.collect

Review comment:
       Current code will not handle multi-partitions properly and Add/Drop partitions is called for each partition. Please change the code as below:
   `   if (!updatePartitionSpecs.isEmpty) {
               val tableIdentifier = new TableIdentifier(carbonTable.getTableName,
                 Some(carbonTable.getDatabaseName))
               // To update partitionSpec in hive metastore, drop and add with latest path.
               val oldPartition: util.List[TablePartitionSpec] =
                 new util.ArrayList[TablePartitionSpec]()
               val newPartition: util.List[TablePartitionSpec] =
                 new util.ArrayList[TablePartitionSpec]()
               updatePartitionSpecs.asScala.foreach {
                 partitionSpec =>
                   var spec = PartitioningUtils.parsePathFragment(
                     String.join(CarbonCommonConstants.FILE_SEPARATOR, partitionSpec.getPartitions))
                   oldPartition.add(spec)
                   val addPartition = mergeRDD.checkAndUpdatePartitionLocation(partitionSpec)
                   spec = PartitioningUtils.parsePathFragment(
                     String.join(CarbonCommonConstants.FILE_SEPARATOR, addPartition.getPartitions))
                   newPartition.add(spec)
               }
               AlterTableDropPartitionCommand(
                 tableIdentifier,
                 oldPartition.asScala,
                 true, false, true).run(sqlContext.sparkSession)
               AlterTableAddPartitionCommand(tableIdentifier,
                 newPartition.asScala.map(p => (p, None)), false).run(sqlContext.sparkSession)
             }`




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

For queries about this service, please contact Infrastructure at:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #4107: [CARBONDATA-4149] Query with SI after add partition based on location on partition table gives incorrect results

GitBox
In reply to this post by GitBox

Indhumathi27 commented on a change in pull request #4107:
URL: https://github.com/apache/carbondata/pull/4107#discussion_r597099633



##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonTableCompactor.scala
##########
@@ -276,7 +290,25 @@ class CarbonTableCompactor(
           segmentMetaDataAccumulator)
       } else {
         if (mergeRDD != null) {
-          mergeRDD.collect
+          val result = mergeRDD.collect

Review comment:
       Current code will not handle multi-partitions properly and Add/Drop partitions is called for each partition. Please change the code as below:
   
    if (!updatePartitionSpecs.isEmpty) {
               val tableIdentifier = new TableIdentifier(carbonTable.getTableName,
                 Some(carbonTable.getDatabaseName))
               // To update partitionSpec in hive metastore, drop and add with latest path.
               val oldPartition: util.List[TablePartitionSpec] =
                 new util.ArrayList[TablePartitionSpec]()
               val newPartition: util.List[TablePartitionSpec] =
                 new util.ArrayList[TablePartitionSpec]()
               updatePartitionSpecs.asScala.foreach {
                 partitionSpec =>
                   var spec = PartitioningUtils.parsePathFragment(
                     String.join(CarbonCommonConstants.FILE_SEPARATOR, partitionSpec.getPartitions))
                   oldPartition.add(spec)
                   val addPartition = mergeRDD.checkAndUpdatePartitionLocation(partitionSpec)
                   spec = PartitioningUtils.parsePathFragment(
                     String.join(CarbonCommonConstants.FILE_SEPARATOR, addPartition.getPartitions))
                   newPartition.add(spec)
               }
               AlterTableDropPartitionCommand(
                 tableIdentifier,
                 oldPartition.asScala,
                 true, false, true).run(sqlContext.sparkSession)
               AlterTableAddPartitionCommand(tableIdentifier,
                 newPartition.asScala.map(p => (p, None)), false).run(sqlContext.sparkSession)
             }




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

For queries about this service, please contact Infrastructure at:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #4107: [CARBONDATA-4149] Query with SI after add partition based on location on partition table gives incorrect results

GitBox
In reply to this post by GitBox

Indhumathi27 commented on a change in pull request #4107:
URL: https://github.com/apache/carbondata/pull/4107#discussion_r597100457



##########
File path: index/secondary-index/src/test/scala/org/apache/carbondata/spark/testsuite/secondaryindex/TestSIWithPartition.scala
##########
@@ -380,6 +384,37 @@ class TestSIWithPartition extends QueryTest with BeforeAndAfterAll {
     sql("drop table if exists partition_table")
   }
 
+  test("test si with add partition based on location on partition table") {
+    sql("drop table if exists partition_table")
+    sql("create table partition_table (id int,name String) " +
+        "partitioned by(email string) stored as carbondata")
+    sql("insert into partition_table select 1,'blue','abc'")
+    sql("CREATE INDEX partitionTable_si  on table partition_table (name) as 'carbondata'")

Review comment:
       add a new test case for multiple partitions as well




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

For queries about this service, please contact Infrastructure at:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #4107: [CARBONDATA-4149] Query with SI after add partition based on location on partition table gives incorrect results

GitBox
In reply to this post by GitBox

Indhumathi27 commented on a change in pull request #4107:
URL: https://github.com/apache/carbondata/pull/4107#discussion_r597099633



##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonTableCompactor.scala
##########
@@ -276,7 +290,25 @@ class CarbonTableCompactor(
           segmentMetaDataAccumulator)
       } else {
         if (mergeRDD != null) {
-          mergeRDD.collect
+          val result = mergeRDD.collect

Review comment:
       Current code will not handle multi-partitions properly and Add/Drop partitions is called for each partition. Please change the code as below:
   
     if (!updatePartitionSpecs.isEmpty) {
               val tableIdentifier = new TableIdentifier(carbonTable.getTableName,
                 Some(carbonTable.getDatabaseName))
               // To update partitionSpec in hive metastore, drop and add with latest path.
               val partitionSpecList: util.List[TablePartitionSpec] =
                 new util.ArrayList[TablePartitionSpec]()
               updatePartitionSpecs.asScala.foreach {
                 partitionSpec =>
                   var spec = PartitioningUtils.parsePathFragment(
                     String.join(CarbonCommonConstants.FILE_SEPARATOR, partitionSpec.getPartitions))
                   partitionSpecList.add(spec)
               }
               AlterTableDropPartitionCommand(
                 tableIdentifier,
                 partitionSpecList.asScala,
                 true, false, true).run(sqlContext.sparkSession)
               AlterTableAddPartitionCommand(tableIdentifier,
                 partitionSpecList.asScala.map(p => (p, None)), false).run(sqlContext.sparkSession)
   }




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

For queries about this service, please contact Infrastructure at:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #4107: [CARBONDATA-4149] Query with SI after add partition based on location on partition table gives incorrect results

GitBox
In reply to this post by GitBox

Indhumathi27 commented on a change in pull request #4107:
URL: https://github.com/apache/carbondata/pull/4107#discussion_r597099633



##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonTableCompactor.scala
##########
@@ -276,7 +290,25 @@ class CarbonTableCompactor(
           segmentMetaDataAccumulator)
       } else {
         if (mergeRDD != null) {
-          mergeRDD.collect
+          val result = mergeRDD.collect

Review comment:
       Current code will not handle multi-partitions properly and Add/Drop partitions is called for each partition. Please change the code as below:
         
    if (!updatePartitionSpecs.isEmpty) {
               val tableIdentifier = new TableIdentifier(carbonTable.getTableName,
                 Some(carbonTable.getDatabaseName))
               // To update partitionSpec in hive metastore, drop and add with latest path.
               val partitionSpecs = updatePartitionSpecs.asScala.map {
                 partitionSpec =>
                   PartitioningUtils.parsePathFragment(
                     String.join(CarbonCommonConstants.FILE_SEPARATOR, partitionSpec.getPartitions))
               }
               AlterTableDropPartitionCommand(
                 tableIdentifier,
                 partitionSpecs,
                 true, false, true).run(sqlContext.sparkSession)
               AlterTableAddPartitionCommand(tableIdentifier,
                 partitionSpecs.map(p => (p, None)), false).run(sqlContext.sparkSession)
             }




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

For queries about this service, please contact Infrastructure at:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #4107: [CARBONDATA-4149] Query with SI after add partition based on location on partition table gives incorrect results

GitBox
In reply to this post by GitBox

Indhumathi27 commented on a change in pull request #4107:
URL: https://github.com/apache/carbondata/pull/4107#discussion_r597112863



##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
##########
@@ -98,6 +98,24 @@ class CarbonMergerRDD[K, V](
     broadCastSplits = sparkContext.broadcast(new CarbonInputSplitWrapper(splits))
   }
 
+  // checks for added partition specs with external path.
+  // after compaction, location path to be updated with table path.
+  def checkAndUpdatePartitionLocation(partitionSpec: PartitionSpec) : PartitionSpec = {
+    if (partitionSpec != null) {
+      carbonLoadModel.getLoadMetadataDetails.asScala.foreach(loadMetaDetail => {
+        if (loadMetaDetail.getPath != null &&
+            loadMetaDetail.getPath.split(",").contains(partitionSpec.getLocation.toString)) {
+          val updatedPartitionLocation = CarbonDataProcessorUtil

Review comment:
       can break from loop, if matches partition




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

For queries about this service, please contact Infrastructure at:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #4107: [CARBONDATA-4149] Query with SI after add partition based on location on partition table gives incorrect results

GitBox
In reply to this post by GitBox

Indhumathi27 commented on a change in pull request #4107:
URL: https://github.com/apache/carbondata/pull/4107#discussion_r597114312



##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonTableCompactor.scala
##########
@@ -263,6 +267,16 @@ class CarbonTableCompactor(
       if (partitionSpecs != null && partitionSpecs.nonEmpty) {
         compactionCallableModel.compactedPartitions = Some(partitionSpecs)
       }
+      partitionSpecs.foreach(partitionSpec => {
+        carbonLoadModel.getLoadMetadataDetails.asScala.foreach(loadMetaDetail => {
+          if (loadMetaDetail.getPath != null &&
+              loadMetaDetail.getPath.split(",").contains(partitionSpec.getLocation.toString)) {

Review comment:
       same comment as above




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

For queries about this service, please contact Infrastructure at:
[hidden email]


12