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

akashrn5 pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/carbondata.git


The following commit(s) were added to refs/heads/master by this push:
     new ec8bfce  [CARBONDATA-3781] Refactor code to optimize partition pruning
ec8bfce is described below

commit ec8bfce1a35467b3aa144a76fefc3fcdf261bfdd
Author: Indhumathi27 <[email protected]>
AuthorDate: Tue Apr 14 16:37:45 2020 +0530

    [CARBONDATA-3781] Refactor code to optimize partition pruning
    
    Why is this PR needed?
    If number of partitions is more on a segment, cost of checking if 
(indexPath equalsIgnoreCase partitionsToPrune)
    in BlockletIndexFactory.getTableBlockUniqueIdentifierWrappers() is high.
    
    What changes were proposed in this PR?
    1.Store location of partition filter in a SET and check if it contains 
indexPath.
    2. Remove unused variables
    
    This closes #3707
---
 .../apache/carbondata/core/index/IndexUtil.java    | 16 ++---
 .../apache/carbondata/core/index/TableIndex.java   | 72 ++++++++++++----------
 .../apache/carbondata/core/index/dev/Index.java    |  9 ++-
 .../carbondata/core/index/dev/IndexFactory.java    | 10 +--
 .../core/index/dev/cgindex/CoarseGrainIndex.java   |  2 +-
 .../core/index/dev/fgindex/FineGrainIndex.java     |  2 +-
 .../core/indexstore/BlockletDetailsFetcher.java    |  5 +-
 .../core/indexstore/SegmentPropertiesFetcher.java  |  6 +-
 .../core/indexstore/blockletindex/BlockIndex.java  |  6 +-
 .../blockletindex/BlockletIndexFactory.java        | 48 +++++++--------
 .../carbondata/hadoop/api/CarbonInputFormat.java   |  2 +-
 .../hadoop/api/CarbonTableInputFormat.java         |  2 +-
 .../index/bloom/BloomCoarseGrainIndex.java         |  3 +-
 .../index/bloom/BloomCoarseGrainIndexFactory.java  |  4 +-
 .../index/lucene/LuceneFineGrainIndex.java         |  5 +-
 .../index/lucene/LuceneFineGrainIndexFactory.java  |  6 +-
 .../joins/BroadCastSIFilterPushJoin.scala          |  3 +-
 .../spark/testsuite/index/CGIndexTestCase.scala    |  4 +-
 .../spark/testsuite/index/FGIndexTestCase.scala    |  4 +-
 .../spark/testsuite/index/IndexWriterSuite.scala   |  3 +-
 .../spark/testsuite/index/TestIndexStatus.scala    |  4 +-
 .../iud/TestInsertAndOtherCommandConcurrent.scala  |  3 +-
 22 files changed, 113 insertions(+), 106 deletions(-)

