You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@carbondata.apache.org by QiangCai <gi...@git.apache.org> on 2017/04/02 19:36:22 UTC

[GitHub] incubator-carbondata pull request #722: [CARBONDATA-782]support unsorted tab...

GitHub user QiangCai opened a pull request:

    https://github.com/apache/incubator-carbondata/pull/722

    [CARBONDATA-782]support unsorted table (when SORT_COLUMNS is empty)

    When SORT_COLUMNS of a table is empty or the table doesn't have dimensions(besides complex type column),  this table is a unsorted table.
    
    1. Support unsorted table creation.
    e.g. tblproperties('sort_columns'='')  sort_columns is empty, this table will be unsorted table.
    
    2. During data loading, unsorted table skip sort step.
    
    3. Support query(filter and non-filter) on unsorted table.
    
    4. Use SORT_COLUMNS to create index's KeyGenerator instead of all dimensions.  
    


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

    $ git pull https://github.com/QiangCai/incubator-carbondata nosorttables

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

    https://github.com/apache/incubator-carbondata/pull/722.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 #722
    
----
commit 5fce5c8a2f88919e1f719fa7b549395b2a1f6e4f
Author: QiangCai <qi...@qq.com>
Date:   2017-04-02T19:12:55Z

    no_sort_columns

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata issue #722: [CARBONDATA-782]support unsorted table (whe...

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

    https://github.com/apache/incubator-carbondata/pull/722
  
    Build Success with Spark 1.6.2, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/1421/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata issue #722: [CARBONDATA-782]support unsorted table (whe...

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

    https://github.com/apache/incubator-carbondata/pull/722
  
    LGTM


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata pull request #722: [CARBONDATA-782]support unsorted tab...

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

    https://github.com/apache/incubator-carbondata/pull/722#discussion_r109815428
  
    --- Diff: integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala ---
    @@ -490,8 +490,20 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
         // All columns in sortkey should be there in create table cols
         val sortKeyOption = tableProperties.get(CarbonCommonConstants.SORT_COLUMNS)
         var sortKeyDimsTmp: Seq[String] = Seq[String]()
    -    if (sortKeyOption.isDefined) {
    -      var sortKey = sortKeyOption.get.split(',').map(_.trim)
    +    val sortKeyString: String = if (sortKeyOption.isDefined) {
    --- End diff --
    
    can you extract this logic (getting value from ' ' or " "), is other options doing the same? maybe it can be shared


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata pull request #722: [CARBONDATA-782]support unsorted tab...

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

    https://github.com/apache/incubator-carbondata/pull/722#discussion_r109867883
  
    --- Diff: processing/src/main/java/org/apache/carbondata/processing/newflow/steps/NoSortProcessorStepImpl.java ---
    @@ -0,0 +1,123 @@
    +/*
    + * 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.newflow.steps;
    +
    +import java.io.IOException;
    +import java.math.BigDecimal;
    +
    +import org.apache.carbondata.core.constants.CarbonCommonConstants;
    +import org.apache.carbondata.core.util.DataTypeUtil;
    +import org.apache.carbondata.processing.newflow.AbstractDataLoadProcessorStep;
    +import org.apache.carbondata.processing.newflow.CarbonDataLoadConfiguration;
    +import org.apache.carbondata.processing.newflow.DataField;
    +import org.apache.carbondata.processing.newflow.row.CarbonRow;
    +import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
    +import org.apache.carbondata.processing.util.NonDictionaryUtil;
    +
    +/**
    + * if the table doesn't have sort_columns, just convert row format.
    + */
    +public class NoSortProcessorStepImpl extends AbstractDataLoadProcessorStep {
    +
    +  private int dimensionCount;
    +
    +  private int dimensionWithComplexCount;
    +
    +  private int noDictCount;
    +
    +  private int measureCount;
    +
    +  private boolean[] isNoDictionaryDimensionColumn;
    +
    +  private char[] aggType;
    +
    +  public NoSortProcessorStepImpl(CarbonDataLoadConfiguration configuration,
    +      AbstractDataLoadProcessorStep child) {
    +    super(configuration, child);
    +    this.dimensionWithComplexCount = configuration.getDimensionCount();
    +    this.noDictCount =
    +        configuration.getNoDictionaryCount() + configuration.getComplexDimensionCount();
    +    this.dimensionCount = configuration.getDimensionCount() - this.noDictCount;
    +    this.measureCount = configuration.getMeasureCount();
    +    this.isNoDictionaryDimensionColumn =
    +        CarbonDataProcessorUtil.getNoDictionaryMapping(configuration.getDataFields());
    +    this.aggType = CarbonDataProcessorUtil
    +        .getAggType(configuration.getMeasureCount(), configuration.getMeasureFields());
    +  }
    +
    +  @Override public DataField[] getOutput() {
    +    return child.getOutput();
    +  }
    +
    +  @Override public void initialize() throws IOException {
    +    child.initialize();
    +  }
    +
    +  @Override protected CarbonRow processRow(CarbonRow row) {
    --- End diff --
    
    fixed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata pull request #722: [CARBONDATA-782]support unsorted tab...

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

    https://github.com/apache/incubator-carbondata/pull/722#discussion_r109815295
  
    --- Diff: integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumns.scala ---
    @@ -154,6 +154,79 @@ class TestSortColumns extends QueryTest with BeforeAndAfterAll {
         checkAnswer(sql("select * from sorttable6 where empname = 'madhan'"), sql("select * from origintable1 where empname = 'madhan'"))
       }
     
    +  test("no sort_columns with and data loading with heap and safe sort config") {
    +    try {
    +      setLoadingProperties("false", "false", "false")
    +      sql("CREATE TABLE sorttable7_heap_safe (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('sort_columns'='')")
    +      sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE sorttable7_heap_safe OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""")
    +      checkAnswer(sql("select * from sorttable7_heap_safe where empno = 11"), sql("select * from origintable1 where empno = 11"))
    +      checkAnswer(sql("select * from sorttable7_heap_safe order by empno"), sql("select * from origintable1 order by empno"))
    +    } finally {
    +      defaultLoadingProperties
    +    }
    +  }
    +
    +  test("no sort_columns with and data loading with heap and unsafe sort config") {
    --- End diff --
    
    please change the test title for all newly added testcase


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata pull request #722: [CARBONDATA-782]support unsorted tab...

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

    https://github.com/apache/incubator-carbondata/pull/722#discussion_r109815081
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datastore/block/SegmentProperties.java ---
    @@ -54,6 +54,8 @@
        */
       private KeyGenerator dimensionKeyGenerator;
     
    +  private KeyGenerator sortColumnsGenerator;
    --- End diff --
    
    please add comment for this member variable and `dimensionKeyGenerator`, to describe their difference


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata pull request #722: [CARBONDATA-782]support unsorted tab...

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

    https://github.com/apache/incubator-carbondata/pull/722#discussion_r109867776
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datastore/block/SegmentProperties.java ---
    @@ -54,6 +54,8 @@
        */
       private KeyGenerator dimensionKeyGenerator;
     
    +  private KeyGenerator sortColumnsGenerator;
    --- End diff --
    
    fixed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata issue #722: [CARBONDATA-782]support unsorted table (whe...

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

    https://github.com/apache/incubator-carbondata/pull/722
  
    Build Success with Spark 1.6.2, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/1405/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata issue #722: [CARBONDATA-782]support unsorted table (whe...

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

    https://github.com/apache/incubator-carbondata/pull/722
  
    Build Failed  with Spark 1.6.2, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/1403/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata issue #722: [CARBONDATA-782]support unsorted table (whe...

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

    https://github.com/apache/incubator-carbondata/pull/722
  
    @jackylk
     I have fixed all comments,  please review again.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata pull request #722: [CARBONDATA-782]support unsorted tab...

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

    https://github.com/apache/incubator-carbondata/pull/722#discussion_r109867805
  
    --- Diff: integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumns.scala ---
    @@ -154,6 +154,79 @@ class TestSortColumns extends QueryTest with BeforeAndAfterAll {
         checkAnswer(sql("select * from sorttable6 where empname = 'madhan'"), sql("select * from origintable1 where empname = 'madhan'"))
       }
     
    +  test("no sort_columns with and data loading with heap and safe sort config") {
    +    try {
    +      setLoadingProperties("false", "false", "false")
    +      sql("CREATE TABLE sorttable7_heap_safe (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('sort_columns'='')")
    +      sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE sorttable7_heap_safe OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""")
    +      checkAnswer(sql("select * from sorttable7_heap_safe where empno = 11"), sql("select * from origintable1 where empno = 11"))
    +      checkAnswer(sql("select * from sorttable7_heap_safe order by empno"), sql("select * from origintable1 order by empno"))
    +    } finally {
    +      defaultLoadingProperties
    +    }
    +  }
    +
    +  test("no sort_columns with and data loading with heap and unsafe sort config") {
    --- End diff --
    
    fixed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata issue #722: [CARBONDATA-782]support unsorted table (whe...

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

    https://github.com/apache/incubator-carbondata/pull/722
  
    Build Failed  with Spark 1.6.2, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/1404/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata pull request #722: [CARBONDATA-782]support unsorted tab...

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

    https://github.com/apache/incubator-carbondata/pull/722#discussion_r109867836
  
    --- Diff: integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala ---
    @@ -490,8 +490,20 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
         // All columns in sortkey should be there in create table cols
         val sortKeyOption = tableProperties.get(CarbonCommonConstants.SORT_COLUMNS)
         var sortKeyDimsTmp: Seq[String] = Seq[String]()
    -    if (sortKeyOption.isDefined) {
    -      var sortKey = sortKeyOption.get.split(',').map(_.trim)
    +    val sortKeyString: String = if (sortKeyOption.isDefined) {
    --- End diff --
    
    fixed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata pull request #722: [CARBONDATA-782]support unsorted tab...

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

    https://github.com/apache/incubator-carbondata/pull/722#discussion_r109867759
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BTreeDataRefNodeFinder.java ---
    @@ -51,16 +51,15 @@
       /**
        * this will be used during search for no dictionary column
        */
    -  private int numberOfNoDictionaryColumns;
    +  private int numberOfNoDictSortColumns;
    --- End diff --
    
    fixed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata pull request #722: [CARBONDATA-782]support unsorted tab...

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

    https://github.com/apache/incubator-carbondata/pull/722#discussion_r109814360
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BTreeDataRefNodeFinder.java ---
    @@ -51,16 +51,15 @@
       /**
        * this will be used during search for no dictionary column
        */
    -  private int numberOfNoDictionaryColumns;
    +  private int numberOfNoDictSortColumns;
    --- End diff --
    
    can you change the comment for this member also. explain what this stored in this member


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata issue #722: [CARBONDATA-782]support unsorted table (whe...

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

    https://github.com/apache/incubator-carbondata/pull/722
  
    Build Success with Spark 1.6.2, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/1431/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata pull request #722: [CARBONDATA-782]support unsorted tab...

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

    https://github.com/apache/incubator-carbondata/pull/722#discussion_r109815736
  
    --- Diff: processing/src/main/java/org/apache/carbondata/processing/newflow/steps/NoSortProcessorStepImpl.java ---
    @@ -0,0 +1,123 @@
    +/*
    + * 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.newflow.steps;
    +
    +import java.io.IOException;
    +import java.math.BigDecimal;
    +
    +import org.apache.carbondata.core.constants.CarbonCommonConstants;
    +import org.apache.carbondata.core.util.DataTypeUtil;
    +import org.apache.carbondata.processing.newflow.AbstractDataLoadProcessorStep;
    +import org.apache.carbondata.processing.newflow.CarbonDataLoadConfiguration;
    +import org.apache.carbondata.processing.newflow.DataField;
    +import org.apache.carbondata.processing.newflow.row.CarbonRow;
    +import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
    +import org.apache.carbondata.processing.util.NonDictionaryUtil;
    +
    +/**
    + * if the table doesn't have sort_columns, just convert row format.
    + */
    +public class NoSortProcessorStepImpl extends AbstractDataLoadProcessorStep {
    +
    +  private int dimensionCount;
    +
    +  private int dimensionWithComplexCount;
    +
    +  private int noDictCount;
    +
    +  private int measureCount;
    +
    +  private boolean[] isNoDictionaryDimensionColumn;
    +
    +  private char[] aggType;
    +
    +  public NoSortProcessorStepImpl(CarbonDataLoadConfiguration configuration,
    +      AbstractDataLoadProcessorStep child) {
    +    super(configuration, child);
    +    this.dimensionWithComplexCount = configuration.getDimensionCount();
    +    this.noDictCount =
    +        configuration.getNoDictionaryCount() + configuration.getComplexDimensionCount();
    +    this.dimensionCount = configuration.getDimensionCount() - this.noDictCount;
    +    this.measureCount = configuration.getMeasureCount();
    +    this.isNoDictionaryDimensionColumn =
    +        CarbonDataProcessorUtil.getNoDictionaryMapping(configuration.getDataFields());
    +    this.aggType = CarbonDataProcessorUtil
    +        .getAggType(configuration.getMeasureCount(), configuration.getMeasureFields());
    +  }
    +
    +  @Override public DataField[] getOutput() {
    +    return child.getOutput();
    +  }
    +
    +  @Override public void initialize() throws IOException {
    +    child.initialize();
    +  }
    +
    +  @Override protected CarbonRow processRow(CarbonRow row) {
    --- End diff --
    
    add more comment here to explain what processing is done in this function. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata pull request #722: [CARBONDATA-782]support unsorted tab...

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

    https://github.com/apache/incubator-carbondata/pull/722


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata issue #722: [CARBONDATA-782]support unsorted table (whe...

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

    https://github.com/apache/incubator-carbondata/pull/722
  
    Build Success with Spark 1.6.2, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/1430/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---