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

[GitHub] carbondata pull request #1953: [CARBONDATA-2091][DataLoad] Support specifyin...

GitHub user xuchuanyin opened a pull request:

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

    [CARBONDATA-2091][DataLoad] Support specifying sort column bounds in data loading

    Enhance data loading performance by specifying sort column bounds
    1. Add row range number during convert-process-step
    2. Dispatch rows to each sorter by range number
    3. Sort/Write process step can be done concurrently in each range
    
    Tests added and docs updated
    
    After implementing this feature, the data load performance has gained about 25% enhancement (80MB/s/Node -> 102MB/s/Node) in my scenario with only 1 bounds provided. 
    
    Be sure to do all of the following checklist to help us incorporate 
    your contribution quickly and easily:
    
     - [x] Any interfaces changed?
     `Only internal used interfaces are changed`
     - [x] Any backward compatibility impacted?
     `No`
     - [x] Document update required?
    `Yes, added the usage of this feature to documents`
     - [x] Testing done
            Please provide details on 
            - Whether new unit test cases have been added or why no new tests are required?
    `Yes`
            - How it is tested? Please attach test report.
    `Tested in 3-node cluster and local machine`
            - Is it a performance related change? Please attach the performance test report.
    `Yes. After implementing this feature, the data load performance has gained about 25% enhancement (80MB/s/Node -> 102MB/s/Node) in my scenario with only 1 bounds provided. `
            - Any additional information to help reviewers in testing this change.
           `I refactored the bucket related feature and treated the range and bucket as the similar logic`
     - [x] For large changes, please consider breaking it into sub-tasks under an umbrella JIRA. 
    `Not related`


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

    $ git pull https://github.com/xuchuanyin/carbondata 0208_support_specifying_sort_column_bounds

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

    https://github.com/apache/carbondata/pull/1953.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 #1953
    
----
commit 11463dd22db17f2e1858e0a1f3ebfeb07e3ec0e9
Author: xuchuanyin <xu...@...>
Date:   2018-02-08T08:30:09Z

    Support specifying sort column bounds in data loading
    
    Enhance data loading performance by specifying sort column bounds
    1. Add row range number during convert-process-step
    2. Dispatch rows to each sorter by range number
    3. Sort/Write process step can be done concurrently in each range
    
    Tests added and docs updated

----


---

[GitHub] carbondata issue #1953: [CARBONDATA-2091][DataLoad] Support specifying sort ...

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

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



---

[GitHub] carbondata issue #1953: [CARBONDATA-2091][DataLoad] Support specifying sort ...

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

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



---

[GitHub] carbondata issue #1953: [CARBONDATA-2091][DataLoad] Support specifying sort ...

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

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



---

[GitHub] carbondata issue #1953: [CARBONDATA-2091][DataLoad] Support specifying sort ...

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

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



---

[GitHub] carbondata issue #1953: [CARBONDATA-2091][DataLoad] Support specifying sort ...

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

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



---

[GitHub] carbondata issue #1953: [CARBONDATA-2091][DataLoad] Support specifying sort ...

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

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



---

[GitHub] carbondata issue #1953: [CARBONDATA-2091][DataLoad] Support specifying sort ...

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

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


---

[GitHub] carbondata issue #1953: [CARBONDATA-2091][DataLoad] Support specifying sort ...

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

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



---

[GitHub] carbondata issue #1953: [CARBONDATA-2091][DataLoad] Support specifying sort ...

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

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



---

[GitHub] carbondata issue #1953: [CARBONDATA-2091][DataLoad] Support specifying sort ...

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

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



---

[GitHub] carbondata issue #1953: [CARBONDATA-2091][DataLoad] Support specifying sort ...

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

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



---

[GitHub] carbondata issue #1953: [CARBONDATA-2091][DataLoad] Support specifying sort ...

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

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



---

[GitHub] carbondata issue #1953: [CARBONDATA-2091][DataLoad] Support specifying sort ...

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

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



---

