You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@carbondata.apache.org by ravipesala <gi...@git.apache.org> on 2018/02/27 11:09:28 UTC

[GitHub] carbondata pull request #2006: Fixed rename table with partitions not workin...

GitHub user ravipesala opened a pull request:

    https://github.com/apache/carbondata/pull/2006

    Fixed rename table with partitions not working issue and batch_sort and no_sort with partition table issue

    
    
    Be sure to do all of the following checklist to help us incorporate 
    your contribution quickly and easily:
    
     - [ ] Any interfaces changed?
     
     - [ ] Any backward compatibility impacted?
     
     - [ ] Document update required?
    
     - [ ] Testing done
            Please provide details on 
            - Whether new unit test cases have been added or why no new tests are required?
            - How it is tested? Please attach test report.
            - Is it a performance related change? Please attach the performance test report.
            - Any additional information to help reviewers in testing this change.
           
     - [ ] For large changes, please consider breaking it into sub-tasks under an umbrella JIRA. 
    


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/ravipesala/incubator-carbondata partition-batchsort-issue

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/carbondata/pull/2006.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #2006
    
----
commit 5f60c0a312a9670f46dccff5c77b9e271a8a8ee4
Author: ravipesala <ra...@...>
Date:   2018-02-27T11:08:09Z

    Fixed rename table with partitions not working issue and batch_sort and no_sort with partition table issue

----


---

[GitHub] carbondata issue #2006: [CARBONDATA-2209] Fixed rename table with partitions...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2006
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/3982/



---

[GitHub] carbondata issue #2006: [CARBONDATA-2209] Fixed rename table with partitions...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2006
  
    Build Failed with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/2801/



---

[GitHub] carbondata issue #2006: [CARBONDATA-2209] Fixed rename table with partitions...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2006
  
    Build Success with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/2774/



---

[GitHub] carbondata issue #2006: [CARBONDATA-2209] Fixed rename table with partitions...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2006
  
    Build Success with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/2738/



---

[GitHub] carbondata issue #2006: [CARBONDATA-2209] Fixed rename table with partitions...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2006
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/4009/



---

[GitHub] carbondata issue #2006: [CARBONDATA-2209] Fixed rename table with partitions...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2006
  
    Build Success with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/2765/



---