diff --git a/core/src/main/java/org/apache/carbondata/core/index/IndexUtil.java 
b/core/src/main/java/org/apache/carbondata/core/index/IndexUtil.java
index c732cab..86954b6 100644
--- a/core/src/main/java/org/apache/carbondata/core/index/IndexUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/index/IndexUtil.java
@@ -187,36 +187,28 @@ public class IndexUtil {
    @param carbonTable
    @param indexExprWrapper
    @param validSegments
-   @param partitionsToPrune
    @throws IOException
    */
   public static void loadIndexes(CarbonTable carbonTable, IndexExprWrapper 
indexExprWrapper,
-      List<Segment> validSegments, List<PartitionSpec> partitionsToPrune) 
throws IOException {
+      List<Segment> validSegments) throws IOException {
     if (!CarbonProperties.getInstance()
         .isDistributedPruningEnabled(carbonTable.getDatabaseName(), 
carbonTable.getTableName())
         && BlockletIndexUtil.loadIndexesParallel(carbonTable)) {
       String clsName = 
"org.apache.spark.sql.secondaryindex.Jobs.SparkBlockletIndexLoaderJob";
       IndexJob indexJob = (IndexJob) createIndexJob(clsName);
       String className = 
"org.apache.spark.sql.secondaryindex.Jobs.BlockletIndexInputFormat";
-      SegmentStatusManager.ValidAndInvalidSegmentsInfo 
validAndInvalidSegmentsInfo =
-          getValidAndInvalidSegments(carbonTable, 
FileFactory.getConfiguration());
-      List<Segment> invalidSegments = 
validAndInvalidSegmentsInfo.getInvalidSegments();
       FileInputFormat indexFormat =
-          createIndexJob(carbonTable, indexExprWrapper, validSegments, 
invalidSegments,
-              partitionsToPrune, className, false);
+          createIndexJob(carbonTable, indexExprWrapper, validSegments, 
className);
       indexJob.execute(carbonTable, indexFormat);
     }
   }
 
   private static FileInputFormat createIndexJob(CarbonTable carbonTable,
-      IndexExprWrapper indexExprWrapper, List<Segment> validsegments,
-      List<Segment> invalidSegments, List<PartitionSpec> partitionsToPrune, 
String clsName,
-      boolean isJobToClearIndexes) {
+      IndexExprWrapper indexExprWrapper, List<Segment> validSegments, String 
clsName) {
     try {
       Constructor<?> cons = 
Class.forName(clsName).getDeclaredConstructors()[0];
       return (FileInputFormat) cons
-          .newInstance(carbonTable, indexExprWrapper, validsegments, 
invalidSegments,
-              partitionsToPrune, isJobToClearIndexes);
+          .newInstance(carbonTable, indexExprWrapper, validSegments);
     } catch (Exception e) {
       throw new RuntimeException(e);
     }
diff --git 
a/core/src/main/java/org/apache/carbondata/core/index/TableIndex.java 
b/core/src/main/java/org/apache/carbondata/core/index/TableIndex.java
index 3d24f90..dd96c39 100644
--- a/core/src/main/java/org/apache/carbondata/core/index/TableIndex.java
+++ b/core/src/main/java/org/apache/carbondata/core/index/TableIndex.java
@@ -20,8 +20,10 @@ package org.apache.carbondata.core.index;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ExecutionException;
@@ -58,6 +60,7 @@ import org.apache.carbondata.events.Event;
 import org.apache.carbondata.events.OperationContext;
 import org.apache.carbondata.events.OperationEventListener;
 
+import org.apache.hadoop.fs.Path;
 import org.apache.log4j.Logger;
 
 /**
@@ -121,8 +124,9 @@ public final class TableIndex extends 
OperationEventListener {
     List<Segment> segments = getCarbonSegments(allsegments);
     final Map<Segment, List<Index>> indexes;
     boolean isFilterPresent = filter != null && !filter.isEmpty();
-    if (table.isHivePartitionTable() && isFilterPresent && partitions != null) 
{
-      indexes = indexFactory.getIndexes(segments, partitions, filter);
+    Set<Path> partitionLocations = getPartitionLocations(partitions);
+    if (table.isHivePartitionTable() && isFilterPresent && 
!partitionLocations.isEmpty()) {
+      indexes = indexFactory.getIndexes(segments, partitionLocations, filter);
     } else {
       indexes = indexFactory.getIndexes(segments, filter);
     }
@@ -156,13 +160,13 @@ public final class TableIndex extends 
OperationEventListener {
       // driver should have minimum threads opened to support multiple 
concurrent queries.
       if (filter == null || filter.isEmpty()) {
         // if filter is not passed, then return all the blocklets.
-        return pruneWithoutFilter(segments, partitions, blocklets);
+        return pruneWithoutFilter(segments, partitionLocations, blocklets);
       }
-      return pruneWithFilter(segments, filter, partitions, blocklets, indexes);
+      return pruneWithFilter(segments, filter, partitionLocations, blocklets, 
indexes);
     }
     // handle by multi-thread
     List<ExtendedBlocklet> extendedBlocklets = pruneMultiThread(
-        segments, filter, partitions, blocklets, indexes, totalFiles);
+        segments, filter, blocklets, indexes, totalFiles);
     return extendedBlocklets;
   }
 
@@ -177,9 +181,10 @@ public final class TableIndex extends 
OperationEventListener {
   }
 
   private List<ExtendedBlocklet> pruneWithoutFilter(List<Segment> segments,
-      List<PartitionSpec> partitions, List<ExtendedBlocklet> blocklets) throws 
IOException {
+      Set<Path> partitionLocations, List<ExtendedBlocklet> blocklets) throws 
IOException {
     for (Segment segment : segments) {
-      List<Blocklet> allBlocklets = 
blockletDetailsFetcher.getAllBlocklets(segment, partitions);
+      List<Blocklet> allBlocklets =
+          blockletDetailsFetcher.getAllBlocklets(segment, partitionLocations);
       blocklets.addAll(
           
addSegmentId(blockletDetailsFetcher.getExtendedBlocklets(allBlocklets, segment),
               segment));
@@ -187,8 +192,18 @@ public final class TableIndex extends 
OperationEventListener {
     return blocklets;
   }
 
+  private Set<Path> getPartitionLocations(List<PartitionSpec> partitionSpecs) {
+    Set<Path> partitionsLocations = new HashSet<>();
+    if (null != partitionSpecs) {
+      for (PartitionSpec partitionSpec : partitionSpecs) {
+        partitionsLocations.add(partitionSpec.getLocation());
+      }
+    }
+    return partitionsLocations;
+  }
+
   private List<ExtendedBlocklet> pruneWithFilter(List<Segment> segments, 
IndexFilter filter,
-      List<PartitionSpec> partitions, List<ExtendedBlocklet> blocklets,
+      Set<Path> partitionLocations, List<ExtendedBlocklet> blocklets,
       Map<Segment, List<Index>> indexes) throws IOException {
     for (Segment segment : segments) {
       if (indexes.get(segment).isEmpty() || indexes.get(segment) == null) {
@@ -197,7 +212,7 @@ public final class TableIndex extends 
OperationEventListener {
       boolean isExternalSegment = segment.getSegmentPath() != null;
       List<Blocklet> pruneBlocklets = new ArrayList<>();
       SegmentProperties segmentProperties =
-          segmentPropertiesFetcher.getSegmentProperties(segment, partitions);
+          segmentPropertiesFetcher.getSegmentProperties(segment, 
partitionLocations);
       if (filter.isResolvedOnSegment(segmentProperties)) {
         FilterExecuter filterExecuter;
         if (!isExternalSegment) {
@@ -212,12 +227,11 @@ public final class TableIndex extends 
OperationEventListener {
         for (Index index : indexes.get(segment)) {
           if (!isExternalSegment) {
             pruneBlocklets.addAll(index
-                .prune(filter.getResolver(), segmentProperties, partitions, 
filterExecuter,
-                    this.table));
+                .prune(filter.getResolver(), segmentProperties, 
filterExecuter, this.table));
           } else {
             pruneBlocklets.addAll(index
-                .prune(filter.getExternalSegmentResolver(), segmentProperties, 
partitions,
-                    filterExecuter, this.table));
+                .prune(filter.getExternalSegmentResolver(), segmentProperties, 
filterExecuter,
+                    this.table));
           }
         }
       } else {
@@ -235,11 +249,10 @@ public final class TableIndex extends 
OperationEventListener {
         for (Index index : indexes.get(segment)) {
           if (!isExternalSegment) {
             pruneBlocklets.addAll(index
-                .prune(filter.getExpression(), segmentProperties, partitions, 
table,
-                    filterExecuter));
+                .prune(filter.getExpression(), segmentProperties, table, 
filterExecuter));
           } else {
             pruneBlocklets.addAll(index
-                .prune(filter.getExternalSegmentFilter(), segmentProperties, 
partitions, table,
+                .prune(filter.getExternalSegmentFilter(), segmentProperties, 
table,
                     filterExecuter));
           }
         }
@@ -252,9 +265,8 @@ public final class TableIndex extends 
OperationEventListener {
   }
 
   private List<ExtendedBlocklet> pruneMultiThread(List<Segment> segments,
-      final IndexFilter filter, final List<PartitionSpec> partitions,
-      List<ExtendedBlocklet> blocklets, final Map<Segment, List<Index>> 
indexes,
-      int totalFiles) {
+      final IndexFilter filter, List<ExtendedBlocklet> blocklets,
+      final Map<Segment, List<Index>> indexes, int totalFiles) {
     /*
      
*********************************************************************************
      * Below is the example of how this part of code works.
@@ -367,12 +379,11 @@ public final class TableIndex extends 
OperationEventListener {
                 List<Blocklet> dmPruneBlocklets;
                 if (!isExternalSegment) {
                   dmPruneBlocklets = indexList.get(i)
-                      .prune(filter.getResolver(), segmentProperties, 
partitions, filterExecuter,
-                          table);
+                      .prune(filter.getResolver(), segmentProperties, 
filterExecuter, table);
                 } else {
                   dmPruneBlocklets = indexList.get(i)
-                      .prune(filter.getExternalSegmentResolver(), 
segmentProperties, partitions,
-                          filterExecuter, table);
+                      .prune(filter.getExternalSegmentResolver(), 
segmentProperties, filterExecuter,
+                          table);
                 }
                 pruneBlocklets.addAll(addSegmentId(
                     
blockletDetailsFetcher.getExtendedBlocklets(dmPruneBlocklets, segment),
@@ -396,12 +407,11 @@ public final class TableIndex extends 
OperationEventListener {
                 List<Blocklet> dmPruneBlocklets;
                 if (!isExternalSegment) {
                   dmPruneBlocklets = indexList.get(i)
-                      .prune(filterExpression, segmentProperties, partitions, 
table,
-                          filterExecuter);
+                      .prune(filterExpression, segmentProperties, table, 
filterExecuter);
                 } else {
                   dmPruneBlocklets = indexList.get(i)
-                      .prune(filter.getExternalSegmentFilter(), 
segmentProperties, partitions,
-                          table, filterExecuter);
+                      .prune(filter.getExternalSegmentFilter(), 
segmentProperties, table,
+                          filterExecuter);
                 }
                 pruneBlocklets.addAll(addSegmentId(
                     
blockletDetailsFetcher.getExtendedBlocklets(dmPruneBlocklets, segment),
@@ -494,15 +504,15 @@ public final class TableIndex extends 
OperationEventListener {
       FilterResolverIntf filterExp, List<PartitionSpec> partitions) throws 
IOException {
     List<ExtendedBlocklet> detailedBlocklets = new ArrayList<>();
     List<Blocklet> blocklets = new ArrayList<>();
-    SegmentProperties segmentProperties =
-        
segmentPropertiesFetcher.getSegmentProperties(distributable.getSegment(), 
partitions);
+    Set<Path> partitionsToPrune = getPartitionLocations(partitions);
+    SegmentProperties segmentProperties = segmentPropertiesFetcher
+        .getSegmentProperties(distributable.getSegment(), partitionsToPrune);
     FilterExecuter filterExecuter = FilterUtil
         .getFilterExecuterTree(filterExp, segmentProperties,
             null, table.getMinMaxCacheColumns(segmentProperties),
             false);
     for (Index index : indices) {
-      blocklets
-          .addAll(index.prune(filterExp, segmentProperties, partitions, 
filterExecuter, table));
+      blocklets.addAll(index.prune(filterExp, segmentProperties, 
filterExecuter, table));
     }
     BlockletSerializer serializer = new BlockletSerializer();
     String writePath =
diff --git a/core/src/main/java/org/apache/carbondata/core/index/dev/Index.java 
b/core/src/main/java/org/apache/carbondata/core/index/dev/Index.java
index 10d894e..3831416 100644
--- a/core/src/main/java/org/apache/carbondata/core/index/dev/Index.java
+++ b/core/src/main/java/org/apache/carbondata/core/index/dev/Index.java
@@ -43,19 +43,18 @@ public interface Index<T extends Blocklet> {
   void init(IndexModel indexModel) throws IOException;
 
   /**
-   * Prune the table with resolved filter expression and partition information.
+   * Prune the table with resolved filter expression.
    * It returns the list of blocklets where these filters can exist.
    */
   List<T> prune(FilterResolverIntf filterExp, SegmentProperties 
segmentProperties,
-      List<PartitionSpec> partitions, FilterExecuter filterExecuter, 
CarbonTable table)
-      throws IOException;
+      FilterExecuter filterExecuter, CarbonTable table) throws IOException;
 
   /**
-   * Prune the table with filter expression and partition information. It 
returns the list of
+   * Prune the table with filter expression. It returns the list of
    * blocklets where these filters can exist.
    */
   List<T> prune(Expression filter, SegmentProperties segmentProperties,
-      List<PartitionSpec> partitions, CarbonTable carbonTable, FilterExecuter 
filterExecuter);
+      CarbonTable carbonTable, FilterExecuter filterExecuter);
 
   /**
    * Prune the data maps for finding the row count. It returns a Map of
diff --git 
a/core/src/main/java/org/apache/carbondata/core/index/dev/IndexFactory.java 
b/core/src/main/java/org/apache/carbondata/core/index/dev/IndexFactory.java
index d0a38f0..a014102 100644
--- a/core/src/main/java/org/apache/carbondata/core/index/dev/IndexFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/index/dev/IndexFactory.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.features.TableOperation;
@@ -31,12 +32,13 @@ import org.apache.carbondata.core.index.IndexMeta;
 import org.apache.carbondata.core.index.Segment;
 import org.apache.carbondata.core.index.dev.cgindex.CoarseGrainIndex;
 import org.apache.carbondata.core.index.dev.expr.IndexInputSplitWrapper;
-import org.apache.carbondata.core.indexstore.PartitionSpec;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.IndexSchema;
 import org.apache.carbondata.events.Event;
 
+import org.apache.hadoop.fs.Path;
+
 /**
  * Factory class for creating the index.
  */