[GitHub] carbondata issue #1953: [CARBONDATA-2091][DataLoad] Support specifying sort ...

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

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



---

[GitHub] carbondata issue #1953: [CARBONDATA-2091][DataLoad] Support specifying sort ...

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

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



---

[GitHub] carbondata issue #1953: [CARBONDATA-2091][DataLoad] Support specifying sort ...

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

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



---

[GitHub] carbondata issue #1953: [CARBONDATA-2091][DataLoad] Support specifying sort ...

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

    https://github.com/apache/carbondata/pull/1953
  
    this PR depends on #1952 


---

[GitHub] carbondata issue #1953: [CARBONDATA-2091][DataLoad] Support specifying sort ...

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

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



---

[GitHub] carbondata issue #1953: [CARBONDATA-2091][DataLoad] Support specifying sort ...

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

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



---

[GitHub] carbondata pull request #1953: [CARBONDATA-2091][DataLoad] Support specifyin...

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

    https://github.com/apache/carbondata/pull/1953#discussion_r170003522
  
    --- Diff: processing/src/main/java/org/apache/carbondata/processing/loading/DataLoadProcessBuilder.java ---
    @@ -231,4 +238,71 @@ public static CarbonDataLoadConfiguration createConfiguration(CarbonLoadModel lo
         return configuration;
       }
     
    +  /**
    +   * set sort column info in configuration
    +   * @param carbonTable carbon table
    +   * @param loadModel load model
    +   * @param configuration configuration
    +   */
    +  private static void setSortColumnInfo(CarbonTable carbonTable, CarbonLoadModel loadModel,
    +      CarbonDataLoadConfiguration configuration) {
    +    List<String> sortCols = carbonTable.getSortColumns(carbonTable.getTableName());
    +    SortScopeOptions.SortScope sortScope = SortScopeOptions.getSortScope(loadModel.getSortScope());
    +    if (!SortScopeOptions.SortScope.LOCAL_SORT.equals(sortScope)
    +        || sortCols.size() == 0
    +        || StringUtils.isBlank(loadModel.getSortColumnsBoundsStr())) {
    +      if (!StringUtils.isBlank(loadModel.getSortColumnsBoundsStr())) {
    +        LOGGER.warn("sort column bounds will be ignored");
    +      }
    +
    +      configuration.setSortColumnRangeInfo(null);
    +      return;
    +    }
    +    // column index for sort columns
    +    int[] sortColIndex = new int[sortCols.size()];
    +    boolean[] isSortColNoDict = new boolean[sortCols.size()];
    +
    +    DataField[] outFields = configuration.getDataFields();
    +    int j = 0;
    +    boolean columnExist;
    +    for (String sortCol : sortCols) {
    +      columnExist = false;
    +
    +      for (int i = 0; !columnExist && i < outFields.length; i++) {
    +        if (outFields[i].getColumn().getColName().equalsIgnoreCase(sortCol)) {
    +          columnExist = true;
    +
    +          sortColIndex[j] = i;
    +          isSortColNoDict[j] = !outFields[i].hasDictionaryEncoding();
    +          j++;
    +        }
    +      }
    +
    +      if (!columnExist) {
    +        throw new RuntimeException("Field " + sortCol + " does not exist.");
    --- End diff --
    
    It  is better to use DataLoadingException


---

[GitHub] carbondata issue #1953: [CARBONDATA-2091][DataLoad] Support specifying sort ...

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

    https://github.com/apache/carbondata/pull/1953
  
    merged into carbonstore branch


---

[GitHub] carbondata issue #1953: [CARBONDATA-2091][DataLoad] Support specifying sort ...

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

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


---

[GitHub] carbondata issue #1953: [CARBONDATA-2091][DataLoad] Support specifying sort ...

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

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



---

[GitHub] carbondata issue #1953: [CARBONDATA-2091][DataLoad] Support specifying sort ...

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

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



---

[GitHub] carbondata issue #1953: [CARBONDATA-2091][DataLoad] Support specifying sort ...

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

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



---

[GitHub] carbondata issue #1953: [CARBONDATA-2091][DataLoad] Support specifying sort ...

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

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


---

[GitHub] carbondata issue #1953: [CARBONDATA-2091][DataLoad] Support specifying sort ...

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

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



---

[GitHub] carbondata issue #1953: [CARBONDATA-2091][DataLoad] Support specifying sort ...

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

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



---

[GitHub] carbondata issue #1953: [CARBONDATA-2091][DataLoad] Support specifying sort ...

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

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



---

[GitHub] carbondata pull request #1953: [CARBONDATA-2091][DataLoad] Support specifyin...

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

    https://github.com/apache/carbondata/pull/1953#discussion_r169990579
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/metadata/schema/ColumnRangeInfo.java ---
    @@ -0,0 +1,26 @@
    +/*
    + * 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.metadata.schema;
    +
    +/**
    + * interface for column range information. Currently we treat bucket and sort_column_range as
    + * column ranges.
    + */
    +public interface ColumnRangeInfo {
    +  int getNumOfRanges();
    +}
    --- End diff --
    
    add one new line at the end of file, otherwise it breaks code style


---

[GitHub] carbondata issue #1953: [CARBONDATA-2091][DataLoad] Support specifying sort ...

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

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



---

[GitHub] carbondata issue #1953: [CARBONDATA-2091][DataLoad] Support specifying sort ...

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

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


---

[GitHub] carbondata issue #1953: [CARBONDATA-2091][DataLoad] Support specifying sort ...

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

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



---

[GitHub] carbondata pull request #1953: [CARBONDATA-2091][DataLoad] Support specifyin...

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

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


---

[GitHub] carbondata issue #1953: [CARBONDATA-2091][DataLoad] Support specifying sort ...

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

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



---

[GitHub] carbondata issue #1953: [CARBONDATA-2091][DataLoad] Support specifying sort ...

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

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



---

[GitHub] carbondata pull request #1953: [CARBONDATA-2091][DataLoad] Support specifyin...

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

    https://github.com/apache/carbondata/pull/1953#discussion_r169989543
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/metadata/schema/BucketingInfo.java ---
    @@ -30,34 +30,32 @@
     /**
      * Bucketing information
      */
    -public class BucketingInfo implements Serializable, Writable {
    -
    +public class BucketingInfo implements ColumnRangeInfo, Serializable, Writable {
       private static final long serialVersionUID = -0L;
    -
       private List<ColumnSchema> listOfColumns;
    -
    -  private int numberOfBuckets;
    +  // number of column ranges
    --- End diff --
    
    Why is it called "column ranges"? Isn't it "value ranges"?


---

[GitHub] carbondata issue #1953: [CARBONDATA-2091][DataLoad] Support specifying sort ...

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

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



---

[GitHub] carbondata issue #1953: [CARBONDATA-2091][DataLoad] Support specifying sort ...

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

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



---

[GitHub] carbondata issue #1953: [CARBONDATA-2091][DataLoad] Support specifying sort ...

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

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



---

[GitHub] carbondata issue #1953: [CARBONDATA-2091][DataLoad] Support specifying sort ...

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

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



---

[GitHub] carbondata issue #1953: [CARBONDATA-2091][DataLoad] Support specifying sort ...

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

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



---

[GitHub] carbondata pull request #1953: [CARBONDATA-2091][DataLoad] Support specifyin...

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

    https://github.com/apache/carbondata/pull/1953#discussion_r170003740
  
    --- Diff: processing/src/main/java/org/apache/carbondata/processing/loading/partition/impl/RangePartitionerImpl.java ---
    @@ -0,0 +1,68 @@
    +/*
    + * 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.processing.loading.partition.impl;
    +
    +import java.util.Arrays;
    +import java.util.Comparator;
    +
    +import org.apache.carbondata.common.logging.LogService;
    +import org.apache.carbondata.common.logging.LogServiceFactory;
    +import org.apache.carbondata.core.datastore.row.CarbonRow;
    +import org.apache.carbondata.processing.loading.partition.Partitioner;
    +
    +public class RangePartitionerImpl implements Partitioner<CarbonRow> {
    +  private static final LogService LOGGER =
    +      LogServiceFactory.getLogService(RangePartitionerImpl.class.getName());
    +  private CarbonRow[] rangeBounds;
    +  private Comparator<CarbonRow> comparator;
    +
    +  public RangePartitionerImpl(CarbonRow[] rangeBounds, Comparator<CarbonRow> comparator) {
    +    this.rangeBounds = rangeBounds;
    +    LOGGER.info("Use range partitioner to distribute data to "
    +        + (rangeBounds.length + 1) + " ranges.");
    +    this.comparator = comparator;
    +  }
    +
    +  /**
    +   * learned from spark org.apache.spark.RangePartitioner
    +   *
    +   * @param key key
    +   * @return partitionId
    +   */
    +  @Override public int getPartition(CarbonRow key) {
    --- End diff --
    
    put all @Override to previous line


---

[GitHub] carbondata issue #1953: [CARBONDATA-2091][DataLoad] Support specifying sort ...

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

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



---

[GitHub] carbondata issue #1953: [CARBONDATA-2091][DataLoad] Support specifying sort ...

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

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



---

[GitHub] carbondata issue #1953: [CARBONDATA-2091][DataLoad] Support specifying sort ...

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

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



---

[GitHub] carbondata issue #1953: [CARBONDATA-2091][DataLoad] Support specifying sort ...

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

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


---

[GitHub] carbondata issue #1953: [CARBONDATA-2091][DataLoad] Support specifying sort ...

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

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



---

[GitHub] carbondata pull request #1953: [CARBONDATA-2091][DataLoad] Support specifyin...

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

    https://github.com/apache/carbondata/pull/1953#discussion_r170002493
  
    --- Diff: docs/data-management-on-carbondata.md ---
    @@ -370,6 +370,17 @@ This tutorial is going to introduce all commands and data operations on CarbonDa
         ```
         NOTE: Date formats are specified by date pattern strings. The date pattern letters in CarbonData are same as in JAVA. Refer to [SimpleDateFormat](http://docs.oracle.com/javase/7/docs/api/java/text/SimpleDateFormat.html).
     
    +  - **SORT COLUMN BOUNDS:** Range bounds for sort columns.
    +
    +    ```
    +    OPTIONS('SORT_COLUMN_BOUNDS'='v11,v21,v31;v12,v22,v32;v12,v23,v33')
    --- End diff --
    
    typo, last value range is `v13,v23,v33`


---

[GitHub] carbondata pull request #1953: [CARBONDATA-2091][DataLoad] Support specifyin...

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

    https://github.com/apache/carbondata/pull/1953#discussion_r170003203
  
    --- Diff: processing/src/main/java/org/apache/carbondata/processing/loading/CarbonDataLoadConfiguration.java ---
    @@ -107,6 +108,7 @@
        */
       private short writingCoresCount;
     
    +  private SortColumnRangeInfo sortColumnRangeInfo;
       public CarbonDataLoadConfiguration() {
    --- End diff --
    
    add one empty line


---

[GitHub] carbondata pull request #1953: [CARBONDATA-2091][DataLoad] Support specifyin...

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

    https://github.com/apache/carbondata/pull/1953#discussion_r170003007
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/metadata/schema/SortColumnRangeInfo.java ---
    @@ -0,0 +1,78 @@
    +/*
    + * 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.metadata.schema;
    +
    +import java.io.Serializable;
    +import java.util.Arrays;
    +
    +/**
    + * column ranges specified by sort column bounds
    + */
    +public class SortColumnRangeInfo implements ColumnRangeInfo, Serializable {
    --- End diff --
    
    For all public class, please annotate with `@InterfaceAudience`, in this PR, all newly added public class should be `@InterfaceAudience.Internal`


---