You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@carbondata.apache.org by sounakr <gi...@git.apache.org> on 2018/05/03 03:54:22 UTC

[GitHub] carbondata pull request #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns...

GitHub user sounakr opened a pull request:

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

    [CARBONDATA-2430][SDK] Reshuffling of Columns given by user in SDK.

    Reshuffling of Columns given by the user in SDK. Order should be Sort COlumns -> Dimension -> Complex --> Measure
    
     - [ ] 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/sounakr/incubator-carbondata complex-type

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

    https://github.com/apache/carbondata/pull/2261.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 #2261
    
----
commit 8c72187531077dc0c22e9b2c9f12091b8ae8e71d
Author: sounakr <so...@...>
Date:   2018-05-02T15:59:57Z

    Reshuffling of Columns

commit e455f275c577a3bbaa9bb0bdc4b203fd6b7448da
Author: sounakr <so...@...>
Date:   2018-05-03T02:47:38Z

    Measure Fields along with Complex DataType in AVRO Test Case. Reshuffling of fields.

----


---

[GitHub] carbondata pull request #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns...

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

    https://github.com/apache/carbondata/pull/2261#discussion_r186058667
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableSchemaBuilder.java ---
    @@ -169,11 +186,11 @@ private void checkRepeatColumnName(StructField field) {
             throw new IllegalArgumentException("column name already exists");
           }
         }
    -    for (ColumnSchema column : otherColumns) {
    -      if (column.getColumnName().equalsIgnoreCase(field.getFieldName())) {
    -        throw new IllegalArgumentException("column name already exists");
    -      }
    -    }
    +//    for (ColumnSchema column : otherColumns) {
    --- End diff --
    
    done


---

[GitHub] carbondata issue #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns given ...

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

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



---