[GitHub] carbondata pull request #2006: [CARBONDATA-2209] Fixed rename table with par...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2006#discussion_r171867005
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/writer/CarbonIndexFileMergeWriter.java ---
    @@ -38,85 +46,158 @@
     
       /**
        * Merge all the carbonindex files of segment to a  merged file
    -   * @param segmentPath
    +   * @param tablePath
        * @param indexFileNamesTobeAdded while merging it comsiders only these files.
        *                                If null then consider all
        * @param readFileFooterFromCarbonDataFile flag to read file footer information from carbondata
        *                                         file. This will used in case of upgrade from version
        *                                         which do not store the blocklet info to current version
        * @throws IOException
        */
    -  private void mergeCarbonIndexFilesOfSegment(String segmentPath,
    -      List<String> indexFileNamesTobeAdded, boolean readFileFooterFromCarbonDataFile)
    -      throws IOException {
    -    CarbonFile[] indexFiles = SegmentIndexFileStore.getCarbonIndexFiles(segmentPath);
    +  private SegmentIndexFIleMergeStatus mergeCarbonIndexFilesOfSegment(String segmentId,
    +      String tablePath, List<String> indexFileNamesTobeAdded,
    +      boolean readFileFooterFromCarbonDataFile) throws IOException {
    +    Segment segment = Segment.getSegment(segmentId, tablePath);
    +    String segmentPath = CarbonTablePath.getSegmentPath(tablePath, segmentId);
    +    CarbonFile[] indexFiles;
    +    SegmentFileStore sfs = null;
    +    if (segment != null && segment.getSegmentFileName() != null) {
    +      sfs = new SegmentFileStore(tablePath, segment.getSegmentFileName());
    +      List<CarbonFile> indexCarbonFiles = sfs.getIndexCarbonFiles();
    +      indexFiles = indexCarbonFiles.toArray(new CarbonFile[indexCarbonFiles.size()]);
    +    } else {
    +      indexFiles = SegmentIndexFileStore.getCarbonIndexFiles(segmentPath);
    +    }
         if (isCarbonIndexFilePresent(indexFiles) || indexFileNamesTobeAdded != null) {
    -      SegmentIndexFileStore fileStore = new SegmentIndexFileStore();
    -      if (readFileFooterFromCarbonDataFile) {
    -        // this case will be used in case of upgrade where old store will not have the blocklet
    -        // info in the index file and therefore blocklet info need to be read from the file footer
    -        // in the carbondata file
    -        fileStore.readAllIndexAndFillBolckletInfo(segmentPath);
    +      if (sfs == null) {
    +        return mergeNormalSegment(indexFileNamesTobeAdded, readFileFooterFromCarbonDataFile,
    +            segmentPath, indexFiles);
           } else {
    -        fileStore.readAllIIndexOfSegment(segmentPath);
    +        return mergePartitionSegment(indexFileNamesTobeAdded, sfs, indexFiles);
           }
    -      Map<String, byte[]> indexMap = fileStore.getCarbonIndexMap();
    -      MergedBlockIndexHeader indexHeader = new MergedBlockIndexHeader();
    -      MergedBlockIndex mergedBlockIndex = new MergedBlockIndex();
    -      List<String> fileNames = new ArrayList<>(indexMap.size());
    -      List<ByteBuffer> data = new ArrayList<>(indexMap.size());
    -      for (Map.Entry<String, byte[]> entry : indexMap.entrySet()) {
    -        if (indexFileNamesTobeAdded == null ||
    -            indexFileNamesTobeAdded.contains(entry.getKey())) {
    -          fileNames.add(entry.getKey());
    -          data.add(ByteBuffer.wrap(entry.getValue()));
    -        }
    +    }
    +    return null;
    +  }
    +
    +
    +  private SegmentIndexFIleMergeStatus mergeNormalSegment(List<String> indexFileNamesTobeAdded,
    +      boolean readFileFooterFromCarbonDataFile, String segmentPath, CarbonFile[] indexFiles)
    +      throws IOException {
    +    SegmentIndexFileStore fileStore = new SegmentIndexFileStore();
    +    if (readFileFooterFromCarbonDataFile) {
    +      // this case will be used in case of upgrade where old store will not have the blocklet
    +      // info in the index file and therefore blocklet info need to be read from the file footer
    +      // in the carbondata file
    +      fileStore.readAllIndexAndFillBolckletInfo(segmentPath);
    +    } else {
    +      fileStore.readAllIIndexOfSegment(segmentPath);
    +    }
    +    Map<String, byte[]> indexMap = fileStore.getCarbonIndexMap();
    +    writeMergeIndexFile(indexFileNamesTobeAdded, segmentPath, indexMap);
    +    for (CarbonFile indexFile : indexFiles) {
    +      indexFile.delete();
    --- End diff --
    
    ok


---

[GitHub] carbondata issue #2006: [CARBONDATA-2209] Fixed rename table with partitions...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2006
  
    Build Success with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/2802/



---

[GitHub] carbondata issue #2006: [CARBONDATA-2209] Fixed rename table with partitions...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2006
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/4018/



---

[GitHub] carbondata issue #2006: [CARBONDATA-2209] Fixed rename table with partitions...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2006
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/4047/



---

[GitHub] carbondata issue #2006: [CARBONDATA-2209] Fixed rename table with partitions...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2006
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/4048/



---

[GitHub] carbondata issue #2006: [CARBONDATA-2209] Fixed rename table with partitions...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/2006
  
    SDV Build Success , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/3739/



---

[GitHub] carbondata issue #2006: [CARBONDATA-2209] Fixed rename table with partitions...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2006
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/3945/



---

[GitHub] carbondata issue #2006: [CARBONDATA-2209] Fixed rename table with partitions...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2006
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/3999/



---

[GitHub] carbondata issue #2006: [CARBONDATA-2209] Fixed rename table with partitions...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2006
  
    Build Failed with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/2755/



---

[GitHub] carbondata issue #2006: [CARBONDATA-2209] Fixed rename table with partitions...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2006
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/3943/



---

[GitHub] carbondata pull request #2006: [CARBONDATA-2209] Fixed rename table with par...

Posted by gvramana <gi...@git.apache.org>.
Github user gvramana commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2006#discussion_r171863712
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala ---
    @@ -138,6 +147,27 @@ private[sql] case class CarbonAlterTableRenameCommand(
               sys.error(s"Folder rename failed for table $oldDatabaseName.$oldTableName")
             }
           }
    +      val updatedParts = updatePartitionLocations(
    --- End diff --
    
    need to check once how hive managed table rename works? whether it renames location or not? If not how table creation with old table name works?


---

[GitHub] carbondata issue #2006: [CARBONDATA-2209] Fixed rename table with partitions...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2006
  
    Build Failed with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/2699/



---

[GitHub] carbondata pull request #2006: [CARBONDATA-2209] Fixed rename table with par...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/carbondata/pull/2006


---

[GitHub] carbondata issue #2006: [CARBONDATA-2209] Fixed rename table with partitions...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/2006
  
    SDV Build Success , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/3706/



---

[GitHub] carbondata pull request #2006: [CARBONDATA-2209] Fixed rename table with par...

Posted by gvramana <gi...@git.apache.org>.
Github user gvramana commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2006#discussion_r171848791
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/SegmentIndexFileStore.java ---
    @@ -108,11 +109,20 @@ public void readAllIIndexOfSegment(SegmentFileStore segmentFileStore, SegmentSta
               location =
                   segmentFileStore.getTablePath() + CarbonCommonConstants.FILE_SEPARATOR + location;
             }
    +        String mergeFileName = locations.getValue().getMergeFileName();
             for (String indexFile : locations.getValue().getFiles()) {
               CarbonFile carbonFile = FileFactory
                   .getCarbonFile(location + CarbonCommonConstants.FILE_SEPARATOR + indexFile);
    -          if (carbonFile.exists()) {
    +          if (carbonFile.exists() && !indexFiles.contains(carbonFile.getAbsolutePath())) {
                 carbonIndexFiles.add(carbonFile);
    +            indexFiles.add(carbonFile.getAbsolutePath());
    +          } else if (mergeFileName != null) {
    --- End diff --
    
    Can move mergeFilename logic out of loop


---

[GitHub] carbondata issue #2006: [CARBONDATA-2209] Fixed rename table with partitions...

Posted by gvramana <gi...@git.apache.org>.
Github user gvramana commented on the issue:

    https://github.com/apache/carbondata/pull/2006
  
    LGTM


---

[GitHub] carbondata pull request #2006: [CARBONDATA-2209] Fixed rename table with par...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2006#discussion_r171867026
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala ---
    @@ -138,6 +147,27 @@ private[sql] case class CarbonAlterTableRenameCommand(
               sys.error(s"Folder rename failed for table $oldDatabaseName.$oldTableName")
             }
           }
    +      val updatedParts = updatePartitionLocations(
    --- End diff --
    
    ok


---

[GitHub] carbondata pull request #2006: [CARBONDATA-2209] Fixed rename table with par...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2006#discussion_r171865808
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/SegmentIndexFileStore.java ---
    @@ -108,11 +109,20 @@ public void readAllIIndexOfSegment(SegmentFileStore segmentFileStore, SegmentSta
               location =
                   segmentFileStore.getTablePath() + CarbonCommonConstants.FILE_SEPARATOR + location;
             }
    +        String mergeFileName = locations.getValue().getMergeFileName();
             for (String indexFile : locations.getValue().getFiles()) {
               CarbonFile carbonFile = FileFactory
                   .getCarbonFile(location + CarbonCommonConstants.FILE_SEPARATOR + indexFile);
    -          if (carbonFile.exists()) {
    +          if (carbonFile.exists() && !indexFiles.contains(carbonFile.getAbsolutePath())) {
                 carbonIndexFiles.add(carbonFile);
    +            indexFiles.add(carbonFile.getAbsolutePath());
    +          } else if (mergeFileName != null) {
    --- End diff --
    
    ok


---

[GitHub] carbondata pull request #2006: [CARBONDATA-2209] Fixed rename table with par...

Posted by gvramana <gi...@git.apache.org>.
Github user gvramana commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2006#discussion_r171852959
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/writer/CarbonIndexFileMergeWriter.java ---
    @@ -38,85 +46,158 @@
     
       /**
        * Merge all the carbonindex files of segment to a  merged file
    -   * @param segmentPath
    +   * @param tablePath
        * @param indexFileNamesTobeAdded while merging it comsiders only these files.
        *                                If null then consider all
        * @param readFileFooterFromCarbonDataFile flag to read file footer information from carbondata
        *                                         file. This will used in case of upgrade from version
        *                                         which do not store the blocklet info to current version
        * @throws IOException
        */
    -  private void mergeCarbonIndexFilesOfSegment(String segmentPath,
    -      List<String> indexFileNamesTobeAdded, boolean readFileFooterFromCarbonDataFile)
    -      throws IOException {
    -    CarbonFile[] indexFiles = SegmentIndexFileStore.getCarbonIndexFiles(segmentPath);
    +  private SegmentIndexFIleMergeStatus mergeCarbonIndexFilesOfSegment(String segmentId,
    +      String tablePath, List<String> indexFileNamesTobeAdded,
    +      boolean readFileFooterFromCarbonDataFile) throws IOException {
    +    Segment segment = Segment.getSegment(segmentId, tablePath);
    +    String segmentPath = CarbonTablePath.getSegmentPath(tablePath, segmentId);
    +    CarbonFile[] indexFiles;
    +    SegmentFileStore sfs = null;
    +    if (segment != null && segment.getSegmentFileName() != null) {
    +      sfs = new SegmentFileStore(tablePath, segment.getSegmentFileName());
    +      List<CarbonFile> indexCarbonFiles = sfs.getIndexCarbonFiles();
    +      indexFiles = indexCarbonFiles.toArray(new CarbonFile[indexCarbonFiles.size()]);
    +    } else {
    +      indexFiles = SegmentIndexFileStore.getCarbonIndexFiles(segmentPath);
    +    }
         if (isCarbonIndexFilePresent(indexFiles) || indexFileNamesTobeAdded != null) {
    -      SegmentIndexFileStore fileStore = new SegmentIndexFileStore();
    -      if (readFileFooterFromCarbonDataFile) {
    -        // this case will be used in case of upgrade where old store will not have the blocklet
    -        // info in the index file and therefore blocklet info need to be read from the file footer
    -        // in the carbondata file
    -        fileStore.readAllIndexAndFillBolckletInfo(segmentPath);
    +      if (sfs == null) {
    +        return mergeNormalSegment(indexFileNamesTobeAdded, readFileFooterFromCarbonDataFile,
    +            segmentPath, indexFiles);
           } else {
    -        fileStore.readAllIIndexOfSegment(segmentPath);
    +        return mergePartitionSegment(indexFileNamesTobeAdded, sfs, indexFiles);
           }
    -      Map<String, byte[]> indexMap = fileStore.getCarbonIndexMap();
    -      MergedBlockIndexHeader indexHeader = new MergedBlockIndexHeader();
    -      MergedBlockIndex mergedBlockIndex = new MergedBlockIndex();
    -      List<String> fileNames = new ArrayList<>(indexMap.size());
    -      List<ByteBuffer> data = new ArrayList<>(indexMap.size());
    -      for (Map.Entry<String, byte[]> entry : indexMap.entrySet()) {
    -        if (indexFileNamesTobeAdded == null ||
    -            indexFileNamesTobeAdded.contains(entry.getKey())) {
    -          fileNames.add(entry.getKey());
    -          data.add(ByteBuffer.wrap(entry.getValue()));
    -        }
    +    }
    +    return null;
    +  }
    +
    +
    +  private SegmentIndexFIleMergeStatus mergeNormalSegment(List<String> indexFileNamesTobeAdded,
    +      boolean readFileFooterFromCarbonDataFile, String segmentPath, CarbonFile[] indexFiles)
    +      throws IOException {
    +    SegmentIndexFileStore fileStore = new SegmentIndexFileStore();
    +    if (readFileFooterFromCarbonDataFile) {
    +      // this case will be used in case of upgrade where old store will not have the blocklet
    +      // info in the index file and therefore blocklet info need to be read from the file footer
    +      // in the carbondata file
    +      fileStore.readAllIndexAndFillBolckletInfo(segmentPath);
    +    } else {
    +      fileStore.readAllIIndexOfSegment(segmentPath);
    +    }
    +    Map<String, byte[]> indexMap = fileStore.getCarbonIndexMap();
    +    writeMergeIndexFile(indexFileNamesTobeAdded, segmentPath, indexMap);
    +    for (CarbonFile indexFile : indexFiles) {
    +      indexFile.delete();
    --- End diff --
    
    deletion should be postponed as parallel read can happen


---

[GitHub] carbondata issue #2006: [CARBONDATA-2209] Fixed rename table with partitions...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2006
  
    Build Success with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/2700/



---