Github user KanakaKumar commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2678#discussion_r216130682
  
    --- Diff: 
core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/SegmentIndexFileStore.java
 ---
    @@ -76,20 +77,29 @@
        */
       private Map<String, List<String>> carbonMergeFileToIndexFilesMap;
     
    +  private Configuration configuration;
    +
       public SegmentIndexFileStore() {
         carbonIndexMap = new HashMap<>();
         carbonIndexMapWithFullPath = new TreeMap<>();
         carbonMergeFileToIndexFilesMap = new HashMap<>();
       }
     
    +  public SegmentIndexFileStore(Configuration configuration) {
    +    carbonIndexMap = new HashMap<>();
    +    carbonIndexMapWithFullPath = new TreeMap<>();
    +    carbonMergeFileToIndexFilesMap = new HashMap<>();
    +    this.configuration = configuration;
    +  }
    +
       /**
        * Read all index files and keep the cache in it.
        *
        * @param segmentPath
        * @throws IOException
        */
       public void readAllIIndexOfSegment(String segmentPath) throws 
IOException {
    -    CarbonFile[] carbonIndexFiles = getCarbonIndexFiles(segmentPath);
    +    CarbonFile[] carbonIndexFiles = getCarbonIndexFiles(segmentPath, 
configuration);
    --- End diff --
    
    If Constructor with Configuration is not used, it will throw NPE in this 
method call..
    
    I have observed below stacktrace in a test. Please take care
    
    java.lang.NullPointerException
        at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:378)
        at org.apache.hadoop.fs.Path.getFileSystem(Path.java:295)
        at 
org.apache.carbondata.core.datastore.filesystem.AbstractDFSCarbonFile.getDataInputStream(AbstractDFSCarbonFile.java:336)
        at 
org.apache.carbondata.core.datastore.filesystem.AbstractDFSCarbonFile.getDataInputStream(AbstractDFSCarbonFile.java:302)
        at 
org.apache.carbondata.core.datastore.impl.FileFactory.getDataInputStream(FileFactory.java:125)
        at 
org.apache.carbondata.core.datastore.impl.FileFactory.getDataInputStream(FileFactory.java:116)
        at 
org.apache.carbondata.core.indexstore.blockletindex.SegmentIndexFileStore.readIndexFile(SegmentIndexFileStore.java:298)
        at 
org.apache.carbondata.core.indexstore.blockletindex.SegmentIndexFileStore.readAllIIndexOfSegment(SegmentIndexFileStore.java:156)
        at 
org.apache.carbondata.core.metadata.SegmentFileStore.readIndexFiles(SegmentFileStore.java:497)
        at 
org.apache.carbondata.core.metadata.SegmentFileStore.readIndexFiles(SegmentFileStore.java:474)
        at 
org.apache.carbondata.core.util.CarbonUtil.getDataSizeAndIndexSize(CarbonUtil.java:2624)
        at 
org.apache.carbondata.core.util.CarbonUtil.getDataSizeAndIndexSize(CarbonUtil.java:2675)
        at 
org.apache.carbondata.processing.util.CarbonLoaderUtil.addDataIndexSizeIntoMetaEntry(CarbonLoaderUtil.java:1124)
        at 
org.apache.carbondata.spark.rdd.CarbonDataRDDFactory$.updateTableStatus(CarbonDataRDDFactory.scala:934)
        at 
org.apache.carbondata.spark.rdd.CarbonDataRDDFactory$.loadCarbonData(CarbonDataRDDFactory.scala:539)
        at 
org.apache.spark.sql.execution.command.management.CarbonLoadDataCommand.loadData(CarbonLoadDataCommand.scala:594)
        at 
org.apache.spark.sql.execution.command.management.CarbonLoadDataCommand.processData(CarbonLoadDataCommand.scala:319)
        at 
org.apache.spark.sql.execution.command.management.CarbonInsertIntoCommand.processData(CarbonInsertIntoCommand.scala:85)


---

Reply via email to