You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@carbondata.apache.org by ajantha-bhat <gi...@git.apache.org> on 2018/04/02 07:24:42 UTC

[GitHub] carbondata pull request #2131: [WIP] Support unmanaged carbon table read and...

GitHub user ajantha-bhat opened a pull request:

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

    [WIP] Support unmanaged carbon table read and write

    * carbon SDK writer will take the input data and write back the carbondata and carbonindex files in the path specified.
    This output doesn't have metadata folder. So, it is called unmanaged carbon table.
    
    *This can be read by creating external table in the location of sdk writer output path.
    Please refer, 
    **TestUnmanagedCarbonTable.scla** for the example scenario.
    
    *Load, insert, compaction, alter, IUD etc features are blocked for unmanaged table.
    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/ajantha-bhat/carbondata unmanaged_table

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

    https://github.com/apache/carbondata/pull/2131.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 #2131
    
----
commit 3075af0a47506509e7b6ba75352a90b1238a8d7e
Author: ajantha-bhat <aj...@...>
Date:   2018-03-27T08:30:48Z

    unmanaged table backup

commit 618ee43c0e03a25bd0036f02ff95cad4d8f896ef
Author: ajantha-bhat <aj...@...>
Date:   2018-03-27T08:32:24Z

    unmanaged table backup_1

commit 1bb8c4d328725a7abb6a3c1e450445162394dc27
Author: ajantha-bhat <aj...@...>
Date:   2018-03-27T11:16:23Z

    unmanaged working

commit c129cfe291c8403812ff04118292d98c9265eaa9
Author: ajantha-bhat <aj...@...>
Date:   2018-03-27T16:14:17Z

    unmanaged metadata folder issue

commit b1146c2b2e82e1af70ece61943f25a0e9571db4d
Author: sounakr <so...@...>
Date:   2018-03-27T11:32:09Z

    Writer And SDK Changes

commit c87788c40834b92226d1fc8ecd33d7d0fd2eaf21
Author: sounakr <so...@...>
Date:   2018-03-28T14:12:57Z

    Create External table fix

commit 59bc5423af6999d9ae38c621ebc9ae99bb714325
Author: ajantha-bhat <aj...@...>
Date:   2018-03-29T06:03:19Z

    findbugs fix

commit 2e4fd0532b1113daec85cdb67a1217aa2dae6791
Author: ajantha-bhat <aj...@...>
Date:   2018-03-29T07:19:28Z

    fixed metadata issue

commit a1fa8e4e19017c3ec8cd05f324ae291609c19d19
Author: ajantha-bhat <aj...@...>
Date:   2018-03-29T14:14:56Z

    Added the testcase for unmanaged table

commit c4ad34a2920e0e00e026aed153096a56238ed029
Author: sounakr <so...@...>
Date:   2018-03-29T14:19:32Z

    SDK changes Phase 1

commit e280f771ea6098f21b1f6fc365b5fcb8c467f1fc
Author: sounakr <so...@...>
Date:   2018-04-01T14:11:52Z

    Committer Reader Implementation

commit cd5d2b8260b37f1e866401fe2ec616ce3697208b
Author: sounakr <so...@...>
Date:   2018-04-02T07:12:49Z

    Rebase Changes

----


---