[GitHub] carbondata pull request #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns...

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

    https://github.com/apache/carbondata/pull/2261#discussion_r186055265
  
    --- Diff: integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestNonTransactionalCarbonTable.scala ---
    @@ -750,7 +750,93 @@ class TestNonTransactionalCarbonTable extends QueryTest with BeforeAndAfterAll {
         buildAvroTestData(3, null)
       }
     
    -  test("Read sdk writer Avro output ") {
    +  def buildAvroTestDataArrayType(rows: Int, options: util.Map[String, String]): Any = {
    +    FileUtils.deleteDirectory(new File(writerPath))
    +    /**
    +     * *
    +     * {
    +     * "name": "address",
    +     * "type": "record",
    +     * "fields": [
    +     * {
    +     * "name": "name",
    +     * "type": "string"
    +     * },
    +     * {
    +     * "name": "age",
    +     * "type": "int"
    +     * },
    +     * {
    +     * "name": "address",
    +     * "type": {
    +     * "type": "array",
    +     * "items": {
    +     * "name": "street",
    +     * "type": "string"
    +     * }
    +     * }
    +     * }
    +     * ]
    +     * }
    +     **/
    +    val mySchema = "{\n" + "\t\"name\": \"address\",\n" + "\t\"type\": \"record\",\n" +
    +                   "\t\"fields\": [\n" + "\t\t{\n" + "\t\t\t\"name\": \"name\",\n" +
    +                   "\t\t\t\"type\": \"string\"\n" + "\t\t},\n" + "\t\t{\n" +
    +                   "\t\t\t\"name\": \"age\",\n" + "\t\t\t\"type\": \"int\"\n" + "\t\t},\n" +
    +                   "\t\t{\n" + "\t\t\t\"name\": \"address\",\n" + "\t\t\t\"type\": {\n" +
    +                   "\t\t\t\t\"type\": \"array\",\n" + "\t\t\t\t\"items\": {\n" +
    +                   "\t\t\t\t\t\"name\": \"street\",\n" +
    +                   "\t\t\t\t\t\"type\": \"string\"\n" + "\t\t\t\t}\n" + "\t\t\t}\n" +
    +                   "\t\t}\n" + "\t]\n" + "}"
    +    /**
    +     * {
    +     * "name": "bob",
    +     * "age": 10,
    +     * "address": [
    +     * "abc", "def"
    +     * ]
    +     * }
    +     **/
    +    val json: String = "{\n" + "\t\"name\": \"bob\",\n" + "\t\"age\": 10,\n" +
    +                       "\t\"address\": [\n" + "\t\t\"abc\", \"defg\"\n" + "\t]\n" + "}"
    +    // conversion to GenericData.Record
    +    val nn = new org.apache.avro.Schema.Parser().parse(mySchema)
    +    val converter = new JsonAvroConverter
    +    val record = converter
    +      .convertToGenericDataRecord(json.getBytes(CharEncoding.UTF_8), nn)
    +    val fields = new Array[Field](3)
    +    fields(0) = new Field("name", DataTypes.STRING)
    +    fields(1) = new Field("age", DataTypes.INT)
    +    // fields[1] = new Field("age", DataTypes.INT);
    +    val fld = new util.ArrayList[StructField]
    +    fld.add(new StructField("street", DataTypes.STRING))
    +    fld.add(new StructField("city", DataTypes.STRING))
    +    fields(2) = new Field("address", "struct", fld)
    +    try {
    --- End diff --
    
    use intercept[Exception] instead of try catch


---

[GitHub] carbondata pull request #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns...

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

    https://github.com/apache/carbondata/pull/2261#discussion_r186055820
  
    --- Diff: store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriterBuilder.java ---
    @@ -367,9 +384,8 @@ private CarbonTable buildCarbonTable() {
           //  user passed size 4 but supplied only 2 fileds
           for (Field field : schema.getFields()) {
             if (null != field) {
    -          if (field.getDataType() == DataTypes.STRING ||
    -              field.getDataType() == DataTypes.DATE ||
    -              field.getDataType() == DataTypes.TIMESTAMP) {
    +          if (field.getDataType() == DataTypes.STRING || field.getDataType() == DataTypes.DATE
    --- End diff --
    
    This check is for creating all the dimension columns as sort columns when by default sort column is not specified. 


---

[GitHub] carbondata issue #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns given ...

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

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



---

[GitHub] carbondata pull request #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns...

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

    https://github.com/apache/carbondata/pull/2261#discussion_r186055914
  
    --- Diff: integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestNonTransactionalCarbonTable.scala ---
    @@ -761,9 +847,29 @@ class TestNonTransactionalCarbonTable extends QueryTest with BeforeAndAfterAll {
         sql("select * from sdkOutputTable").show(false)
     
         checkAnswer(sql("select * from sdkOutputTable"), Seq(
    -      Row("bob", "10", Row("abc","bang")),
    -      Row("bob", "10", Row("abc","bang")),
    -      Row("bob", "10", Row("abc","bang"))))
    +      Row("bob", 10, Row("abc","bang")),
    +      Row("bob", 10, Row("abc","bang")),
    +      Row("bob", 10, Row("abc","bang"))))
    +
    +    sql("DROP TABLE sdkOutputTable")
    +    // drop table should not delete the files
    +    assert(new File(writerPath).exists())
    --- End diff --
    
    check for file existence inside writerPath instead of the folder


---

[GitHub] carbondata pull request #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns...

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

    https://github.com/apache/carbondata/pull/2261#discussion_r186057583
  
    --- Diff: store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriterBuilder.java ---
    @@ -416,16 +411,58 @@ private CarbonTable buildCarbonTable() {
         }
         TableSchema schema = tableSchemaBuilder.build();
         schema.setTableName(tableName);
    -    CarbonTable table = CarbonTable.builder()
    -        .tableName(schema.getTableName())
    -        .databaseName(dbName)
    -        .tablePath(path)
    -        .tableSchema(schema)
    -        .isTransactionalTable(isTransactionalTable)
    -        .build();
    +    CarbonTable table =
    +        CarbonTable.builder().tableName(schema.getTableName()).databaseName(dbName).tablePath(path)
    +            .tableSchema(schema).isTransactionalTable(isTransactionalTable).build();
         return table;
       }
     
    +  private void buildTableSchema(Field[] fields, TableSchemaBuilder tableSchemaBuilder,
    +      List<String> sortColumnsList, ColumnSchema[] sortColumnsSchemaList) {
    +    for (Field field : fields) {
    +      if (null != field) {
    +        int isSortColumn = sortColumnsList.indexOf(field.getFieldName());
    +        if (isSortColumn > -1) {
    +          // unsupported types for ("array", "struct", "double", "float", "decimal")
    +          if (field.getDataType() == DataTypes.DOUBLE || field.getDataType() == DataTypes.FLOAT
    +              || DataTypes.isDecimal(field.getDataType()) || DataTypes
    +              .isArrayType(field.getDataType()) || DataTypes.isStructType(field.getDataType())) {
    --- End diff --
    
    Done


---

[GitHub] carbondata pull request #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns...

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

    https://github.com/apache/carbondata/pull/2261#discussion_r185712926
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableSchemaBuilder.java ---
    @@ -169,11 +186,11 @@ private void checkRepeatColumnName(StructField field) {
             throw new IllegalArgumentException("column name already exists");
           }
         }
    -    for (ColumnSchema column : otherColumns) {
    -      if (column.getColumnName().equalsIgnoreCase(field.getFieldName())) {
    -        throw new IllegalArgumentException("column name already exists");
    -      }
    -    }
    +//    for (ColumnSchema column : otherColumns) {
    --- End diff --
    
    Duplicate column handling is required for non sort column, please handle it.


---

[GitHub] carbondata pull request #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns...

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

    https://github.com/apache/carbondata/pull/2261#discussion_r186055186
  
    --- Diff: integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestNonTransactionalCarbonTable.scala ---
    @@ -750,7 +750,93 @@ class TestNonTransactionalCarbonTable extends QueryTest with BeforeAndAfterAll {
         buildAvroTestData(3, null)
       }
     
    -  test("Read sdk writer Avro output ") {
    +  def buildAvroTestDataArrayType(rows: Int, options: util.Map[String, String]): Any = {
    +    FileUtils.deleteDirectory(new File(writerPath))
    +    /**
    +     * *
    +     * {
    +     * "name": "address",
    +     * "type": "record",
    +     * "fields": [
    +     * {
    +     * "name": "name",
    +     * "type": "string"
    +     * },
    +     * {
    +     * "name": "age",
    +     * "type": "int"
    +     * },
    +     * {
    +     * "name": "address",
    +     * "type": {
    +     * "type": "array",
    +     * "items": {
    +     * "name": "street",
    +     * "type": "string"
    +     * }
    +     * }
    +     * }
    +     * ]
    +     * }
    +     **/
    +    val mySchema = "{\n" + "\t\"name\": \"address\",\n" + "\t\"type\": \"record\",\n" +
    +                   "\t\"fields\": [\n" + "\t\t{\n" + "\t\t\t\"name\": \"name\",\n" +
    +                   "\t\t\t\"type\": \"string\"\n" + "\t\t},\n" + "\t\t{\n" +
    +                   "\t\t\t\"name\": \"age\",\n" + "\t\t\t\"type\": \"int\"\n" + "\t\t},\n" +
    +                   "\t\t{\n" + "\t\t\t\"name\": \"address\",\n" + "\t\t\t\"type\": {\n" +
    +                   "\t\t\t\t\"type\": \"array\",\n" + "\t\t\t\t\"items\": {\n" +
    +                   "\t\t\t\t\t\"name\": \"street\",\n" +
    +                   "\t\t\t\t\t\"type\": \"string\"\n" + "\t\t\t\t}\n" + "\t\t\t}\n" +
    +                   "\t\t}\n" + "\t]\n" + "}"
    +    /**
    +     * {
    +     * "name": "bob",
    +     * "age": 10,
    +     * "address": [
    +     * "abc", "def"
    +     * ]
    +     * }
    +     **/
    +    val json: String = "{\n" + "\t\"name\": \"bob\",\n" + "\t\"age\": 10,\n" +
    +                       "\t\"address\": [\n" + "\t\t\"abc\", \"defg\"\n" + "\t]\n" + "}"
    +    // conversion to GenericData.Record
    +    val nn = new org.apache.avro.Schema.Parser().parse(mySchema)
    +    val converter = new JsonAvroConverter
    +    val record = converter
    +      .convertToGenericDataRecord(json.getBytes(CharEncoding.UTF_8), nn)
    +    val fields = new Array[Field](3)
    +    fields(0) = new Field("name", DataTypes.STRING)
    +    fields(1) = new Field("age", DataTypes.INT)
    +    // fields[1] = new Field("age", DataTypes.INT);
    --- End diff --
    
    remove commented code


---

[GitHub] carbondata pull request #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns...

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

    https://github.com/apache/carbondata/pull/2261#discussion_r186058580
  
    --- Diff: integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestNonTransactionalCarbonTable.scala ---
    @@ -761,9 +847,29 @@ class TestNonTransactionalCarbonTable extends QueryTest with BeforeAndAfterAll {
         sql("select * from sdkOutputTable").show(false)
     
         checkAnswer(sql("select * from sdkOutputTable"), Seq(
    -      Row("bob", "10", Row("abc","bang")),
    -      Row("bob", "10", Row("abc","bang")),
    -      Row("bob", "10", Row("abc","bang"))))
    +      Row("bob", 10, Row("abc","bang")),
    +      Row("bob", 10, Row("abc","bang")),
    +      Row("bob", 10, Row("abc","bang"))))
    +
    +    sql("DROP TABLE sdkOutputTable")
    +    // drop table should not delete the files
    +    assert(new File(writerPath).exists())
    --- End diff --
    
    done


---

[GitHub] carbondata issue #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns given ...

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

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


---

[GitHub] carbondata issue #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns given ...

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

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



---

[GitHub] carbondata issue #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns given ...

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

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



---

[GitHub] carbondata issue #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns given ...

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

    https://github.com/apache/carbondata/pull/2261
  
    @sounakr Building schema should be common across spark-integration and sdk, there should not be two paths. I feel it is better to refactor to use common code.


---

[GitHub] carbondata pull request #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns...

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

    https://github.com/apache/carbondata/pull/2261#discussion_r186049025
  
    --- Diff: store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriterBuilder.java ---
    @@ -367,9 +384,8 @@ private CarbonTable buildCarbonTable() {
           //  user passed size 4 but supplied only 2 fileds
           for (Field field : schema.getFields()) {
             if (null != field) {
    -          if (field.getDataType() == DataTypes.STRING ||
    -              field.getDataType() == DataTypes.DATE ||
    -              field.getDataType() == DataTypes.TIMESTAMP) {
    +          if (field.getDataType() == DataTypes.STRING || field.getDataType() == DataTypes.DATE
    --- End diff --
    
    Sort column is also supported for long, short, int data type , add a reverse check if data type is not of  double, bigdecimal, struct, array then add to sort column


---

[GitHub] carbondata issue #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns given ...

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

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



---

[GitHub] carbondata issue #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns given ...

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

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



---

[GitHub] carbondata issue #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns given ...

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

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


---

[GitHub] carbondata pull request #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns...

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

    https://github.com/apache/carbondata/pull/2261#discussion_r186056900
  
    --- Diff: store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriterBuilder.java ---
    @@ -416,16 +411,58 @@ private CarbonTable buildCarbonTable() {
         }
         TableSchema schema = tableSchemaBuilder.build();
         schema.setTableName(tableName);
    -    CarbonTable table = CarbonTable.builder()
    -        .tableName(schema.getTableName())
    -        .databaseName(dbName)
    -        .tablePath(path)
    -        .tableSchema(schema)
    -        .isTransactionalTable(isTransactionalTable)
    -        .build();
    +    CarbonTable table =
    +        CarbonTable.builder().tableName(schema.getTableName()).databaseName(dbName).tablePath(path)
    +            .tableSchema(schema).isTransactionalTable(isTransactionalTable).build();
         return table;
       }
     
    +  private void buildTableSchema(Field[] fields, TableSchemaBuilder tableSchemaBuilder,
    +      List<String> sortColumnsList, ColumnSchema[] sortColumnsSchemaList) {
    +    for (Field field : fields) {
    +      if (null != field) {
    +        int isSortColumn = sortColumnsList.indexOf(field.getFieldName());
    +        if (isSortColumn > -1) {
    +          // unsupported types for ("array", "struct", "double", "float", "decimal")
    +          if (field.getDataType() == DataTypes.DOUBLE || field.getDataType() == DataTypes.FLOAT
    +              || DataTypes.isDecimal(field.getDataType()) || DataTypes
    +              .isArrayType(field.getDataType()) || DataTypes.isStructType(field.getDataType())) {
    --- End diff --
    
    use field.getDataType().isComplexType() to check for complex types


---

[GitHub] carbondata issue #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns given ...

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

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



---

[GitHub] carbondata issue #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns given ...

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

    https://github.com/apache/carbondata/pull/2261
  
    It is `reorder`, not `reshuffle`. Better to modify the title of PR and commit


---

[GitHub] carbondata pull request #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns...

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

    https://github.com/apache/carbondata/pull/2261#discussion_r186055072
  
    --- Diff: examples/spark2/src/main/scala/org/apache/carbondata/examples/DataFrameComplexTypeExample.scala ---
    @@ -56,6 +72,38 @@ object DataFrameComplexTypeExample {
              | 'dictionary_include'='city')
              | """.stripMargin)
     
    +    spark.sql(
    +      s"""
    +         | CREATE TABLE ${ complexTypeNoDictionaryTableNameArray }(
    +         | id INT,
    +         | name STRING,
    +         | city STRING,
    +         | salary FLOAT,
    +         | file array<string>
    +         | )
    +         | STORED BY 'carbondata'
    +         | TBLPROPERTIES(
    +         | 'sort_columns'='name',
    +         | 'dictionary_include'='city')
    +         | """.stripMargin)
    +
    +
    +    spark.sql(
    +      s"""
    +         | CREATE TABLE ${ complexTypeNoDictionaryTableName }(
    +         | id INT,
    +         | name STRING,
    +         | city STRING,
    +         | salary FLOAT,
    +         | file struct<school:array<string>, school1:array<string>, age:int>
    +         | )
    +         | STORED BY 'carbondata'
    +         | TBLPROPERTIES(
    +         | 'sort_columns'='name',
    +         | 'dictionary_exclude'='val')
    --- End diff --
    
    Need to fix this example class after PR#2266 is merged. It will make complex columns as dictionary_exclude by default.


---

[GitHub] carbondata pull request #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns...

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

    https://github.com/apache/carbondata/pull/2261#discussion_r186055858
  
    --- Diff: store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriterBuilder.java ---
    @@ -416,16 +411,58 @@ private CarbonTable buildCarbonTable() {
         }
         TableSchema schema = tableSchemaBuilder.build();
         schema.setTableName(tableName);
    -    CarbonTable table = CarbonTable.builder()
    -        .tableName(schema.getTableName())
    -        .databaseName(dbName)
    -        .tablePath(path)
    -        .tableSchema(schema)
    -        .isTransactionalTable(isTransactionalTable)
    -        .build();
    +    CarbonTable table =
    +        CarbonTable.builder().tableName(schema.getTableName()).databaseName(dbName).tablePath(path)
    +            .tableSchema(schema).isTransactionalTable(isTransactionalTable).build();
         return table;
       }
     
    +  private void buildTableSchema(Field[] fields, TableSchemaBuilder tableSchemaBuilder,
    +      List<String> sortColumnsList, ColumnSchema[] sortColumnsSchemaList) {
    +    for (Field field : fields) {
    +      if (null != field) {
    +        int isSortColumn = sortColumnsList.indexOf(field.getFieldName());
    +        if (isSortColumn > -1) {
    +          // unsupported types for ("array", "struct", "double", "float", "decimal")
    +          if (field.getDataType() == DataTypes.DOUBLE || field.getDataType() == DataTypes.FLOAT
    +              || DataTypes.isDecimal(field.getDataType()) || DataTypes
    +              .isArrayType(field.getDataType()) || DataTypes.isStructType(field.getDataType())) {
    +            throw new RuntimeException(
    +                " sort columns not supported for " + "array, struct, double, float, decimal ");
    +          }
    +        }
    +
    +        if (field.getChildren() != null && field.getChildren().size() > 0) {
    +          if (field.getDataType().getName().equalsIgnoreCase("ARRAY")) {
    +            // Loop through the inner columns and for a StructData
    +            DataType complexType =
    +                DataTypes.createArrayType(field.getChildren().get(0).getDataType());
    +            tableSchemaBuilder.addColumn(new StructField(field.getFieldName(), complexType), false);
    +          } else if (field.getDataType().getName().equalsIgnoreCase("STRUCT")) {
    +            // Loop through the inner columns and for a StructData
    +            List<StructField> structFieldsArray =
    +                new ArrayList<StructField>(field.getChildren().size());
    +            for (StructField childFld : field.getChildren()) {
    +              structFieldsArray
    +                  .add(new StructField(childFld.getFieldName(), childFld.getDataType()));
    +            }
    +            DataType complexType = DataTypes.createStructType(structFieldsArray);
    +            tableSchemaBuilder.addColumn(new StructField(field.getFieldName(), complexType), false);
    +          }
    +        } else {
    +
    --- End diff --
    
    Done


---

[GitHub] carbondata issue #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns given ...

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

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



---

[GitHub] carbondata issue #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns given ...

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

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



---

[GitHub] carbondata issue #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns given ...

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

    https://github.com/apache/carbondata/pull/2261
  
    @ravipesala Refactoring of Building schema will be done as part of a separate PR as the changes are more and can impact the CarbonTable creation. 


---

[GitHub] carbondata issue #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns given ...

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

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



---

[GitHub] carbondata issue #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns given ...

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

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



---

[GitHub] carbondata issue #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns given ...

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

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



---

[GitHub] carbondata issue #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns given ...

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

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



---

[GitHub] carbondata issue #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns given ...

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

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



---

[GitHub] carbondata issue #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns given ...

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

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



---

[GitHub] carbondata issue #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns given ...

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

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



---

[GitHub] carbondata issue #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns given ...

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

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



---

[GitHub] carbondata issue #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns given ...

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

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



---

[GitHub] carbondata issue #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns given ...

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

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


---

[GitHub] carbondata issue #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns given ...

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

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



---

[GitHub] carbondata issue #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns given ...

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

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


---

[GitHub] carbondata issue #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns given ...

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

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



---

[GitHub] carbondata issue #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns given ...

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

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



---

[GitHub] carbondata pull request #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns...

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

    https://github.com/apache/carbondata/pull/2261#discussion_r186057833
  
    --- Diff: integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestNonTransactionalCarbonTable.scala ---
    @@ -750,7 +750,93 @@ class TestNonTransactionalCarbonTable extends QueryTest with BeforeAndAfterAll {
         buildAvroTestData(3, null)
       }
     
    -  test("Read sdk writer Avro output ") {
    +  def buildAvroTestDataArrayType(rows: Int, options: util.Map[String, String]): Any = {
    +    FileUtils.deleteDirectory(new File(writerPath))
    +    /**
    +     * *
    +     * {
    +     * "name": "address",
    +     * "type": "record",
    +     * "fields": [
    +     * {
    +     * "name": "name",
    +     * "type": "string"
    +     * },
    +     * {
    +     * "name": "age",
    +     * "type": "int"
    +     * },
    +     * {
    +     * "name": "address",
    +     * "type": {
    +     * "type": "array",
    +     * "items": {
    +     * "name": "street",
    +     * "type": "string"
    +     * }
    +     * }
    +     * }
    +     * ]
    +     * }
    +     **/
    +    val mySchema = "{\n" + "\t\"name\": \"address\",\n" + "\t\"type\": \"record\",\n" +
    +                   "\t\"fields\": [\n" + "\t\t{\n" + "\t\t\t\"name\": \"name\",\n" +
    +                   "\t\t\t\"type\": \"string\"\n" + "\t\t},\n" + "\t\t{\n" +
    +                   "\t\t\t\"name\": \"age\",\n" + "\t\t\t\"type\": \"int\"\n" + "\t\t},\n" +
    +                   "\t\t{\n" + "\t\t\t\"name\": \"address\",\n" + "\t\t\t\"type\": {\n" +
    +                   "\t\t\t\t\"type\": \"array\",\n" + "\t\t\t\t\"items\": {\n" +
    +                   "\t\t\t\t\t\"name\": \"street\",\n" +
    +                   "\t\t\t\t\t\"type\": \"string\"\n" + "\t\t\t\t}\n" + "\t\t\t}\n" +
    +                   "\t\t}\n" + "\t]\n" + "}"
    +    /**
    +     * {
    +     * "name": "bob",
    +     * "age": 10,
    +     * "address": [
    +     * "abc", "def"
    +     * ]
    +     * }
    +     **/
    +    val json: String = "{\n" + "\t\"name\": \"bob\",\n" + "\t\"age\": 10,\n" +
    +                       "\t\"address\": [\n" + "\t\t\"abc\", \"defg\"\n" + "\t]\n" + "}"
    +    // conversion to GenericData.Record
    +    val nn = new org.apache.avro.Schema.Parser().parse(mySchema)
    +    val converter = new JsonAvroConverter
    +    val record = converter
    +      .convertToGenericDataRecord(json.getBytes(CharEncoding.UTF_8), nn)
    +    val fields = new Array[Field](3)
    +    fields(0) = new Field("name", DataTypes.STRING)
    +    fields(1) = new Field("age", DataTypes.INT)
    +    // fields[1] = new Field("age", DataTypes.INT);
    --- End diff --
    
    Done


---

[GitHub] carbondata issue #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns given ...

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

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



---

[GitHub] carbondata issue #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns given ...

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

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



---

[GitHub] carbondata issue #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns given ...

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

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



---

[GitHub] carbondata issue #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns given ...

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

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



---

[GitHub] carbondata issue #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns given ...

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

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


---

[GitHub] carbondata issue #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns given ...

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

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



---

[GitHub] carbondata pull request #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns...

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

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


---

[GitHub] carbondata pull request #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns...

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

    https://github.com/apache/carbondata/pull/2261#discussion_r186056928
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableSchemaBuilder.java ---
    @@ -108,21 +115,36 @@ public void setSortColumns(List<ColumnSchema> sortColumns) {
       }
     
       public ColumnSchema addColumn(StructField field, boolean isSortColumn) {
    +    return addColumn(field, null, isSortColumn, false);
    +  }
    +
    +  private ColumnSchema addColumn(StructField field, String parentName, boolean isSortColumn,
    +      boolean isComplexChild) {
         Objects.requireNonNull(field);
         checkRepeatColumnName(field);
         ColumnSchema newColumn = new ColumnSchema();
    -    newColumn.setColumnName(field.getFieldName());
    +    if (parentName != null) {
    +      newColumn.setColumnName(parentName + "." + field.getFieldName());
    +    } else {
    +      newColumn.setColumnName(field.getFieldName());
    +    }
         newColumn.setDataType(field.getDataType());
         if (isSortColumn ||
             field.getDataType() == DataTypes.STRING ||
    --- End diff --
    
    In case if i have to place a negative condition then we have to add 
    Non Dictionary 
    -------------------
    Boolean
    short
    int
    float
    long
    double
    Null
    Byte
    
    
    Dictionary 
    -------------
    string
    array
    struct
    timestamp
    date
    char
    
    
    AS dictionary check is smaller keeping it as it is. 


---

[GitHub] carbondata issue #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns given ...

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

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


---

[GitHub] carbondata pull request #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns...

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

    https://github.com/apache/carbondata/pull/2261#discussion_r186053512
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableSchemaBuilder.java ---
    @@ -135,25 +157,41 @@ public ColumnSchema addColumn(StructField field, boolean isSortColumn) {
         newColumn.setColumnReferenceId(newColumn.getColumnUniqueId());
         newColumn.setEncodingList(createEncoding(field.getDataType(), isSortColumn));
         if (field.getDataType().isComplexType()) {
    -      newColumn.setNumberOfChild(((StructType) field.getDataType()).getFields().size());
    +      if (field.getDataType().getName().equalsIgnoreCase("ARRAY")) {
    +        newColumn.setNumberOfChild(1);
    +      } else {
    +        newColumn.setNumberOfChild(((StructType) field.getDataType()).getFields().size());
    +      }
         }
         if (DataTypes.isDecimal(field.getDataType())) {
           DecimalType decimalType = (DecimalType) field.getDataType();
           newColumn.setPrecision(decimalType.getPrecision());
           newColumn.setScale(decimalType.getScale());
         }
         if (!isSortColumn) {
    -      otherColumns.add(newColumn);
    +      if (!newColumn.isDimensionColumn()) {
    +        measures.add(newColumn);
    +      } else if (DataTypes.isStructType(field.getDataType()) ||
    +          DataTypes.isArrayType(field.getDataType()) || isComplexChild) {
    --- End diff --
    
    Instead of DataTypes.isStructType(field.getDataType()) || DataTypes.isArrayType(field.getDataType()) use field.getDataType().isComplexType()


---

[GitHub] carbondata pull request #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns...

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

    https://github.com/apache/carbondata/pull/2261#discussion_r186043853
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableSchemaBuilder.java ---
    @@ -108,21 +115,36 @@ public void setSortColumns(List<ColumnSchema> sortColumns) {
       }
     
       public ColumnSchema addColumn(StructField field, boolean isSortColumn) {
    +    return addColumn(field, null, isSortColumn, false);
    +  }
    +
    +  private ColumnSchema addColumn(StructField field, String parentName, boolean isSortColumn,
    +      boolean isComplexChild) {
         Objects.requireNonNull(field);
         checkRepeatColumnName(field);
         ColumnSchema newColumn = new ColumnSchema();
    -    newColumn.setColumnName(field.getFieldName());
    +    if (parentName != null) {
    +      newColumn.setColumnName(parentName + "." + field.getFieldName());
    +    } else {
    +      newColumn.setColumnName(field.getFieldName());
    +    }
         newColumn.setDataType(field.getDataType());
         if (isSortColumn ||
             field.getDataType() == DataTypes.STRING ||
    --- End diff --
    
    Instead of adding so many conditions in if add if(!(field.getDataType()==DataTypes.Double || field.getDataType()==DataTypes.BigDecimal)). Dimension column can be long, short, int, currently it is not handled


---

[GitHub] carbondata pull request #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns...

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

    https://github.com/apache/carbondata/pull/2261#discussion_r186041329
  
    --- Diff: store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriterBuilder.java ---
    @@ -416,16 +411,58 @@ private CarbonTable buildCarbonTable() {
         }
         TableSchema schema = tableSchemaBuilder.build();
         schema.setTableName(tableName);
    -    CarbonTable table = CarbonTable.builder()
    -        .tableName(schema.getTableName())
    -        .databaseName(dbName)
    -        .tablePath(path)
    -        .tableSchema(schema)
    -        .isTransactionalTable(isTransactionalTable)
    -        .build();
    +    CarbonTable table =
    +        CarbonTable.builder().tableName(schema.getTableName()).databaseName(dbName).tablePath(path)
    +            .tableSchema(schema).isTransactionalTable(isTransactionalTable).build();
         return table;
       }
     
    +  private void buildTableSchema(Field[] fields, TableSchemaBuilder tableSchemaBuilder,
    +      List<String> sortColumnsList, ColumnSchema[] sortColumnsSchemaList) {
    +    for (Field field : fields) {
    +      if (null != field) {
    +        int isSortColumn = sortColumnsList.indexOf(field.getFieldName());
    +        if (isSortColumn > -1) {
    +          // unsupported types for ("array", "struct", "double", "float", "decimal")
    +          if (field.getDataType() == DataTypes.DOUBLE || field.getDataType() == DataTypes.FLOAT
    +              || DataTypes.isDecimal(field.getDataType()) || DataTypes
    +              .isArrayType(field.getDataType()) || DataTypes.isStructType(field.getDataType())) {
    +            throw new RuntimeException(
    +                " sort columns not supported for " + "array, struct, double, float, decimal ");
    +          }
    +        }
    +
    +        if (field.getChildren() != null && field.getChildren().size() > 0) {
    +          if (field.getDataType().getName().equalsIgnoreCase("ARRAY")) {
    +            // Loop through the inner columns and for a StructData
    +            DataType complexType =
    +                DataTypes.createArrayType(field.getChildren().get(0).getDataType());
    +            tableSchemaBuilder.addColumn(new StructField(field.getFieldName(), complexType), false);
    +          } else if (field.getDataType().getName().equalsIgnoreCase("STRUCT")) {
    +            // Loop through the inner columns and for a StructData
    +            List<StructField> structFieldsArray =
    +                new ArrayList<StructField>(field.getChildren().size());
    +            for (StructField childFld : field.getChildren()) {
    +              structFieldsArray
    +                  .add(new StructField(childFld.getFieldName(), childFld.getDataType()));
    +            }
    +            DataType complexType = DataTypes.createStructType(structFieldsArray);
    +            tableSchemaBuilder.addColumn(new StructField(field.getFieldName(), complexType), false);
    +          }
    +        } else {
    +
    --- End diff --
    
    remove empty lines


---

[GitHub] carbondata issue #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns given ...

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

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



---

[GitHub] carbondata issue #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns given ...

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

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


---

[GitHub] carbondata issue #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns given ...

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

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



---

[GitHub] carbondata issue #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns given ...

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

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



---

[GitHub] carbondata pull request #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns...

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

    https://github.com/apache/carbondata/pull/2261#discussion_r186059780
  
    --- Diff: integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala ---
    @@ -604,31 +604,58 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
             tableProperties.get(CarbonCommonConstants.DICTIONARY_EXCLUDE).get.split(',').map(_.trim)
           dictExcludeCols
             .foreach { dictExcludeCol =>
    -          if (!fields.exists(x => x.column.equalsIgnoreCase(dictExcludeCol))) {
    +          if (!checkFields(fields, dictExcludeCol)) {
                 val errormsg = "DICTIONARY_EXCLUDE column: " + dictExcludeCol +
                                " does not exist in table. Please check create table statement."
                 throw new MalformedCarbonCommandException(errormsg)
               } else {
    -            val dataType = fields.find(x =>
    -              x.column.equalsIgnoreCase(dictExcludeCol)).get.dataType.get
    -            if (isComplexDimDictionaryExclude(dataType)) {
    -              val errormsg = "DICTIONARY_EXCLUDE is unsupported for complex datatype column: " +
    -                             dictExcludeCol
    -              throw new MalformedCarbonCommandException(errormsg)
    -            } else if (!isDataTypeSupportedForDictionary_Exclude(dataType)) {
    +            val dataType = findField(fields, dictExcludeCol).get.dataType.get
    +            if (!isDataTypeSupportedForDictionary_Exclude(dataType)) {
                   val errorMsg = "DICTIONARY_EXCLUDE is unsupported for " + dataType.toLowerCase() +
                                  " data type column: " + dictExcludeCol
                   throw new MalformedCarbonCommandException(errorMsg)
                 }
               }
             }
         }
    +
    +
    +    def checkFields(y: Seq[Field], colToMatch: String): Boolean = {
    +      y.exists { fld =>
    +        if (fld.column.equalsIgnoreCase(colToMatch)) {
    +          true
    +        } else if (fld.children.isDefined && fld.children.get != null) {
    +          checkFields(fld.children.get, colToMatch)
    +        } else {
    +          false
    +        }
    +      }
    +    }
    +
    +    def findField(y: Seq[Field], colToMatch: String): Option[Field] = {
    --- End diff --
    
    Done


---

[GitHub] carbondata issue #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns given ...

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

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


---

[GitHub] carbondata issue #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns given ...

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

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



---

[GitHub] carbondata issue #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns given ...

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

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



---

[GitHub] carbondata issue #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns given ...

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

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



---

[GitHub] carbondata pull request #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns...

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

    https://github.com/apache/carbondata/pull/2261#discussion_r186057854
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableSchemaBuilder.java ---
    @@ -135,25 +157,41 @@ public ColumnSchema addColumn(StructField field, boolean isSortColumn) {
         newColumn.setColumnReferenceId(newColumn.getColumnUniqueId());
         newColumn.setEncodingList(createEncoding(field.getDataType(), isSortColumn));
         if (field.getDataType().isComplexType()) {
    -      newColumn.setNumberOfChild(((StructType) field.getDataType()).getFields().size());
    +      if (field.getDataType().getName().equalsIgnoreCase("ARRAY")) {
    +        newColumn.setNumberOfChild(1);
    +      } else {
    +        newColumn.setNumberOfChild(((StructType) field.getDataType()).getFields().size());
    +      }
         }
         if (DataTypes.isDecimal(field.getDataType())) {
           DecimalType decimalType = (DecimalType) field.getDataType();
           newColumn.setPrecision(decimalType.getPrecision());
           newColumn.setScale(decimalType.getScale());
         }
         if (!isSortColumn) {
    -      otherColumns.add(newColumn);
    +      if (!newColumn.isDimensionColumn()) {
    +        measures.add(newColumn);
    +      } else if (DataTypes.isStructType(field.getDataType()) ||
    +          DataTypes.isArrayType(field.getDataType()) || isComplexChild) {
    --- End diff --
    
    Done


---

[GitHub] carbondata issue #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns given ...

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

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



---

[GitHub] carbondata issue #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns given ...

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

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



---

[GitHub] carbondata pull request #2261: [CARBONDATA-2430][SDK] Reshuffling of Columns...

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

    https://github.com/apache/carbondata/pull/2261#discussion_r186056119
  
    --- Diff: integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala ---
    @@ -604,31 +604,58 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
             tableProperties.get(CarbonCommonConstants.DICTIONARY_EXCLUDE).get.split(',').map(_.trim)
           dictExcludeCols
             .foreach { dictExcludeCol =>
    -          if (!fields.exists(x => x.column.equalsIgnoreCase(dictExcludeCol))) {
    +          if (!checkFields(fields, dictExcludeCol)) {
                 val errormsg = "DICTIONARY_EXCLUDE column: " + dictExcludeCol +
                                " does not exist in table. Please check create table statement."
                 throw new MalformedCarbonCommandException(errormsg)
               } else {
    -            val dataType = fields.find(x =>
    -              x.column.equalsIgnoreCase(dictExcludeCol)).get.dataType.get
    -            if (isComplexDimDictionaryExclude(dataType)) {
    -              val errormsg = "DICTIONARY_EXCLUDE is unsupported for complex datatype column: " +
    -                             dictExcludeCol
    -              throw new MalformedCarbonCommandException(errormsg)
    -            } else if (!isDataTypeSupportedForDictionary_Exclude(dataType)) {
    +            val dataType = findField(fields, dictExcludeCol).get.dataType.get
    +            if (!isDataTypeSupportedForDictionary_Exclude(dataType)) {
                   val errorMsg = "DICTIONARY_EXCLUDE is unsupported for " + dataType.toLowerCase() +
                                  " data type column: " + dictExcludeCol
                   throw new MalformedCarbonCommandException(errorMsg)
                 }
               }
             }
         }
    +
    +
    +    def checkFields(y: Seq[Field], colToMatch: String): Boolean = {
    +      y.exists { fld =>
    +        if (fld.column.equalsIgnoreCase(colToMatch)) {
    +          true
    +        } else if (fld.children.isDefined && fld.children.get != null) {
    +          checkFields(fld.children.get, colToMatch)
    +        } else {
    +          false
    +        }
    +      }
    +    }
    +
    +    def findField(y: Seq[Field], colToMatch: String): Option[Field] = {
    --- End diff --
    
    Write description for checkFields and findFields


---