@@ -97,10 +99,10 @@ public abstract class IndexFactory<T extends Index> {
    * matches the partition.
    */
   public Map<Segment, List<CoarseGrainIndex>> getIndexes(List<Segment> 
segments,
-      List<PartitionSpec> partitionSpecs, IndexFilter indexFilter) throws 
IOException {
+      Set<Path> partitionLocations, IndexFilter indexFilter) throws 
IOException {
     Map<Segment, List<CoarseGrainIndex>> indexes = new HashMap<>();
     for (Segment segment : segments) {
-      indexes.put(segment, (List<CoarseGrainIndex>) this.getIndexes(segment, 
partitionSpecs));
+      indexes.put(segment, (List<CoarseGrainIndex>) this.getIndexes(segment, 
partitionLocations));
     }
     return indexes;
   }
@@ -113,7 +115,7 @@ public abstract class IndexFactory<T extends Index> {
   /**
    * Get the index for segmentId with matched partitions
    */
-  public abstract List<T> getIndexes(Segment segment, List<PartitionSpec> 
partitions)
+  public abstract List<T> getIndexes(Segment segment, Set<Path> 
partitionLocations)
       throws IOException;
 
   /**
diff --git 
a/core/src/main/java/org/apache/carbondata/core/index/dev/cgindex/CoarseGrainIndex.java
 
b/core/src/main/java/org/apache/carbondata/core/index/dev/cgindex/CoarseGrainIndex.java
index c76a8f0..c38172b 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/index/dev/cgindex/CoarseGrainIndex.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/index/dev/cgindex/CoarseGrainIndex.java
@@ -40,7 +40,7 @@ public abstract class CoarseGrainIndex implements 
Index<Blocklet> {
 
   @Override
   public List<Blocklet> prune(Expression expression, SegmentProperties 
segmentProperties,
-      List<PartitionSpec> partitions, CarbonTable carbonTable, FilterExecuter 
filterExecuter) {
+      CarbonTable carbonTable, FilterExecuter filterExecuter) {
     throw new UnsupportedOperationException("Filter expression not supported");
   }
 
diff --git 
a/core/src/main/java/org/apache/carbondata/core/index/dev/fgindex/FineGrainIndex.java
 
b/core/src/main/java/org/apache/carbondata/core/index/dev/fgindex/FineGrainIndex.java
index 50cbd1f..e689248 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/index/dev/fgindex/FineGrainIndex.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/index/dev/fgindex/FineGrainIndex.java
@@ -39,7 +39,7 @@ public abstract class FineGrainIndex implements 
Index<FineGrainBlocklet> {
 
   @Override
   public List<FineGrainBlocklet> prune(Expression filter, SegmentProperties 
segmentProperties,
-      List<PartitionSpec> partitions, CarbonTable carbonTable, FilterExecuter 
filterExecuter) {
+      CarbonTable carbonTable, FilterExecuter filterExecuter) {
     throw new UnsupportedOperationException("Filter expression not supported");
   }
 
diff --git 
a/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDetailsFetcher.java
 
b/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDetailsFetcher.java
index e0ef3da..03c137e 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDetailsFetcher.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDetailsFetcher.java
@@ -19,9 +19,12 @@ package org.apache.carbondata.core.indexstore;
 
 import java.io.IOException;
 import java.util.List;
+import java.util.Set;
 
 import org.apache.carbondata.core.index.Segment;
 
+import org.apache.hadoop.fs.Path;
+
 /**
  * Fetches the detailed blocklet which has more information to execute the 
query
  */
@@ -54,7 +57,7 @@ public interface BlockletDetailsFetcher {
    * @param segment
    * @return
    */
-  List<Blocklet> getAllBlocklets(Segment segment, List<PartitionSpec> 
partitions)
+  List<Blocklet> getAllBlocklets(Segment segment, Set<Path> partitionLocations)
       throws IOException;
 
   /**
diff --git 
a/core/src/main/java/org/apache/carbondata/core/indexstore/SegmentPropertiesFetcher.java
 
b/core/src/main/java/org/apache/carbondata/core/indexstore/SegmentPropertiesFetcher.java
index c271a17..5ca767f 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/indexstore/SegmentPropertiesFetcher.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/indexstore/SegmentPropertiesFetcher.java
@@ -18,12 +18,14 @@
 package org.apache.carbondata.core.indexstore;
 
 import java.io.IOException;
-import java.util.List;
+import java.util.Set;
 
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.index.Segment;
 import org.apache.carbondata.core.index.dev.Index;
 
+import org.apache.hadoop.fs.Path;
+
 /**
  * Fetches the detailed segmentProperties which has more information to 
execute the query
  */
@@ -38,7 +40,7 @@ public interface SegmentPropertiesFetcher {
   SegmentProperties getSegmentProperties(Segment segment)
       throws IOException;
 
-  SegmentProperties getSegmentProperties(Segment segment, List<PartitionSpec> 
partitionSpecs)
+  SegmentProperties getSegmentProperties(Segment segment, Set<Path> 
partitionLocations)
       throws IOException;
 
   SegmentProperties getSegmentPropertiesFromIndex(Index coarseGrainIndex);
diff --git 
a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockIndex.java
 
b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockIndex.java
index 966a87f..5412ac8 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockIndex.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockIndex.java
@@ -721,14 +721,14 @@ public class BlockIndex extends CoarseGrainIndex
 
   @Override
   public List<Blocklet> prune(Expression expression, SegmentProperties 
properties,
-      List<PartitionSpec> partitions, CarbonTable carbonTable, FilterExecuter 
filterExecuter) {
+      CarbonTable carbonTable, FilterExecuter filterExecuter) {
     return prune(new IndexFilter(properties, carbonTable, 
expression).getResolver(), properties,
-        partitions, filterExecuter, carbonTable);
+        filterExecuter, carbonTable);
   }
 
   @Override
   public List<Blocklet> prune(FilterResolverIntf filterExp, SegmentProperties 
segmentProperties,
-      List<PartitionSpec> partitions, FilterExecuter filterExecuter, 
CarbonTable table) {
+      FilterExecuter filterExecuter, CarbonTable table) {
     if (memoryDMStore.getRowCount() == 0) {
       return new ArrayList<>();
     }
diff --git 
a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletIndexFactory.java
 
b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletIndexFactory.java
index bb4a601..77251c4 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletIndexFactory.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletIndexFactory.java
@@ -54,7 +54,6 @@ import org.apache.carbondata.core.indexstore.Blocklet;
 import org.apache.carbondata.core.indexstore.BlockletDetailsFetcher;
 import org.apache.carbondata.core.indexstore.BlockletIndexWrapper;
 import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
-import org.apache.carbondata.core.indexstore.PartitionSpec;
 import org.apache.carbondata.core.indexstore.SegmentBlockIndexInfo;
 import org.apache.carbondata.core.indexstore.SegmentPropertiesFetcher;
 import org.apache.carbondata.core.indexstore.TableBlockIndexUniqueIdentifier;
@@ -73,6 +72,8 @@ import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.events.Event;
 
+import org.apache.hadoop.fs.Path;
+
 /**
  * Index for blocklet
  */
@@ -136,14 +137,14 @@ public class BlockletIndexFactory extends 
CoarseGrainIndexFactory
    */
   public Map<Segment, List<CoarseGrainIndex>> getIndexes(List<Segment> 
segments,
       IndexFilter filter) throws IOException {
-    return getIndexes(segments, null, filter);
+    return getIndexes(segments, new HashSet<>(), filter);
   }
 
   /**
    * Get the index for all segments
    */
   public Map<Segment, List<CoarseGrainIndex>> getIndexes(List<Segment> 
segments,
-      List<PartitionSpec> partitionsToPrune, IndexFilter filter) throws 
IOException {
+      Set<Path> partitionLocations, IndexFilter filter) throws IOException {
     List<TableBlockIndexUniqueIdentifierWrapper> 
tableBlockIndexUniqueIdentifierWrappers =
         new ArrayList<>();
     Map<Segment, List<CoarseGrainIndex>> indexMap = new HashMap<>();
@@ -152,9 +153,9 @@ public class BlockletIndexFactory extends 
CoarseGrainIndexFactory
       segmentMap.put(segment.getSegmentNo(), segment);
       Set<TableBlockIndexUniqueIdentifier> identifiers =
           getTableBlockIndexUniqueIdentifiers(segment);
-      if (null != partitionsToPrune) {
+      if (!partitionLocations.isEmpty()) {
         // get tableBlockIndexUniqueIdentifierWrappers from segment file info
-        getTableBlockUniqueIdentifierWrappers(partitionsToPrune,
+        getTableBlockUniqueIdentifierWrappers(partitionLocations,
             tableBlockIndexUniqueIdentifierWrappers, identifiers);
       } else {
         SegmentMetaDataInfo segmentMetaDataInfo = 
segment.getSegmentMetaDataInfo();
@@ -190,24 +191,22 @@ public class BlockletIndexFactory extends 
CoarseGrainIndexFactory
   }
 
   /**
-   * get tableBlockUniqueIdentifierWrappers from segment info. If 
partitionsToPrune is defined,
+   * get tableBlockUniqueIdentifierWrappers from segment info. If 
partitionLocations is defined,
    * then get tableBlockUniqueIdentifierWrappers for the matched partitions.
    */
-  private void getTableBlockUniqueIdentifierWrappers(List<PartitionSpec> 
partitionsToPrune,
+  private void getTableBlockUniqueIdentifierWrappers(Set<Path> 
partitionLocations,
       List<TableBlockIndexUniqueIdentifierWrapper> 
tableBlockIndexUniqueIdentifierWrappers,
       Set<TableBlockIndexUniqueIdentifier> identifiers) {
     for (TableBlockIndexUniqueIdentifier tableBlockIndexUniqueIdentifier : 
identifiers) {
-      if (null != partitionsToPrune) {
+      if (!partitionLocations.isEmpty()) {
         // add only tableBlockUniqueIdentifier that matches the partition
         // get the indexFile Parent path and compare with the PartitionPath, 
if matches, then add
         // the corresponding tableBlockIndexUniqueIdentifier for pruning
-        for (PartitionSpec partitionSpec : partitionsToPrune) {
-          if (partitionSpec.getLocation().toString()
-              
.equalsIgnoreCase(tableBlockIndexUniqueIdentifier.getIndexFilePath())) {
-            tableBlockIndexUniqueIdentifierWrappers.add(
-                new 
TableBlockIndexUniqueIdentifierWrapper(tableBlockIndexUniqueIdentifier,
-                    this.getCarbonTable()));
-          }
+        if (partitionLocations
+            .contains(new 
Path(tableBlockIndexUniqueIdentifier.getIndexFilePath()))) {
+          tableBlockIndexUniqueIdentifierWrappers.add(
+              new 
TableBlockIndexUniqueIdentifierWrapper(tableBlockIndexUniqueIdentifier,
+                  this.getCarbonTable()));
         }
       } else {
         tableBlockIndexUniqueIdentifierWrappers.add(
@@ -328,18 +327,18 @@ public class BlockletIndexFactory extends 
CoarseGrainIndexFactory
 
   @Override
   public List<CoarseGrainIndex> getIndexes(Segment segment) throws IOException 
{
-    return getIndexes(segment, null);
+    return getIndexes(segment, new HashSet<>());
   }
 
   @Override
   public List<CoarseGrainIndex> getIndexes(Segment segment,
-      List<PartitionSpec> partitionsToPrune) throws IOException {
+      Set<Path> partitionLocations) throws IOException {
     List<CoarseGrainIndex> indexes = new ArrayList<>();
     Set<TableBlockIndexUniqueIdentifier> identifiers =
         getTableBlockIndexUniqueIdentifiers(segment);
     List<TableBlockIndexUniqueIdentifierWrapper> 
tableBlockIndexUniqueIdentifierWrappers =
         new ArrayList<>(identifiers.size());
-    getTableBlockUniqueIdentifierWrappers(partitionsToPrune,
+    getTableBlockUniqueIdentifierWrappers(partitionLocations,
         tableBlockIndexUniqueIdentifierWrappers, identifiers);
     List<BlockletIndexWrapper> blockletIndexWrappers =
         cache.getAll(tableBlockIndexUniqueIdentifierWrappers);
@@ -627,13 +626,13 @@ public class BlockletIndexFactory extends 
CoarseGrainIndexFactory
 
   @Override
   public SegmentProperties getSegmentProperties(Segment segment) throws 
IOException {
-    return getSegmentProperties(segment, null);
+    return getSegmentProperties(segment, new HashSet<>());
   }
 
   @Override
-  public SegmentProperties getSegmentProperties(Segment segment, 
List<PartitionSpec> partitions)
+  public SegmentProperties getSegmentProperties(Segment segment, Set<Path> 
partitionLocations)
       throws IOException {
-    List<CoarseGrainIndex> indexes = getIndexes(segment, partitions);
+    List<CoarseGrainIndex> indexes = getIndexes(segment, partitionLocations);
     assert (indexes.size() > 0);
     CoarseGrainIndex coarseGrainIndex = indexes.get(0);
     assert (coarseGrainIndex instanceof BlockIndex);
@@ -649,18 +648,17 @@ public class BlockletIndexFactory extends 
CoarseGrainIndexFactory
   }
 
   @Override
-  public List<Blocklet> getAllBlocklets(Segment segment, List<PartitionSpec> 
partitions)
+  public List<Blocklet> getAllBlocklets(Segment segment, Set<Path> 
partitionLocations)
       throws IOException {
     List<Blocklet> blocklets = new ArrayList<>();
-    List<CoarseGrainIndex> indexes = getIndexes(segment, partitions);
+    List<CoarseGrainIndex> indexes = getIndexes(segment, partitionLocations);
     if (indexes.size() == 0) {
       return blocklets;
     }
     SegmentProperties segmentProperties = 
getSegmentPropertiesFromIndex(indexes.get(0));
     for (CoarseGrainIndex index : indexes) {
       blocklets.addAll(index
-          .prune((FilterResolverIntf) null, segmentProperties, partitions,
-              null, this.getCarbonTable()));
+          .prune(null, segmentProperties, null, this.getCarbonTable()));
     }
     return blocklets;
   }
diff --git 
a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonInputFormat.java 
b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonInputFormat.java
index b5a4b09..868b52b 100644
--- 
a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonInputFormat.java
+++ 
b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonInputFormat.java
@@ -559,7 +559,7 @@ public abstract class CarbonInputFormat<T> extends 
FileInputFormat<Void, T> {
       if (carbonTable.isTransactionalTable()) {
         IndexExprWrapper indexExprWrapper =
             
IndexChooser.getDefaultIndex(getOrCreateCarbonTable(job.getConfiguration()), 
null);
-        IndexUtil.loadIndexes(carbonTable, indexExprWrapper, segmentIds, 
partitionsToPrune);
+        IndexUtil.loadIndexes(carbonTable, indexExprWrapper, segmentIds);
       }
       prunedBlocklets = defaultIndex.prune(segmentIds, filter, 
partitionsToPrune);
 
diff --git 
a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
 
b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
index 7d2c2f7..8d77b92 100644
--- 
a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
+++ 
b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
@@ -478,7 +478,7 @@ public class CarbonTableInputFormat<T> extends 
CarbonInputFormat<T> {
     }
     IndexExprWrapper indexExprWrapper =
         
IndexChooser.getDefaultIndex(getOrCreateCarbonTable(job.getConfiguration()), 
null);
-    IndexUtil.loadIndexes(table, indexExprWrapper, filteredSegment, 
partitions);
+    IndexUtil.loadIndexes(table, indexExprWrapper, filteredSegment);
     if (isIUDTable || isUpdateFlow) {
       Map<String, Long> blockletToRowCountMap = new HashMap<>();
       if (CarbonProperties.getInstance()
diff --git 
a/index/bloom/src/main/java/org/apache/carbondata/index/bloom/BloomCoarseGrainIndex.java
 
b/index/bloom/src/main/java/org/apache/carbondata/index/bloom/BloomCoarseGrainIndex.java
index df05470..40fdcaa 100644
--- 
a/index/bloom/src/main/java/org/apache/carbondata/index/bloom/BloomCoarseGrainIndex.java
+++ 
b/index/bloom/src/main/java/org/apache/carbondata/index/bloom/BloomCoarseGrainIndex.java
@@ -35,7 +35,6 @@ import 
org.apache.carbondata.core.datastore.page.encoding.bool.BooleanConvert;
 import org.apache.carbondata.core.index.dev.IndexModel;
 import org.apache.carbondata.core.index.dev.cgindex.CoarseGrainIndex;
 import org.apache.carbondata.core.indexstore.Blocklet;
-import org.apache.carbondata.core.indexstore.PartitionSpec;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
@@ -133,7 +132,7 @@ public class BloomCoarseGrainIndex extends CoarseGrainIndex 
{
 
   @Override
   public List<Blocklet> prune(FilterResolverIntf filterExp, SegmentProperties 
segmentProperties,
-      List<PartitionSpec> partitions, FilterExecuter filterExecuter, 
CarbonTable carbonTable) {
+      FilterExecuter filterExecuter, CarbonTable carbonTable) {
     Set<Blocklet> hitBlocklets = null;
     if (filterExp == null) {
       // null is different from empty here. Empty means after pruning, no 
blocklet need to scan.
diff --git 
a/index/bloom/src/main/java/org/apache/carbondata/index/bloom/BloomCoarseGrainIndexFactory.java
 
b/index/bloom/src/main/java/org/apache/carbondata/index/bloom/BloomCoarseGrainIndexFactory.java
index f589ed9..c80da44 100644
--- 
a/index/bloom/src/main/java/org/apache/carbondata/index/bloom/BloomCoarseGrainIndexFactory.java
+++ 
b/index/bloom/src/main/java/org/apache/carbondata/index/bloom/BloomCoarseGrainIndexFactory.java
@@ -46,7 +46,6 @@ import org.apache.carbondata.core.index.dev.IndexBuilder;
 import org.apache.carbondata.core.index.dev.IndexFactory;
 import org.apache.carbondata.core.index.dev.IndexWriter;
 import org.apache.carbondata.core.index.dev.cgindex.CoarseGrainIndex;
-import org.apache.carbondata.core.indexstore.PartitionSpec;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.IndexSchema;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
@@ -57,6 +56,7 @@ import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.events.Event;
 
 import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.fs.Path;
 import org.apache.log4j.Logger;
 
 /**
@@ -284,7 +284,7 @@ public class BloomCoarseGrainIndexFactory extends 
IndexFactory<CoarseGrainIndex>
   }
 
   @Override
-  public List<CoarseGrainIndex> getIndexes(Segment segment, 
List<PartitionSpec> partitionSpecs)
+  public List<CoarseGrainIndex> getIndexes(Segment segment, Set<Path> 
partitionLocations)
       throws IOException {
     return getIndexes(segment);
   }
diff --git 
a/index/lucene/src/main/java/org/apache/carbondata/index/lucene/LuceneFineGrainIndex.java
 
b/index/lucene/src/main/java/org/apache/carbondata/index/lucene/LuceneFineGrainIndex.java
index 8616479..09572d0 100644
--- 
a/index/lucene/src/main/java/org/apache/carbondata/index/lucene/LuceneFineGrainIndex.java
+++ 
b/index/lucene/src/main/java/org/apache/carbondata/index/lucene/LuceneFineGrainIndex.java
@@ -32,7 +32,6 @@ import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.index.dev.IndexModel;
 import org.apache.carbondata.core.index.dev.fgindex.FineGrainBlocklet;
 import org.apache.carbondata.core.index.dev.fgindex.FineGrainIndex;
-import org.apache.carbondata.core.indexstore.PartitionSpec;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.IndexSchema;
 import org.apache.carbondata.core.scan.expression.Expression;
@@ -203,8 +202,8 @@ public class LuceneFineGrainIndex extends FineGrainIndex {
    */
   @Override
   public List<FineGrainBlocklet> prune(FilterResolverIntf filterExp,
-      SegmentProperties segmentProperties, List<PartitionSpec> partitions,
-      FilterExecuter filterExecuter, CarbonTable carbonTable) throws 
IOException {
+      SegmentProperties segmentProperties, FilterExecuter filterExecuter,
+      CarbonTable carbonTable) throws IOException {
 
     // convert filter expr into lucene list query
     List<String> fields = new ArrayList<String>();
diff --git 
a/index/lucene/src/main/java/org/apache/carbondata/index/lucene/LuceneFineGrainIndexFactory.java
 
b/index/lucene/src/main/java/org/apache/carbondata/index/lucene/LuceneFineGrainIndexFactory.java
index 650482f..505ae47 100644
--- 
a/index/lucene/src/main/java/org/apache/carbondata/index/lucene/LuceneFineGrainIndexFactory.java
+++ 
b/index/lucene/src/main/java/org/apache/carbondata/index/lucene/LuceneFineGrainIndexFactory.java
@@ -20,6 +20,7 @@ package org.apache.carbondata.index.lucene;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Set;
 
 import org.apache.carbondata.common.annotations.InterfaceAudience;
 import 
org.apache.carbondata.common.exceptions.sql.MalformedIndexCommandException;
@@ -31,10 +32,11 @@ import org.apache.carbondata.core.index.Segment;
 import org.apache.carbondata.core.index.dev.IndexModel;
 import org.apache.carbondata.core.index.dev.IndexWriter;
 import org.apache.carbondata.core.index.dev.fgindex.FineGrainIndex;
-import org.apache.carbondata.core.indexstore.PartitionSpec;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.IndexSchema;
 
+import org.apache.hadoop.fs.Path;
+
 /**
  * FG level of lucene Index
  */
@@ -65,7 +67,7 @@ public class LuceneFineGrainIndexFactory extends 
LuceneIndexFactoryBase<FineGrai
   }
 
   @Override
-  public List<FineGrainIndex> getIndexes(Segment segment, List<PartitionSpec> 
partitions)
+  public List<FineGrainIndex> getIndexes(Segment segment, Set<Path> 
partitionLocations)
       throws IOException {
     return getIndexes(segment);
   }
diff --git 
a/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/joins/BroadCastSIFilterPushJoin.scala
 
b/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/joins/BroadCastSIFilterPushJoin.scala
index 9859ccf..1bbf0ae 100644
--- 
a/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/joins/BroadCastSIFilterPushJoin.scala
+++ 
b/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/joins/BroadCastSIFilterPushJoin.scala
@@ -412,8 +412,7 @@ object BroadCastSIFilterPushJoin {
     IndexUtil.loadIndexes(
       carbonTable,
       indexExprWrapper,
-      segmentIds,
-      CarbonInputFormat.getPartitionsToPrune(configuration))
+      segmentIds)
     blockletMap.pruneSegments(segmentIds, filterResolverIntf)
   }
 
diff --git 
a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/index/CGIndexTestCase.scala
 
b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/index/CGIndexTestCase.scala
index 2fd7930..dc56419 100644
--- 
a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/index/CGIndexTestCase.scala
+++ 
b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/index/CGIndexTestCase.scala
@@ -25,6 +25,7 @@ import scala.collection.mutable.ArrayBuffer
 
 import com.sun.xml.internal.messaging.saaj.util.ByteOutputStream
 import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
 import org.apache.spark.sql.test.util.QueryTest
 import org.scalatest.BeforeAndAfterAll
 
@@ -160,7 +161,7 @@ class CGIndexFactory(
    * Get the indexSchema for segmentId and partitionSpecs
    */
   override def getIndexes(segment: Segment,
-      partitions: java.util.List[PartitionSpec]): 
java.util.List[CoarseGrainIndex] = {
+      partitionLocations: java.util.Set[Path]): 
java.util.List[CoarseGrainIndex] = {
     getIndexes(segment);
   }
 }
@@ -201,7 +202,6 @@ class CGIndex extends CoarseGrainIndex {
   override def prune(
       filterExp: FilterResolverIntf,
       segmentProperties: SegmentProperties,
-      partitions: java.util.List[PartitionSpec],
       filterExecuter: FilterExecuter,
       carbonTable: CarbonTable): java.util.List[Blocklet] = {
     val buffer: ArrayBuffer[Expression] = new ArrayBuffer[Expression]()
diff --git 
a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/index/FGIndexTestCase.scala
 
b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/index/FGIndexTestCase.scala
index 2b7b50c..4f0b49e 100644
--- 
a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/index/FGIndexTestCase.scala
+++ 
b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/index/FGIndexTestCase.scala
@@ -23,6 +23,7 @@ import scala.collection.mutable.ArrayBuffer
 
 import com.sun.xml.internal.messaging.saaj.util.ByteOutputStream
 import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
 import org.apache.spark.sql.test.util.QueryTest
 import org.scalatest.BeforeAndAfterAll
 
@@ -156,7 +157,7 @@ class FGIndexFactory(carbonTable: CarbonTable,
    * Get the indexSchema for segmentId
    */
   override def getIndexes(segment: Segment,
-      partitions: java.util.List[PartitionSpec]): 
java.util.List[FineGrainIndex] = {
+      partitionLocations: java.util.Set[Path]): java.util.List[FineGrainIndex] 
= {
     getIndexes(segment)
   }
 }
@@ -196,7 +197,6 @@ class FGIndex extends FineGrainIndex {
   override def prune(
       filterExp: FilterResolverIntf,
       segmentProperties: SegmentProperties,
-      partitions: java.util.List[PartitionSpec],
       filterExecuter: FilterExecuter,
       carbonTable: CarbonTable): java.util.List[FineGrainBlocklet] = {
     val buffer: ArrayBuffer[Expression] = new ArrayBuffer[Expression]()
diff --git 
a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/index/IndexWriterSuite.scala
 
b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/index/IndexWriterSuite.scala
index 876cf57..c5c58f1 100644
--- 
a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/index/IndexWriterSuite.scala
+++ 
b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/index/IndexWriterSuite.scala
@@ -21,6 +21,7 @@ import java.util
 
 import scala.collection.JavaConverters._
 
+import org.apache.hadoop.fs.Path
 import org.apache.spark.sql.test.util.QueryTest
 import org.apache.spark.sql.{DataFrame, SaveMode}
 import org.scalatest.BeforeAndAfterAll
@@ -98,7 +99,7 @@ class C2IndexFactory(
    * Get the indexSchema for segmentId and partitionSpecs
    */
   override def getIndexes(segment: Segment,
-      partitions: util.List[PartitionSpec]): util.List[CoarseGrainIndex] = {
+      partitionLocations: util.Set[Path]): util.List[CoarseGrainIndex] = {
     ???
   }
 }
diff --git 
a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/index/TestIndexStatus.scala
 
b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/index/TestIndexStatus.scala
index 89dd708..31ee149 100644
--- 
a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/index/TestIndexStatus.scala
+++ 
b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/index/TestIndexStatus.scala
@@ -22,6 +22,7 @@ import java.util
 
 import scala.collection.JavaConverters._
 
+import org.apache.hadoop.fs.Path
 import org.apache.spark.sql.CarbonEnv
 import org.apache.spark.sql.test.util.QueryTest
 import org.scalatest.BeforeAndAfterAll
@@ -34,7 +35,6 @@ import 
org.apache.carbondata.core.index.dev.cgindex.{CoarseGrainIndex, CoarseGra
 import org.apache.carbondata.core.index.dev.{IndexBuilder, IndexWriter}
 import org.apache.carbondata.core.index.status.IndexStatus
 import org.apache.carbondata.core.index.{IndexInputSplit, IndexMeta, Segment}
-import org.apache.carbondata.core.indexstore.PartitionSpec
 import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, 
IndexSchema}
 import org.apache.carbondata.core.scan.filter.intf.ExpressionType
 import org.apache.carbondata.core.util.CarbonProperties
@@ -246,7 +246,7 @@ class TestIndexFactory(
    * Get the indexSchema for segmentId and partitionSpecs
    */
   override def getIndexes(segment: Segment,
-      partitions: util.List[PartitionSpec]): util.List[CoarseGrainIndex] = {
+      partitionLocations: util.Set[Path]): util.List[CoarseGrainIndex] = {
     ???
   }
 }
diff --git 
a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/iud/TestInsertAndOtherCommandConcurrent.scala
 
b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/iud/TestInsertAndOtherCommandConcurrent.scala
index 88d1cf8..cc35915 100644
--- 
a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/iud/TestInsertAndOtherCommandConcurrent.scala
+++ 
b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/iud/TestInsertAndOtherCommandConcurrent.scala
@@ -23,6 +23,7 @@ import java.util.concurrent.{Callable, ExecutorService, 
Executors, Future}
 
 import scala.collection.JavaConverters._
 
+import org.apache.hadoop.fs.Path
 import org.apache.spark.sql.test.util.QueryTest
 import org.apache.spark.sql.types.StructType
 import org.apache.spark.sql.{DataFrame, SaveMode}
@@ -373,7 +374,7 @@ class WaitingIndexFactory(
    * Get the index for segmentId and partitionSpecs
    */
   override def getIndexes(segment: Segment,
-      partitions: util.List[PartitionSpec]): util.List[CoarseGrainIndex] = {
+      partitionLocations: util.Set[Path]): util.List[CoarseGrainIndex] = {
     ???
   }
 }
\ No newline at end of file

Reply via email to