[GitHub] carbondata pull request #2131: [CARBONDATA-2313] Support unmanaged carbon ta...

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

    https://github.com/apache/carbondata/pull/2131#discussion_r179361367
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/readcommitter/TableStatusReadCommittedScope.java ---
    @@ -0,0 +1,76 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.carbondata.core.readcommitter;
    +
    +import java.io.IOException;
    +import java.util.Map;
    +
    +import org.apache.carbondata.common.annotations.InterfaceAudience;
    +import org.apache.carbondata.common.annotations.InterfaceStability;
    +import org.apache.carbondata.core.datamap.Segment;
    +import org.apache.carbondata.core.indexstore.blockletindex.SegmentIndexFileStore;
    +import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
    +import org.apache.carbondata.core.metadata.SegmentFileStore;
    +import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
    +import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
    +import org.apache.carbondata.core.util.path.CarbonTablePath;
    +
    +/**
    + * ReadCommittedScope for the managed carbon table
    + */
    +@InterfaceAudience.Internal
    +@InterfaceStability.Stable
    +public class TableStatusReadCommittedScope implements ReadCommittedScope {
    +  private LoadMetadataDetails[] loadMetadataDetails;
    +  private AbsoluteTableIdentifier identifier;
    +
    +  public TableStatusReadCommittedScope(AbsoluteTableIdentifier identifier) {
    +    this.identifier = identifier;
    +    takeCarbonIndexFileSnapShot();
    +  }
    +
    +  @Override public LoadMetadataDetails[] getSegmentList() throws IOException {
    +    try {
    +      this.loadMetadataDetails = SegmentStatusManager
    +          .readTableStatusFile(CarbonTablePath.getTableStatusFilePath(identifier.getTablePath()));
    +      return this.loadMetadataDetails;
    +    } catch (IOException ex) {
    +      this.loadMetadataDetails = new LoadMetadataDetails[0];
    --- End diff --
    
    should throw exception here, it is not valid behaviour to return empty list


---

[GitHub] carbondata issue #2131: [CARBONDATA-2313] Support unmanaged carbon table rea...

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

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



---

[GitHub] carbondata pull request #2131: [WIP] Support unmanaged carbon table read and...

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

    https://github.com/apache/carbondata/pull/2131#discussion_r178813379
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java ---
    @@ -83,28 +85,44 @@ public DataMapWriter createWriter(Segment segment, String writeDirectoryPath) {
       }
     
       @Override
    -  public List<CoarseGrainDataMap> getDataMaps(Segment segment) throws IOException {
    +  public List<CoarseGrainDataMap> getDataMaps(Segment segment, ReadCommitted readCommitted)
    +      throws IOException {
         List<TableBlockIndexUniqueIdentifier> tableBlockIndexUniqueIdentifiers =
    -        getTableBlockIndexUniqueIdentifiers(segment);
    +        getTableBlockIndexUniqueIdentifiers(segment, readCommitted);
         return cache.getAll(tableBlockIndexUniqueIdentifiers);
       }
     
    -  private List<TableBlockIndexUniqueIdentifier> getTableBlockIndexUniqueIdentifiers(
    -      Segment segment) throws IOException {
    +  private List<TableBlockIndexUniqueIdentifier> getTableBlockIndexUniqueIdentifiers(Segment segment,
    +      ReadCommitted readCommitted) throws IOException {
         List<TableBlockIndexUniqueIdentifier> tableBlockIndexUniqueIdentifiers =
             segmentMap.get(segment.getSegmentNo());
         if (tableBlockIndexUniqueIdentifiers == null) {
           tableBlockIndexUniqueIdentifiers = new ArrayList<>();
    +
           Map<String, String> indexFiles;
    -      if (segment.getSegmentFileName() == null) {
    -        String path =
    -            CarbonTablePath.getSegmentPath(identifier.getTablePath(), segment.getSegmentNo());
    -        indexFiles = new SegmentIndexFileStore().getIndexFilesFromSegment(path);
    +      if (CarbonUtil.isUnmanagedCarbonTable(identifier.getTablePath(), true)) {
    --- End diff --
    
    done


---

[GitHub] carbondata issue #2131: [CARBONDATA-2313] Support unmanaged carbon table rea...

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

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


---

[GitHub] carbondata issue #2131: [CARBONDATA-2313] Support unmanaged carbon table rea...

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

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



---

[GitHub] carbondata issue #2131: [CARBONDATA-2313] Support unmanaged carbon table rea...

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

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



---

[GitHub] carbondata issue #2131: [WIP] Support unmanaged carbon table read and write

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

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



---

[GitHub] carbondata issue #2131: [WIP] Support unmanaged carbon table read and write

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

    https://github.com/apache/carbondata/pull/2131
  
    Retest this please.


---

[GitHub] carbondata pull request #2131: [WIP] Support unmanaged carbon table read and...

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

    https://github.com/apache/carbondata/pull/2131#discussion_r178717012
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/readcommitter/LatestFilesReadCommitted.java ---
    @@ -0,0 +1,122 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.carbondata.core.readcommitter;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.HashMap;
    +import java.util.Map;
    +
    +import org.apache.carbondata.core.datamap.Segment;
    +import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
    +import org.apache.carbondata.core.indexstore.blockletindex.SegmentIndexFileStore;
    +import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
    +import org.apache.carbondata.core.statusmanager.SegmentStatus;
    +import org.apache.carbondata.core.util.path.CarbonTablePath;
    +
    +public class LatestFilesReadCommitted implements ReadCommitted {
    +
    +  private String carbonFilePath;
    +  private ReadCommittedIndexFileSnapShot readCommittedIndexFileSnapShot;
    +
    +  public LatestFilesReadCommitted(String path) {
    +    this.carbonFilePath = path;
    +    try {
    +      takeCarbonIndexFileSnapShot();
    +    } catch (IOException ex) {
    +      // TODO Put proper Log and throw the exception out.
    +      System.out.println("Error while reding index file");
    +    }
    +  }
    +
    +  @Override public LoadMetadataDetails[] getSegmentList() throws IOException {
    +    // Read the Segment path and form the LoadMetadataDetails array.
    +    File fs = new File(carbonFilePath);
    +
    +    if (fs.isDirectory()) {
    +
    +      CarbonFile[] carbonIndexFiles = SegmentIndexFileStore.getCarbonIndexFiles(carbonFilePath);
    --- End diff --
    
    should read from readCommittedIndexFileSnapShot, not filesystem


---

[GitHub] carbondata issue #2131: [WIP] Support unmanaged carbon table read and write

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

    https://github.com/apache/carbondata/pull/2131
  
    SDV Build Fail , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/4285/



---

[GitHub] carbondata issue #2131: [WIP] Support unmanaged carbon table read and write

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

    https://github.com/apache/carbondata/pull/2131
  
    SDV Build Fail , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/4287/



---

[GitHub] carbondata pull request #2131: [WIP] Support unmanaged carbon table read and...

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

    https://github.com/apache/carbondata/pull/2131#discussion_r178811570
  
    --- Diff: hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java ---
    @@ -224,22 +198,28 @@ protected CarbonTable getOrCreateCarbonTable(Configuration configuration) throws
     
         // do block filtering and get split
         List<InputSplit> splits =
    -        getSplits(job, filterInterface, filteredSegmentToAccess, matchedPartitions, partitionInfo,
    -            null, updateStatusManager);
    +         getSplits(job, filterInterface, filteredSegmentToAccess, matchedPartitions, partitionInfo,
    +            null, updateStatusManager, readCommitted);
    +
         // pass the invalid segment to task side in order to remove index entry in task side
    -    if (invalidSegments.size() > 0) {
    -      for (InputSplit split : splits) {
    -        ((org.apache.carbondata.hadoop.CarbonInputSplit) split).setInvalidSegments(invalidSegments);
    -        ((org.apache.carbondata.hadoop.CarbonInputSplit) split)
    -            .setInvalidTimestampRange(invalidTimestampsList);
    +    if (readCommitted instanceof TableStatusReadCommitted) {
    --- End diff --
    
    done.


---

[GitHub] carbondata issue #2131: [CARBONDATA-2313] Support unmanaged carbon table rea...

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

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



---

[GitHub] carbondata issue #2131: [WIP] Support unmanaged carbon table read and write

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

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



---

[GitHub] carbondata issue #2131: [CARBONDATA-2313] Support unmanaged carbon table rea...

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

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



---

[GitHub] carbondata pull request #2131: [WIP] Support unmanaged carbon table read and...

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

    https://github.com/apache/carbondata/pull/2131#discussion_r178714736
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/SegmentIndexFileStore.java ---
    @@ -200,6 +216,90 @@ public void readAllIIndexOfSegment(CarbonFile[] carbonFiles) throws IOException
         return indexFiles;
       }
     
    +  /**
    +   * Read all index file names of the carbon File Path.
    +   *
    +   * @return
    +   * @throws IOException
    +   */
    +  public Map<String, String> getReadCommittedIndexFilesFromPath(String carbonFilePath)
    +      throws IOException {
    +    // Get only those index files which are mentioned in the
    +    CarbonFile[] carbonIndexFiles = getCarbonIndexFiles(carbonFilePath, carbonIndexMap);
    +    Map<String, String> indexFiles = new HashMap<>();
    +    for (int i = 0; i < carbonIndexFiles.length; i++) {
    +      if (carbonIndexFiles[i].getName().endsWith(CarbonTablePath.MERGE_INDEX_FILE_EXT)) {
    +        List<String> indexFilesFromMergeFile =
    +            getIndexFilesFromMergeFile(carbonIndexFiles[i].getCanonicalPath());
    +        for (String file : indexFilesFromMergeFile) {
    +          indexFiles.put(carbonIndexFiles[i].getParentFile().getAbsolutePath()
    +              + CarbonCommonConstants.FILE_SEPARATOR + file, carbonIndexFiles[i].getName());
    +        }
    +      } else if (carbonIndexFiles[i].getName().endsWith(CarbonTablePath.INDEX_FILE_EXT)) {
    +        indexFiles.put(carbonIndexFiles[i].getAbsolutePath(), null);
    +      }
    +    }
    +    return indexFiles;
    +  }
    +
    +  /**
    +   * Read all index file names of the segment
    +   *
    +   * @return
    +   * @throws IOException
    +   */
    +  public Map<String, String> getReadCommittedIndexFilesFromPath(String path, List<Segment> segments)
    +      throws IOException {
    +    // Only returns indexes matching the segment.
    +    Map<String, String> indexFiles = new HashMap<>();
    +    for (Segment seg : segments) {
    +      CarbonFile[] carbonIndexFiles = getCarbonIndexFiles(path, carbonIndexMap);
    +      for (int i = 0; i < carbonIndexFiles.length; i++) {
    --- End diff --
    
    Duplicate logic across functions


---

[GitHub] carbondata issue #2131: [WIP] Support unmanaged carbon table read and write

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

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



---

[GitHub] carbondata issue #2131: [WIP] Support unmanaged carbon table read and write

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

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



---

[GitHub] carbondata issue #2131: [CARBONDATA-2313] Support unmanaged carbon table rea...

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

    https://github.com/apache/carbondata/pull/2131
  
    retest this please


---

[GitHub] carbondata issue #2131: [CARBONDATA-2313] Support unmanaged carbon table rea...

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

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



---

[GitHub] carbondata issue #2131: [WIP] Support unmanaged carbon table read and write

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

    https://github.com/apache/carbondata/pull/2131
  
    Retest this please


---

[GitHub] carbondata pull request #2131: [WIP] Support unmanaged carbon table read and...

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

    https://github.com/apache/carbondata/pull/2131#discussion_r178594069
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java ---
    @@ -79,26 +80,27 @@
        *
        * @param segments
        * @param filterExp
    +   * @param readCommitted
        * @return
        */
       public List<ExtendedBlocklet> prune(List<Segment> segments, FilterResolverIntf filterExp,
    -      List<PartitionSpec> partitions) throws IOException {
    +      List<PartitionSpec> partitions, ReadCommitted readCommitted) throws IOException {
    --- End diff --
    
    ReadCommited is implemented in order to set the read scope. Currently for Unmanaged Table the Read Scope is LatestFileRead which is going to take a snapshot of the list of Index files during query phase starting i.e. in CarbonInputSplit. In later phase of query life cycle even though new files get placed, only files read during initial snapshot is only returned. 
    Similarly TableStatusReadCommitter is going to take a snapshot of Table Status file and will be used by Managed Table. One time snapshot of LoadMetadataDetails will be taken in CarbonTableInputFormat and later the Table Status snapshot will be only referred to return the list of files. So in parallel in case Table Status gets updated then new updated segment and file list wont be reflected. This way we defined the read commit scope. 


---

[GitHub] carbondata issue #2131: [WIP] Support unmanaged carbon table read and write

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

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



---

[GitHub] carbondata issue #2131: [CARBONDATA-2313] Support unmanaged carbon table rea...

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

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



---

[GitHub] carbondata pull request #2131: [WIP] Support unmanaged carbon table read and...

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

    https://github.com/apache/carbondata/pull/2131#discussion_r178732017
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/readcommitter/LatestFilesReadCommitted.java ---
    @@ -0,0 +1,122 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.carbondata.core.readcommitter;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.HashMap;
    +import java.util.Map;
    +
    +import org.apache.carbondata.core.datamap.Segment;
    +import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
    +import org.apache.carbondata.core.indexstore.blockletindex.SegmentIndexFileStore;
    +import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
    +import org.apache.carbondata.core.statusmanager.SegmentStatus;
    +import org.apache.carbondata.core.util.path.CarbonTablePath;
    +
    +public class LatestFilesReadCommitted implements ReadCommitted {
    +
    +  private String carbonFilePath;
    +  private ReadCommittedIndexFileSnapShot readCommittedIndexFileSnapShot;
    +
    +  public LatestFilesReadCommitted(String path) {
    +    this.carbonFilePath = path;
    +    try {
    +      takeCarbonIndexFileSnapShot();
    +    } catch (IOException ex) {
    +      // TODO Put proper Log and throw the exception out.
    +      System.out.println("Error while reding index file");
    +    }
    +  }
    +
    +  @Override public LoadMetadataDetails[] getSegmentList() throws IOException {
    +    // Read the Segment path and form the LoadMetadataDetails array.
    +    File fs = new File(carbonFilePath);
    +
    +    if (fs.isDirectory()) {
    +
    +      CarbonFile[] carbonIndexFiles = SegmentIndexFileStore.getCarbonIndexFiles(carbonFilePath);
    --- End diff --
    
    Currently it is read from the FileSystem and matched with the readCommittedIndexFileSnapShot. In case the file is present in readCommittedIndexFileSnapShot then only it is choosen. 


---

[GitHub] carbondata issue #2131: [WIP] Support unmanaged carbon table read and write

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

    https://github.com/apache/carbondata/pull/2131
  
    retest this please


---

[GitHub] carbondata pull request #2131: [WIP] Support unmanaged carbon table read and...

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

    https://github.com/apache/carbondata/pull/2131#discussion_r178795222
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java ---
    @@ -83,28 +85,44 @@ public DataMapWriter createWriter(Segment segment, String writeDirectoryPath) {
       }
     
       @Override
    -  public List<CoarseGrainDataMap> getDataMaps(Segment segment) throws IOException {
    +  public List<CoarseGrainDataMap> getDataMaps(Segment segment, ReadCommitted readCommitted)
    +      throws IOException {
         List<TableBlockIndexUniqueIdentifier> tableBlockIndexUniqueIdentifiers =
    -        getTableBlockIndexUniqueIdentifiers(segment);
    +        getTableBlockIndexUniqueIdentifiers(segment, readCommitted);
         return cache.getAll(tableBlockIndexUniqueIdentifiers);
       }
     
    -  private List<TableBlockIndexUniqueIdentifier> getTableBlockIndexUniqueIdentifiers(
    -      Segment segment) throws IOException {
    +  private List<TableBlockIndexUniqueIdentifier> getTableBlockIndexUniqueIdentifiers(Segment segment,
    +      ReadCommitted readCommitted) throws IOException {
         List<TableBlockIndexUniqueIdentifier> tableBlockIndexUniqueIdentifiers =
             segmentMap.get(segment.getSegmentNo());
         if (tableBlockIndexUniqueIdentifiers == null) {
           tableBlockIndexUniqueIdentifiers = new ArrayList<>();
    +
           Map<String, String> indexFiles;
    -      if (segment.getSegmentFileName() == null) {
    -        String path =
    -            CarbonTablePath.getSegmentPath(identifier.getTablePath(), segment.getSegmentNo());
    -        indexFiles = new SegmentIndexFileStore().getIndexFilesFromSegment(path);
    +      if (CarbonUtil.isUnmanagedCarbonTable(identifier.getTablePath(), true)) {
    --- End diff --
    
    Now this logic is not required, as readCommitted.getCommittedIndexFile(segment); should handle both the cases


---

[GitHub] carbondata issue #2131: [WIP] Support unmanaged carbon table read and write

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

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



---

[GitHub] carbondata pull request #2131: [WIP] Support unmanaged carbon table read and...

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

    https://github.com/apache/carbondata/pull/2131#discussion_r178812558
  
    --- Diff: hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonInputFormat.java ---
    @@ -159,6 +162,11 @@ public static void setTablePath(Configuration configuration, String tablePath) {
         configuration.set(FileInputFormat.INPUT_DIR, tablePath);
       }
     
    +  public static void setCarbonUnmanagedTable(Configuration configuration,
    --- End diff --
    
    configuration.set supports string only now. followed like other config set in CarbinTableInputFormat


---

[GitHub] carbondata issue #2131: [CARBONDATA-2313] Support unmanaged carbon table rea...

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

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



---

[GitHub] carbondata issue #2131: [WIP] Support unmanaged carbon table read and write

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

    https://github.com/apache/carbondata/pull/2131
  
    SDV Build Fail , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/4254/



---

[GitHub] carbondata pull request #2131: [WIP] Support unmanaged carbon table read and...

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

    https://github.com/apache/carbondata/pull/2131#discussion_r179032115
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/CarbonCountStar.scala ---
    @@ -76,6 +76,11 @@ case class CarbonCountStar(
         SparkHadoopUtil.get.addCredentials(jobConf)
         val job = new Job(jobConf)
         FileInputFormat.addInputPath(job, new Path(absoluteTableIdentifier.getTablePath))
    +    if (carbonTable.getTableInfo.isUnManagedTable) {
    +      CarbonInputFormat.setCarbonUnmanagedTable(job.getConfiguration, "true")
    --- End diff --
    
    setter change it to boolean


---

[GitHub] carbondata issue #2131: [CARBONDATA-2313] Support unmanaged carbon table rea...

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

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



---

[GitHub] carbondata pull request #2131: [CARBONDATA-2313] Support unmanaged carbon ta...

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

    https://github.com/apache/carbondata/pull/2131#discussion_r179360518
  
    --- Diff: hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java ---
    @@ -128,12 +134,18 @@ protected CarbonTable getOrCreateCarbonTable(Configuration configuration) throws
       @Override
       public List<InputSplit> getSplits(JobContext job) throws IOException {
         AbsoluteTableIdentifier identifier = getAbsoluteTableIdentifier(job.getConfiguration());
    -    LoadMetadataDetails[] loadMetadataDetails = SegmentStatusManager
    -        .readTableStatusFile(CarbonTablePath.getTableStatusFilePath(identifier.getTablePath()));
    +
         CarbonTable carbonTable = getOrCreateCarbonTable(job.getConfiguration());
         if (null == carbonTable) {
           throw new IOException("Missing/Corrupt schema file for table.");
         }
    +    this.readCommittedScope = getReadCommitted(job, identifier);
    --- End diff --
    
    done


---

[GitHub] carbondata issue #2131: [WIP] Support unmanaged carbon table read and write

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

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



---

[GitHub] carbondata issue #2131: [CARBONDATA-2313] Support unmanaged carbon table rea...

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

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



---

[GitHub] carbondata issue #2131: [WIP] Support unmanaged carbon table read and write

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

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



---

[GitHub] carbondata pull request #2131: [WIP] Support unmanaged carbon table read and...

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

    https://github.com/apache/carbondata/pull/2131#discussion_r178739855
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableBuilder.java ---
    @@ -48,23 +48,39 @@ public CarbonTableBuilder tablePath(String tablePath) {
         return this;
       }
     
    +
    +  public CarbonTableBuilder isUnManagedTable(boolean isUnManagedTable) {
    +    Objects.requireNonNull(isUnManagedTable, "UnManaged Table should not be null");
    +    this.unManagedTable = isUnManagedTable;
    +    return this;
    +  }
    +
       public CarbonTableBuilder tableSchema(TableSchema tableSchema) {
         Objects.requireNonNull(tableSchema, "tableSchema should not be null");
         this.tableSchema = tableSchema;
         return this;
       }
     
    +  public CarbonTableBuilder setUUID(long uuid) {
    --- End diff --
    
    Removed


---

[GitHub] carbondata issue #2131: [CARBONDATA-2313] Support unmanaged carbon table rea...

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

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



---

[GitHub] carbondata pull request #2131: [CARBONDATA-2313] Support unmanaged carbon ta...

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

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


---

[GitHub] carbondata pull request #2131: [CARBONDATA-2313] Support unmanaged carbon ta...

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

    https://github.com/apache/carbondata/pull/2131#discussion_r179088904
  
    --- Diff: processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModel.java ---
    @@ -47,6 +47,13 @@
     
       private String tablePath;
     
    +  /*
    +     This points if the carbonTable is a Unmanaged Table or not.
    +     The path will be pointed by the tablePath. And there will be
    +     no Metadata folder present for the unmanaged Table.
    +   */
    +  private boolean carbonUnmanagedTable;
    --- End diff --
    
    already class name has carbon word, so not required to repeat, this again. better use "unmanagedTable"


---

[GitHub] carbondata issue #2131: [CARBONDATA-2313] Support unmanaged carbon table rea...

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

    https://github.com/apache/carbondata/pull/2131
  
    retest this please 


---

[GitHub] carbondata issue #2131: [WIP] Support unmanaged carbon table read and write

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

    https://github.com/apache/carbondata/pull/2131
  
    SDV Build Fail , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/4279/



---

[GitHub] carbondata issue #2131: [CARBONDATA-2313] Support unmanaged carbon table rea...

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

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



---

[GitHub] carbondata issue #2131: [WIP] Support unmanaged carbon table read and write

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

    https://github.com/apache/carbondata/pull/2131
  
    retest this please


---

[GitHub] carbondata issue #2131: [CARBONDATA-2313] Support unmanaged carbon table rea...

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

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



---

[GitHub] carbondata pull request #2131: [WIP] Support unmanaged carbon table read and...

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

    https://github.com/apache/carbondata/pull/2131#discussion_r178739606
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java ---
    @@ -83,28 +85,59 @@ public DataMapWriter createWriter(Segment segment, String writeDirectoryPath) {
       }
     
       @Override
    -  public List<CoarseGrainDataMap> getDataMaps(Segment segment) throws IOException {
    +  public List<CoarseGrainDataMap> getDataMaps(Segment segment, ReadCommitted readCommitted)
    +      throws IOException {
         List<TableBlockIndexUniqueIdentifier> tableBlockIndexUniqueIdentifiers =
    -        getTableBlockIndexUniqueIdentifiers(segment);
    +        getTableBlockIndexUniqueIdentifiers(segment, readCommitted);
         return cache.getAll(tableBlockIndexUniqueIdentifiers);
       }
     
    -  private List<TableBlockIndexUniqueIdentifier> getTableBlockIndexUniqueIdentifiers(
    -      Segment segment) throws IOException {
    +  private List<TableBlockIndexUniqueIdentifier> getTableBlockIndexUniqueIdentifiers(Segment segment,
    +      ReadCommitted readCommitted) throws IOException {
         List<TableBlockIndexUniqueIdentifier> tableBlockIndexUniqueIdentifiers =
             segmentMap.get(segment.getSegmentNo());
         if (tableBlockIndexUniqueIdentifiers == null) {
           tableBlockIndexUniqueIdentifiers = new ArrayList<>();
    +      // TODO: integrate with ReadCommitted
    +      //      ReadCommitted readCommitted;
    +      //      if (job.getConfiguration().get(CARBON_UNMANAGED_TABLE).equalsIgnoreCase("true")) {
    +      //        updateStatusManager = null;
    +      //        readCommitted = new LatestFilesReadCommitted(identifier.getTablePath());
    +      //      } else {
    +      //        loadMetadataDetails = SegmentStatusManager
    +      //         .readTableStatusFile(CarbonTablePath
    +      //          .getTableStatusFilePath(identifier.getTablePath()));
    +      //        updateStatusManager =
    +      //          new SegmentUpdateStatusManager(identifier, loadMetadataDetails);
    +      //        readCommitted =
    +      //          new TableStatusReadCommitted(job, this, loadMetadataDetails, updateStatusManager);
    +      //      }
    +      //            Map<String, String> indexFiles = readCommitted.getCommittedIndexList(segment);
           Map<String, String> indexFiles;
    -      if (segment.getSegmentFileName() == null) {
    -        String path =
    -            CarbonTablePath.getSegmentPath(identifier.getTablePath(), segment.getSegmentNo());
    -        indexFiles = new SegmentIndexFileStore().getIndexFilesFromSegment(path);
    +      if (CarbonUtil.isUnmanagedCarbonTable(identifier.getTablePath(), true)) {
    +        if (null != readCommitted) {
    +          indexFiles = readCommitted.getCommittedIndexMapSegments();
    +        } else {
    +          indexFiles =
    +              new SegmentIndexFileStore().getIndexFilesFromSegment(identifier.getTablePath());
    +        }
           } else {
    -        SegmentFileStore fileStore =
    -            new SegmentFileStore(identifier.getTablePath(), segment.getSegmentFileName());
    -        indexFiles = fileStore.getIndexFiles();
    +        if (segment.getSegmentFileName() == null) {
    +
    +          if (null != readCommitted) {
    +            indexFiles = readCommitted.getCommittedIndexMapPerSegment(segment);
    --- End diff --
    
    Done


---

[GitHub] carbondata issue #2131: [WIP] Support unmanaged carbon table read and write

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

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



---

[GitHub] carbondata pull request #2131: [CARBONDATA-2313] Support unmanaged carbon ta...

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

    https://github.com/apache/carbondata/pull/2131#discussion_r179085175
  
    --- Diff: store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriterBuilder.java ---
    @@ -152,15 +167,23 @@ private CarbonTable buildCarbonTable() {
               new StructField(field.getFieldName(), field.getDataType()),
               sortColumnsList.contains(field.getFieldName()));
         }
    -    String tableName = "_tempTable";
    -    String dbName = "_tempDB";
    +    String tableName;
    --- End diff --
    
    Not required to add tablename here, in this class, keep it same as previous


---

[GitHub] carbondata pull request #2131: [WIP] Support unmanaged carbon table read and...

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

    https://github.com/apache/carbondata/pull/2131#discussion_r178571705
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java ---
    @@ -79,26 +80,27 @@
        *
        * @param segments
        * @param filterExp
    +   * @param readCommitted
        * @return
        */
       public List<ExtendedBlocklet> prune(List<Segment> segments, FilterResolverIntf filterExp,
    -      List<PartitionSpec> partitions) throws IOException {
    +      List<PartitionSpec> partitions, ReadCommitted readCommitted) throws IOException {
    --- End diff --
    
    Can you explain what is ReadCommitted and why is it needed?


---

[GitHub] carbondata issue #2131: [WIP] Support unmanaged carbon table read and write

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

    https://github.com/apache/carbondata/pull/2131
  
    retest this please


---

[GitHub] carbondata pull request #2131: [WIP] Support unmanaged carbon table read and...

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

    https://github.com/apache/carbondata/pull/2131#discussion_r178505154
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableBuilder.java ---
    @@ -48,23 +48,39 @@ public CarbonTableBuilder tablePath(String tablePath) {
         return this;
       }
     
    +
    +  public CarbonTableBuilder isUnManagedTable(boolean isUnManagedTable) {
    +    Objects.requireNonNull(isUnManagedTable, "UnManaged Table should not be null");
    +    this.unManagedTable = isUnManagedTable;
    +    return this;
    +  }
    +
       public CarbonTableBuilder tableSchema(TableSchema tableSchema) {
         Objects.requireNonNull(tableSchema, "tableSchema should not be null");
         this.tableSchema = tableSchema;
         return this;
       }
     
    +  public CarbonTableBuilder setUUID(long uuid) {
    --- End diff --
    
    UID cannot be in Tablelevel datastructure, as it is unique for one load. Move it to LoadModel


---

[GitHub] carbondata issue #2131: [WIP] Support unmanaged carbon table read and write

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

    https://github.com/apache/carbondata/pull/2131
  
    retest this please


---

[GitHub] carbondata pull request #2131: [WIP] Support unmanaged carbon table read and...

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

    https://github.com/apache/carbondata/pull/2131#discussion_r178739522
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/readcommitter/LatestFilesReadCommitted.java ---
    @@ -0,0 +1,122 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.carbondata.core.readcommitter;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.HashMap;
    +import java.util.Map;
    +
    +import org.apache.carbondata.core.datamap.Segment;
    +import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
    +import org.apache.carbondata.core.indexstore.blockletindex.SegmentIndexFileStore;
    +import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
    +import org.apache.carbondata.core.statusmanager.SegmentStatus;
    +import org.apache.carbondata.core.util.path.CarbonTablePath;
    +
    +public class LatestFilesReadCommitted implements ReadCommitted {
    +
    +  private String carbonFilePath;
    +  private ReadCommittedIndexFileSnapShot readCommittedIndexFileSnapShot;
    +
    +  public LatestFilesReadCommitted(String path) {
    +    this.carbonFilePath = path;
    +    try {
    +      takeCarbonIndexFileSnapShot();
    +    } catch (IOException ex) {
    +      // TODO Put proper Log and throw the exception out.
    +      System.out.println("Error while reding index file");
    +    }
    +  }
    +
    +  @Override public LoadMetadataDetails[] getSegmentList() throws IOException {
    +    // Read the Segment path and form the LoadMetadataDetails array.
    +    File fs = new File(carbonFilePath);
    +
    +    if (fs.isDirectory()) {
    +
    +      CarbonFile[] carbonIndexFiles = SegmentIndexFileStore.getCarbonIndexFiles(carbonFilePath);
    +      LoadMetadataDetails[] loadMetadataDetailsArray =
    +          new LoadMetadataDetails[carbonIndexFiles.length];
    +      int loadCount = 0;
    +      for (int i = 0; i < carbonIndexFiles.length; i++) {
    --- End diff --
    
    Rectified.


---

[GitHub] carbondata issue #2131: [WIP] Support unmanaged carbon table read and write

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

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



---

[GitHub] carbondata pull request #2131: [WIP] Support unmanaged carbon table read and...

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

    https://github.com/apache/carbondata/pull/2131#discussion_r178518466
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java ---
    @@ -83,28 +85,59 @@ public DataMapWriter createWriter(Segment segment, String writeDirectoryPath) {
       }
     
       @Override
    -  public List<CoarseGrainDataMap> getDataMaps(Segment segment) throws IOException {
    +  public List<CoarseGrainDataMap> getDataMaps(Segment segment, ReadCommitted readCommitted)
    +      throws IOException {
         List<TableBlockIndexUniqueIdentifier> tableBlockIndexUniqueIdentifiers =
    -        getTableBlockIndexUniqueIdentifiers(segment);
    +        getTableBlockIndexUniqueIdentifiers(segment, readCommitted);
         return cache.getAll(tableBlockIndexUniqueIdentifiers);
       }
     
    -  private List<TableBlockIndexUniqueIdentifier> getTableBlockIndexUniqueIdentifiers(
    -      Segment segment) throws IOException {
    +  private List<TableBlockIndexUniqueIdentifier> getTableBlockIndexUniqueIdentifiers(Segment segment,
    +      ReadCommitted readCommitted) throws IOException {
         List<TableBlockIndexUniqueIdentifier> tableBlockIndexUniqueIdentifiers =
             segmentMap.get(segment.getSegmentNo());
         if (tableBlockIndexUniqueIdentifiers == null) {
           tableBlockIndexUniqueIdentifiers = new ArrayList<>();
    +      // TODO: integrate with ReadCommitted
    +      //      ReadCommitted readCommitted;
    +      //      if (job.getConfiguration().get(CARBON_UNMANAGED_TABLE).equalsIgnoreCase("true")) {
    +      //        updateStatusManager = null;
    +      //        readCommitted = new LatestFilesReadCommitted(identifier.getTablePath());
    +      //      } else {
    +      //        loadMetadataDetails = SegmentStatusManager
    +      //         .readTableStatusFile(CarbonTablePath
    +      //          .getTableStatusFilePath(identifier.getTablePath()));
    +      //        updateStatusManager =
    +      //          new SegmentUpdateStatusManager(identifier, loadMetadataDetails);
    +      //        readCommitted =
    +      //          new TableStatusReadCommitted(job, this, loadMetadataDetails, updateStatusManager);
    +      //      }
    +      //            Map<String, String> indexFiles = readCommitted.getCommittedIndexList(segment);
           Map<String, String> indexFiles;
    -      if (segment.getSegmentFileName() == null) {
    -        String path =
    -            CarbonTablePath.getSegmentPath(identifier.getTablePath(), segment.getSegmentNo());
    -        indexFiles = new SegmentIndexFileStore().getIndexFilesFromSegment(path);
    +      if (CarbonUtil.isUnmanagedCarbonTable(identifier.getTablePath(), true)) {
    +        if (null != readCommitted) {
    +          indexFiles = readCommitted.getCommittedIndexMapSegments();
    +        } else {
    +          indexFiles =
    +              new SegmentIndexFileStore().getIndexFilesFromSegment(identifier.getTablePath());
    +        }
           } else {
    -        SegmentFileStore fileStore =
    -            new SegmentFileStore(identifier.getTablePath(), segment.getSegmentFileName());
    -        indexFiles = fileStore.getIndexFiles();
    +        if (segment.getSegmentFileName() == null) {
    +
    +          if (null != readCommitted) {
    +            indexFiles = readCommitted.getCommittedIndexMapPerSegment(segment);
    --- End diff --
    
    This logic should be common across managed and unmanaged


---

[GitHub] carbondata issue #2131: [WIP] Support unmanaged carbon table read and write

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

    https://github.com/apache/carbondata/pull/2131
  
    SDV Build Fail , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/4277/



---

[GitHub] carbondata issue #2131: [WIP] Support unmanaged carbon table read and write

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

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



---

[GitHub] carbondata issue #2131: [CARBONDATA-2313] Support unmanaged carbon table rea...

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

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



---

[GitHub] carbondata issue #2131: [CARBONDATA-2313] Support unmanaged carbon table rea...

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

    https://github.com/apache/carbondata/pull/2131
  
    retest this please


---

[GitHub] carbondata issue #2131: [WIP] Support unmanaged carbon table read and write

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

    https://github.com/apache/carbondata/pull/2131
  
    SDV Build Fail , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/4248/



---

[GitHub] carbondata issue #2131: [CARBONDATA-2313] Support unmanaged carbon table rea...

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

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



---

[GitHub] carbondata issue #2131: [CARBONDATA-2313] Support unmanaged carbon table rea...

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

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



---

[GitHub] carbondata issue #2131: [WIP] Support unmanaged carbon table read and write

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

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



---

[GitHub] carbondata pull request #2131: [WIP] Support unmanaged carbon table read and...

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

    https://github.com/apache/carbondata/pull/2131#discussion_r178505075
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java ---
    @@ -143,6 +143,16 @@
     
       private boolean hasDataMapSchema;
     
    +  /**
    +   * The boolean field which points if the data written for UnManaged Table
    +   * or Managed Table. The difference between managed and unManaged table is
    +   * unManaged Table will not contain any Metadata folder and subsequently
    +   * no TableStatus or Schema files.
    +   */
    +  private boolean isUnManagedTable;
    +
    +  private long UUID;
    --- End diff --
    
    UID cannot be in Tablelevel datastructure, as it is unique for one load. Move it to LoadModel


---

[GitHub] carbondata issue #2131: [WIP] Support unmanaged carbon table read and write

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

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



---

[GitHub] carbondata issue #2131: [WIP] Support unmanaged carbon table read and write

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

    https://github.com/apache/carbondata/pull/2131
  
    retest this please


---

[GitHub] carbondata issue #2131: [CARBONDATA-2313] Support unmanaged carbon table rea...

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

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



---

[GitHub] carbondata pull request #2131: [WIP] Support unmanaged carbon table read and...

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

    https://github.com/apache/carbondata/pull/2131#discussion_r178787139
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java ---
    @@ -143,6 +143,16 @@
     
       private boolean hasDataMapSchema;
     
    +  /**
    +   * The boolean field which points if the data written for UnManaged Table
    +   * or Managed Table. The difference between managed and unManaged table is
    +   * unManaged Table will not contain any Metadata folder and subsequently
    +   * no TableStatus or Schema files.
    +   */
    +  private boolean isUnManagedTable;
    +
    +  private long UUID;
    --- End diff --
    
    removed


---

[GitHub] carbondata issue #2131: [WIP] Support unmanaged carbon table read and write

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

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



---

[GitHub] carbondata issue #2131: [CARBONDATA-2313] Support unmanaged carbon table rea...

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

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



---

[GitHub] carbondata pull request #2131: [WIP] Support unmanaged carbon table read and...

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

    https://github.com/apache/carbondata/pull/2131#discussion_r178788948
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java ---
    @@ -77,6 +77,19 @@
        */
       private String tablePath;
     
    +  /**
    +   * The boolean field which points if the data written for UnManaged Table
    +   * or Managed Table. The difference between managed and unManaged table is
    +   * unManaged Table will not contain any Metadata folder and subsequently
    +   * no TableStatus or Schema files.
    +   */
    +  private boolean isUnManagedTable;
    +
    +  /**
    +   * Unique ID
    +   */
    +  private long UUID;
    --- End diff --
    
    removed


---

[GitHub] carbondata pull request #2131: [WIP] Support unmanaged carbon table read and...

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

    https://github.com/apache/carbondata/pull/2131#discussion_r179098290
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/CarbonCountStar.scala ---
    @@ -76,6 +76,11 @@ case class CarbonCountStar(
         SparkHadoopUtil.get.addCredentials(jobConf)
         val job = new Job(jobConf)
         FileInputFormat.addInputPath(job, new Path(absoluteTableIdentifier.getTablePath))
    +    if (carbonTable.getTableInfo.isUnManagedTable) {
    +      CarbonInputFormat.setCarbonUnmanagedTable(job.getConfiguration, "true")
    --- End diff --
    
    Done


---

[GitHub] carbondata issue #2131: [CARBONDATA-2313] Support unmanaged carbon table rea...

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

    https://github.com/apache/carbondata/pull/2131
  
    retest this please


---

[GitHub] carbondata issue #2131: [WIP] Support unmanaged carbon table read and write

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

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



---

[GitHub] carbondata issue #2131: [WIP] Support unmanaged carbon table read and write

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

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



---

[GitHub] carbondata issue #2131: [WIP] Support unmanaged carbon table read and write

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

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



---

[GitHub] carbondata pull request #2131: [CARBONDATA-2313] Support unmanaged carbon ta...

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

    https://github.com/apache/carbondata/pull/2131#discussion_r179360469
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java ---
    @@ -2910,10 +2914,16 @@ private static long getMaxOfBlockAndFileSize(long blockSize, long fileSize) {
        * @return
        */
       public static String getBlockId(AbsoluteTableIdentifier identifier, String filePath,
    -      String segmentId) {
    +      String segmentId, boolean isUnmangedTable) {
         String blockId;
         String blockName = filePath.substring(filePath.lastIndexOf("/") + 1, filePath.length());
         String tablePath = identifier.getTablePath();
    +
    --- End diff --
    
    OK. Hanlded.


---

[GitHub] carbondata pull request #2131: [WIP] Support unmanaged carbon table read and...

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

    https://github.com/apache/carbondata/pull/2131#discussion_r178715161
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/readcommitter/LatestFilesReadCommitted.java ---
    @@ -0,0 +1,122 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.carbondata.core.readcommitter;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.HashMap;
    +import java.util.Map;
    +
    +import org.apache.carbondata.core.datamap.Segment;
    +import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
    +import org.apache.carbondata.core.indexstore.blockletindex.SegmentIndexFileStore;
    +import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
    +import org.apache.carbondata.core.statusmanager.SegmentStatus;
    +import org.apache.carbondata.core.util.path.CarbonTablePath;
    +
    +public class LatestFilesReadCommitted implements ReadCommitted {
    +
    +  private String carbonFilePath;
    +  private ReadCommittedIndexFileSnapShot readCommittedIndexFileSnapShot;
    +
    +  public LatestFilesReadCommitted(String path) {
    +    this.carbonFilePath = path;
    +    try {
    +      takeCarbonIndexFileSnapShot();
    +    } catch (IOException ex) {
    +      // TODO Put proper Log and throw the exception out.
    +      System.out.println("Error while reding index file");
    +    }
    +  }
    +
    +  @Override public LoadMetadataDetails[] getSegmentList() throws IOException {
    +    // Read the Segment path and form the LoadMetadataDetails array.
    +    File fs = new File(carbonFilePath);
    +
    +    if (fs.isDirectory()) {
    +
    +      CarbonFile[] carbonIndexFiles = SegmentIndexFileStore.getCarbonIndexFiles(carbonFilePath);
    +      LoadMetadataDetails[] loadMetadataDetailsArray =
    +          new LoadMetadataDetails[carbonIndexFiles.length];
    +      int loadCount = 0;
    +      for (int i = 0; i < carbonIndexFiles.length; i++) {
    --- End diff --
    
    There can be multiple indexfiles under same logical segment/transactionid


---

[GitHub] carbondata issue #2131: [WIP] Support unmanaged carbon table read and write

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

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



---

[GitHub] carbondata pull request #2131: [WIP] Support unmanaged carbon table read and...

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

    https://github.com/apache/carbondata/pull/2131#discussion_r178511046
  
    --- Diff: hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java ---
    @@ -224,22 +198,28 @@ protected CarbonTable getOrCreateCarbonTable(Configuration configuration) throws
     
         // do block filtering and get split
         List<InputSplit> splits =
    -        getSplits(job, filterInterface, filteredSegmentToAccess, matchedPartitions, partitionInfo,
    -            null, updateStatusManager);
    +         getSplits(job, filterInterface, filteredSegmentToAccess, matchedPartitions, partitionInfo,
    +            null, updateStatusManager, readCommitted);
    +
         // pass the invalid segment to task side in order to remove index entry in task side
    -    if (invalidSegments.size() > 0) {
    -      for (InputSplit split : splits) {
    -        ((org.apache.carbondata.hadoop.CarbonInputSplit) split).setInvalidSegments(invalidSegments);
    -        ((org.apache.carbondata.hadoop.CarbonInputSplit) split)
    -            .setInvalidTimestampRange(invalidTimestampsList);
    +    if (readCommitted instanceof TableStatusReadCommitted) {
    --- End diff --
    
    One inputformat will read table according to one readCommitted.  So make it member variable of inputformat


---

[GitHub] carbondata issue #2131: [CARBONDATA-2313] Support unmanaged carbon table rea...

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

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



---

[GitHub] carbondata issue #2131: [WIP] Support unmanaged carbon table read and write

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

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



---

[GitHub] carbondata issue #2131: [CARBONDATA-2313] Support unmanaged carbon table rea...

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

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



---

[GitHub] carbondata issue #2131: [WIP] Support unmanaged carbon table read and write

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

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



---

[GitHub] carbondata pull request #2131: [CARBONDATA-2313] Support unmanaged carbon ta...

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

    https://github.com/apache/carbondata/pull/2131#discussion_r179150325
  
    --- Diff: hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java ---
    @@ -128,12 +134,18 @@ protected CarbonTable getOrCreateCarbonTable(Configuration configuration) throws
       @Override
       public List<InputSplit> getSplits(JobContext job) throws IOException {
         AbsoluteTableIdentifier identifier = getAbsoluteTableIdentifier(job.getConfiguration());
    -    LoadMetadataDetails[] loadMetadataDetails = SegmentStatusManager
    -        .readTableStatusFile(CarbonTablePath.getTableStatusFilePath(identifier.getTablePath()));
    +
         CarbonTable carbonTable = getOrCreateCarbonTable(job.getConfiguration());
         if (null == carbonTable) {
           throw new IOException("Missing/Corrupt schema file for table.");
         }
    +    this.readCommittedScope = getReadCommitted(job, identifier);
    --- End diff --
    
    When empty data.. select should give empty records


---

[GitHub] carbondata issue #2131: [WIP] Support unmanaged carbon table read and write

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

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



---

[GitHub] carbondata issue #2131: [WIP] Support unmanaged carbon table read and write

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

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



---

[GitHub] carbondata pull request #2131: [WIP] Support unmanaged carbon table read and...

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

    https://github.com/apache/carbondata/pull/2131#discussion_r178517359
  
    --- Diff: hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonInputFormat.java ---
    @@ -159,6 +162,11 @@ public static void setTablePath(Configuration configuration, String tablePath) {
         configuration.set(FileInputFormat.INPUT_DIR, tablePath);
       }
     
    +  public static void setCarbonUnmanagedTable(Configuration configuration,
    --- End diff --
    
    use boolean to setvalue, not string


---

[GitHub] carbondata issue #2131: [WIP] Support unmanaged carbon table read and write

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

    https://github.com/apache/carbondata/pull/2131
  
    SDV Build Fail , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/4262/



---

[GitHub] carbondata issue #2131: [WIP] Support unmanaged carbon table read and write

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

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



---

[GitHub] carbondata issue #2131: [CARBONDATA-2313] Support unmanaged carbon table rea...

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

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



---

[GitHub] carbondata pull request #2131: [WIP] Support unmanaged carbon table read and...

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

    https://github.com/apache/carbondata/pull/2131#discussion_r178739559
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/SegmentIndexFileStore.java ---
    @@ -200,6 +216,90 @@ public void readAllIIndexOfSegment(CarbonFile[] carbonFiles) throws IOException
         return indexFiles;
       }
     
    +  /**
    +   * Read all index file names of the carbon File Path.
    +   *
    +   * @return
    +   * @throws IOException
    +   */
    +  public Map<String, String> getReadCommittedIndexFilesFromPath(String carbonFilePath)
    +      throws IOException {
    +    // Get only those index files which are mentioned in the
    +    CarbonFile[] carbonIndexFiles = getCarbonIndexFiles(carbonFilePath, carbonIndexMap);
    +    Map<String, String> indexFiles = new HashMap<>();
    +    for (int i = 0; i < carbonIndexFiles.length; i++) {
    +      if (carbonIndexFiles[i].getName().endsWith(CarbonTablePath.MERGE_INDEX_FILE_EXT)) {
    +        List<String> indexFilesFromMergeFile =
    +            getIndexFilesFromMergeFile(carbonIndexFiles[i].getCanonicalPath());
    +        for (String file : indexFilesFromMergeFile) {
    +          indexFiles.put(carbonIndexFiles[i].getParentFile().getAbsolutePath()
    +              + CarbonCommonConstants.FILE_SEPARATOR + file, carbonIndexFiles[i].getName());
    +        }
    +      } else if (carbonIndexFiles[i].getName().endsWith(CarbonTablePath.INDEX_FILE_EXT)) {
    +        indexFiles.put(carbonIndexFiles[i].getAbsolutePath(), null);
    +      }
    +    }
    +    return indexFiles;
    +  }
    +
    +  /**
    +   * Read all index file names of the segment
    +   *
    +   * @return
    +   * @throws IOException
    +   */
    +  public Map<String, String> getReadCommittedIndexFilesFromPath(String path, List<Segment> segments)
    +      throws IOException {
    +    // Only returns indexes matching the segment.
    +    Map<String, String> indexFiles = new HashMap<>();
    +    for (Segment seg : segments) {
    +      CarbonFile[] carbonIndexFiles = getCarbonIndexFiles(path, carbonIndexMap);
    +      for (int i = 0; i < carbonIndexFiles.length; i++) {
    --- End diff --
    
    Removed


---

[GitHub] carbondata pull request #2131: [CARBONDATA-2313] Support unmanaged carbon ta...

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

    https://github.com/apache/carbondata/pull/2131#discussion_r179148873
  
    --- Diff: processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModel.java ---
    @@ -47,6 +47,13 @@
     
       private String tablePath;
     
    +  /*
    +     This points if the carbonTable is a Unmanaged Table or not.
    +     The path will be pointed by the tablePath. And there will be
    +     no Metadata folder present for the unmanaged Table.
    +   */
    +  private boolean carbonUnmanagedTable;
    --- End diff --
    
    Done


---

[GitHub] carbondata issue #2131: [WIP] Support unmanaged carbon table read and write

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

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



---

[GitHub] carbondata pull request #2131: [WIP] Support unmanaged carbon table read and...

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

    https://github.com/apache/carbondata/pull/2131#discussion_r178505178
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java ---
    @@ -77,6 +77,19 @@
        */
       private String tablePath;
     
    +  /**
    +   * The boolean field which points if the data written for UnManaged Table
    +   * or Managed Table. The difference between managed and unManaged table is
    +   * unManaged Table will not contain any Metadata folder and subsequently
    +   * no TableStatus or Schema files.
    +   */
    +  private boolean isUnManagedTable;
    +
    +  /**
    +   * Unique ID
    +   */
    +  private long UUID;
    --- End diff --
    
    UID cannot be in Tablelevel datastructure, as it is unique for one load. Move it to LoadModel


---

[GitHub] carbondata issue #2131: [WIP] Support unmanaged carbon table read and write

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

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



---

[GitHub] carbondata issue #2131: [WIP] Support unmanaged carbon table read and write

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

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



---

[GitHub] carbondata pull request #2131: [CARBONDATA-2313] Support unmanaged carbon ta...

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

    https://github.com/apache/carbondata/pull/2131#discussion_r179154952
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java ---
    @@ -2910,10 +2914,16 @@ private static long getMaxOfBlockAndFileSize(long blockSize, long fileSize) {
        * @return
        */
       public static String getBlockId(AbsoluteTableIdentifier identifier, String filePath,
    -      String segmentId) {
    +      String segmentId, boolean isUnmangedTable) {
         String blockId;
         String blockName = filePath.substring(filePath.lastIndexOf("/") + 1, filePath.length());
         String tablePath = identifier.getTablePath();
    +
    --- End diff --
    
    this block id should be commonly constructed for unmanaged and managed case. As unmanged partition is also possible


---

[GitHub] carbondata issue #2131: [WIP] Support unmanaged carbon table read and write

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

    https://github.com/apache/carbondata/pull/2131
  
    Retest this please


---

[GitHub] carbondata issue #2131: [CARBONDATA-2313] Support unmanaged carbon table rea...

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

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



---

[GitHub] carbondata issue #2131: [WIP] Support unmanaged carbon table read and write

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

    https://github.com/apache/carbondata/pull/2131
  
    SDV Build Fail , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/4272/



---

[GitHub] carbondata issue #2131: [WIP] Support unmanaged carbon table read and write

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

    https://github.com/apache/carbondata/pull/2131
  
    SDV Build Fail , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/4250/



---

[GitHub] carbondata issue #2131: [WIP] Support unmanaged carbon table read and write

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

    https://github.com/apache/carbondata/pull/2131
  
    retest this please


---

[GitHub] carbondata issue #2131: [CARBONDATA-2313] Support unmanaged carbon table rea...

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

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



---