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

[GitHub] carbondata pull request #2436: [CARBONDATA-2682]fix create table with long_s...

GitHub user Sssan520 opened a pull request:

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

    [CARBONDATA-2682]fix create table with long_string_columns properties bugs

    
    


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

    $ git pull https://github.com/Sssan520/carbondata dts

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

    https://github.com/apache/carbondata/pull/2436.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 #2436
    
----
commit ef3876cafa1ad446ced27b018771a81d2d897e04
Author: Sssan520 <li...@...>
Date:   2018-07-02T11:12:24Z

    fix create table with long_string_columns properties bugs

----


---

[GitHub] carbondata pull request #2436: [CARBONDATA-2682][32K] fix create table with ...

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

    https://github.com/apache/carbondata/pull/2436#discussion_r200898463
  
    --- Diff: integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala ---
    @@ -486,6 +504,60 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
         }
       }
     
    +  /**
    +   * This method validates the long string columns
    --- End diff --
    
    Add comment for what is validated in the func


---

[GitHub] carbondata issue #2436: [CARBONDATA-2682]fix create table with long_string_c...

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

    https://github.com/apache/carbondata/pull/2436
  
    Can one of the admins verify this patch?


---

[GitHub] carbondata pull request #2436: [CARBONDATA-2682][32K] fix create table with ...

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

    https://github.com/apache/carbondata/pull/2436#discussion_r200896197
  
    --- Diff: integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/longstring/VarcharDataTypesBasicTestCase.scala ---
    @@ -133,6 +133,64 @@ class VarcharDataTypesBasicTestCase extends QueryTest with BeforeAndAfterEach wi
         assert(exceptionCaught.getMessage.contains("its data type is not string"))
       }
     
    +  test("long string columns cannot contain duplicate columns") {
    +    val exceptionCaught = intercept[Exception] {
    +      sql(
    +        s"""
    +           | CREATE TABLE if not exists $longStringTable(
    +           | id INT, name STRING, description STRING, address STRING, note STRING
    +           | ) STORED BY 'carbondata'
    +           | TBLPROPERTIES('LONG_STRING_COLUMNS'='address, note, Note')
    +           |""".
    +          stripMargin)
    +    }
    +    assert(exceptionCaught.getMessage.contains("long_string_columns"))
    +    assert(exceptionCaught.getMessage.contains("Duplicate columns are not allowed"))
    +  }
    +
    +  test("long_string_columns: column does not exist in table ") {
    +    val exceptionCaught = intercept[Exception] {
    --- End diff --
    
    please be more specific about the Exception, do not just intercept all Exception


---

[GitHub] carbondata pull request #2436: [CARBONDATA-2682][32K] fix create table with ...

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

    https://github.com/apache/carbondata/pull/2436#discussion_r200897566
  
    --- Diff: integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala ---
    @@ -391,6 +391,24 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
             tableProperties.get(CarbonCommonConstants.CACHE_LEVEL).get,
             tableProperties)
         }
    +    // validate long_string_columns from tale properties
    +    var longStringColumns = varcharColumns.map(column => column.toUpperCase)
    +    var noInvColIntersecLongStrCols = longStringColumns
    +      .intersect(noInvertedIdxCols.map(col => col.toUpperCase))
    +    if (!noInvColIntersecLongStrCols.isEmpty) {
    +      throw new MalformedCarbonCommandException(
    +        "Column(s):" + noInvColIntersecLongStrCols.mkString(",") +
    +        " both in no_inverted_index and " +
    --- End diff --
    
    Why this is not allowed? Please add comment in this if check


---

[GitHub] carbondata pull request #2436: [CARBONDATA-2682][32K] fix create table with ...

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

    https://github.com/apache/carbondata/pull/2436#discussion_r200896170
  
    --- Diff: integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/longstring/VarcharDataTypesBasicTestCase.scala ---
    @@ -133,6 +133,64 @@ class VarcharDataTypesBasicTestCase extends QueryTest with BeforeAndAfterEach wi
         assert(exceptionCaught.getMessage.contains("its data type is not string"))
       }
     
    +  test("long string columns cannot contain duplicate columns") {
    +    val exceptionCaught = intercept[Exception] {
    --- End diff --
    
    please be more specific about the Exception, do not just intercept all Exception


---

[GitHub] carbondata issue #2436: [CARBONDATA-2682][32K] fix create table with long_st...

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

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


---

[GitHub] carbondata issue #2436: [CARBONDATA-2682][32K] fix create table with long_st...

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

    https://github.com/apache/carbondata/pull/2436
  
    all code view comments has been handled.


---

[GitHub] carbondata issue #2436: [CARBONDATA-2682][32K] fix create table with long_st...

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

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


---

[GitHub] carbondata pull request #2436: [CARBONDATA-2682][32K] fix create table with ...

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

    https://github.com/apache/carbondata/pull/2436#discussion_r201542905
  
    --- Diff: integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/longstring/VarcharDataTypesBasicTestCase.scala ---
    @@ -133,6 +133,64 @@ class VarcharDataTypesBasicTestCase extends QueryTest with BeforeAndAfterEach wi
         assert(exceptionCaught.getMessage.contains("its data type is not string"))
       }
     
    +  test("long string columns cannot contain duplicate columns") {
    +    val exceptionCaught = intercept[Exception] {
    +      sql(
    +        s"""
    +           | CREATE TABLE if not exists $longStringTable(
    +           | id INT, name STRING, description STRING, address STRING, note STRING
    +           | ) STORED BY 'carbondata'
    +           | TBLPROPERTIES('LONG_STRING_COLUMNS'='address, note, Note')
    +           |""".
    +          stripMargin)
    +    }
    +    assert(exceptionCaught.getMessage.contains("long_string_columns"))
    +    assert(exceptionCaught.getMessage.contains("Duplicate columns are not allowed"))
    +  }
    +
    +  test("long_string_columns: column does not exist in table ") {
    +    val exceptionCaught = intercept[Exception] {
    +      sql(
    +        s"""
    +           | CREATE TABLE if not exists $longStringTable(
    +           | id INT, name STRING, description STRING, address STRING, note STRING
    +           | ) STORED BY 'carbondata'
    +           | TBLPROPERTIES('LONG_STRING_COLUMNS'='address, note, NoteS')
    +           |""".
    +          stripMargin)
    +    }
    +    assert(exceptionCaught.getMessage.contains("long_string_columns"))
    +    assert(exceptionCaught.getMessage.contains("does not exist in table"))
    +  }
    +
    +  test("long_string_columns: columns cannot exist in patitions columns") {
    +    val exceptionCaught = intercept[Exception] {
    +      sql(
    +        s"""
    +           | CREATE TABLE if not exists $longStringTable(
    +           | id INT, name STRING, description STRING, address STRING
    +           | ) partitioned by (note string) STORED BY 'carbondata'
    +           | TBLPROPERTIES('LONG_STRING_COLUMNS'='note')
    +           |""".
    +          stripMargin)
    +    }
    +    assert(exceptionCaught.getMessage.contains("both in partition and long_string_columns"))
    +  }
    +
    +  test("long_string_columns: columns cannot exist in no_inverted_index columns") {
    +    val exceptionCaught = intercept[Exception] {
    --- End diff --
    
    ok


---

[GitHub] carbondata pull request #2436: [CARBONDATA-2682][32K] fix create table with ...

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

    https://github.com/apache/carbondata/pull/2436#discussion_r201544083
  
    --- Diff: integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala ---
    @@ -486,6 +504,60 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
         }
       }
     
    +  /**
    +   * This method validates the long string columns
    --- End diff --
    
    ok


---

[GitHub] carbondata issue #2436: [CARBONDATA-2682][32K] fix create table with long_st...

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

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


---

[GitHub] carbondata pull request #2436: [CARBONDATA-2682][32K] fix create table with ...

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

    https://github.com/apache/carbondata/pull/2436#discussion_r201542897
  
    --- Diff: integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/longstring/VarcharDataTypesBasicTestCase.scala ---
    @@ -133,6 +133,64 @@ class VarcharDataTypesBasicTestCase extends QueryTest with BeforeAndAfterEach wi
         assert(exceptionCaught.getMessage.contains("its data type is not string"))
       }
     
    +  test("long string columns cannot contain duplicate columns") {
    +    val exceptionCaught = intercept[Exception] {
    +      sql(
    +        s"""
    +           | CREATE TABLE if not exists $longStringTable(
    +           | id INT, name STRING, description STRING, address STRING, note STRING
    +           | ) STORED BY 'carbondata'
    +           | TBLPROPERTIES('LONG_STRING_COLUMNS'='address, note, Note')
    +           |""".
    +          stripMargin)
    +    }
    +    assert(exceptionCaught.getMessage.contains("long_string_columns"))
    +    assert(exceptionCaught.getMessage.contains("Duplicate columns are not allowed"))
    +  }
    +
    +  test("long_string_columns: column does not exist in table ") {
    +    val exceptionCaught = intercept[Exception] {
    +      sql(
    +        s"""
    +           | CREATE TABLE if not exists $longStringTable(
    +           | id INT, name STRING, description STRING, address STRING, note STRING
    +           | ) STORED BY 'carbondata'
    +           | TBLPROPERTIES('LONG_STRING_COLUMNS'='address, note, NoteS')
    +           |""".
    +          stripMargin)
    +    }
    +    assert(exceptionCaught.getMessage.contains("long_string_columns"))
    +    assert(exceptionCaught.getMessage.contains("does not exist in table"))
    +  }
    +
    +  test("long_string_columns: columns cannot exist in patitions columns") {
    +    val exceptionCaught = intercept[Exception] {
    --- End diff --
    
    ok


---

[GitHub] carbondata pull request #2436: [CARBONDATA-2682][32K] fix create table with ...

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

    https://github.com/apache/carbondata/pull/2436#discussion_r200897815
  
    --- Diff: integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala ---
    @@ -391,6 +391,24 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
             tableProperties.get(CarbonCommonConstants.CACHE_LEVEL).get,
             tableProperties)
         }
    +    // validate long_string_columns from tale properties
    +    var longStringColumns = varcharColumns.map(column => column.toUpperCase)
    --- End diff --
    
    change to `varcharColumns.map(toUpperCase)`, same for next line


---

[GitHub] carbondata pull request #2436: [CARBONDATA-2682][32K] fix create table with ...

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

    https://github.com/apache/carbondata/pull/2436#discussion_r200898149
  
    --- Diff: integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala ---
    @@ -486,6 +504,60 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
         }
       }
     
    +  /**
    +   * This method validates the long string columns
    +   *
    +   * @param fields
    +   * @param varcharCols
    +   * @return
    +   */
    +  private def validateLongStringColumns(fields: Seq[Field],
    +      varcharCols: Seq[String]): Unit = {
    +    var longStringColumnsMap: Map[String, Field] = Map[String, Field]()
    +    fields.foreach(field =>
    +      longStringColumnsMap.put(field.column.toUpperCase, field)
    +    )
    +    var dataTypeErr: Set[String] = Set[String]()
    +    var duplicateColumnErr: Map[String, Int] = Map[String, Int]()
    +    var nullColumnErr: Set[String] = Set[String]()
    +    var tmpStr: String = ""
    +    varcharCols.foreach {
    +      column =>
    +        tmpStr = column.toUpperCase
    +        duplicateColumnErr.get(tmpStr) match {
    +          case None => duplicateColumnErr.put(tmpStr, 1)
    +          case Some(count) => duplicateColumnErr.put(tmpStr, count + 1)
    +        }
    +        longStringColumnsMap.get(tmpStr) match {
    +          case None => nullColumnErr += column
    +          case Some(field) => if (!DataTypes.STRING.getName.equalsIgnoreCase(field.dataType.get)) {
    +            dataTypeErr += column
    +          }
    +        }
    +    }
    +    if (!nullColumnErr.isEmpty) {
    +      val errMsg = "long_string_columns:" +
    --- End diff --
    
    do not use `+` to concatenate strings


---

[GitHub] carbondata pull request #2436: [CARBONDATA-2682][32K] fix create table with ...

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

    https://github.com/apache/carbondata/pull/2436#discussion_r200897980
  
    --- Diff: integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala ---
    @@ -486,6 +504,60 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
         }
       }
     
    +  /**
    +   * This method validates the long string columns
    +   *
    +   * @param fields
    +   * @param varcharCols
    +   * @return
    +   */
    +  private def validateLongStringColumns(fields: Seq[Field],
    --- End diff --
    
    Why not put line 395 validation logic also in this func?


---

[GitHub] carbondata issue #2436: [CARBONDATA-2682]fix create table with long_string_c...

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

    https://github.com/apache/carbondata/pull/2436
  
    Can one of the admins verify this patch?


---

[GitHub] carbondata issue #2436: [CARBONDATA-2682][32K] fix create table with long_st...

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

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



---

[GitHub] carbondata issue #2436: [CARBONDATA-2682][32K] fix create table with long_st...

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

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


---

[GitHub] carbondata pull request #2436: [CARBONDATA-2682][32K] fix create table with ...

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

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


---

[GitHub] carbondata issue #2436: [CARBONDATA-2682]fix create table with long_string_c...

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

    https://github.com/apache/carbondata/pull/2436
  
    Can one of the admins verify this patch?


---

[GitHub] carbondata pull request #2436: [CARBONDATA-2682][32K] fix create table with ...

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

    https://github.com/apache/carbondata/pull/2436#discussion_r201545482
  
    --- Diff: integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala ---
    @@ -486,6 +504,60 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
         }
       }
     
    +  /**
    +   * This method validates the long string columns
    +   *
    +   * @param fields
    +   * @param varcharCols
    +   * @return
    +   */
    +  private def validateLongStringColumns(fields: Seq[Field],
    +      varcharCols: Seq[String]): Unit = {
    +    var longStringColumnsMap: Map[String, Field] = Map[String, Field]()
    +    fields.foreach(field =>
    +      longStringColumnsMap.put(field.column.toUpperCase, field)
    +    )
    +    var dataTypeErr: Set[String] = Set[String]()
    +    var duplicateColumnErr: Map[String, Int] = Map[String, Int]()
    +    var nullColumnErr: Set[String] = Set[String]()
    +    var tmpStr: String = ""
    +    varcharCols.foreach {
    +      column =>
    +        tmpStr = column.toUpperCase
    +        duplicateColumnErr.get(tmpStr) match {
    +          case None => duplicateColumnErr.put(tmpStr, 1)
    +          case Some(count) => duplicateColumnErr.put(tmpStr, count + 1)
    +        }
    +        longStringColumnsMap.get(tmpStr) match {
    +          case None => nullColumnErr += column
    +          case Some(field) => if (!DataTypes.STRING.getName.equalsIgnoreCase(field.dataType.get)) {
    +            dataTypeErr += column
    +          }
    +        }
    +    }
    +    if (!nullColumnErr.isEmpty) {
    +      val errMsg = "long_string_columns:" +
    --- End diff --
    
    ok


---

[GitHub] carbondata pull request #2436: [CARBONDATA-2682][32K] fix create table with ...

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

    https://github.com/apache/carbondata/pull/2436#discussion_r200896241
  
    --- Diff: integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/longstring/VarcharDataTypesBasicTestCase.scala ---
    @@ -133,6 +133,64 @@ class VarcharDataTypesBasicTestCase extends QueryTest with BeforeAndAfterEach wi
         assert(exceptionCaught.getMessage.contains("its data type is not string"))
       }
     
    +  test("long string columns cannot contain duplicate columns") {
    +    val exceptionCaught = intercept[Exception] {
    +      sql(
    +        s"""
    +           | CREATE TABLE if not exists $longStringTable(
    +           | id INT, name STRING, description STRING, address STRING, note STRING
    +           | ) STORED BY 'carbondata'
    +           | TBLPROPERTIES('LONG_STRING_COLUMNS'='address, note, Note')
    +           |""".
    +          stripMargin)
    +    }
    +    assert(exceptionCaught.getMessage.contains("long_string_columns"))
    +    assert(exceptionCaught.getMessage.contains("Duplicate columns are not allowed"))
    +  }
    +
    +  test("long_string_columns: column does not exist in table ") {
    +    val exceptionCaught = intercept[Exception] {
    +      sql(
    +        s"""
    +           | CREATE TABLE if not exists $longStringTable(
    +           | id INT, name STRING, description STRING, address STRING, note STRING
    +           | ) STORED BY 'carbondata'
    +           | TBLPROPERTIES('LONG_STRING_COLUMNS'='address, note, NoteS')
    +           |""".
    +          stripMargin)
    +    }
    +    assert(exceptionCaught.getMessage.contains("long_string_columns"))
    +    assert(exceptionCaught.getMessage.contains("does not exist in table"))
    +  }
    +
    +  test("long_string_columns: columns cannot exist in patitions columns") {
    +    val exceptionCaught = intercept[Exception] {
    +      sql(
    +        s"""
    +           | CREATE TABLE if not exists $longStringTable(
    +           | id INT, name STRING, description STRING, address STRING
    +           | ) partitioned by (note string) STORED BY 'carbondata'
    +           | TBLPROPERTIES('LONG_STRING_COLUMNS'='note')
    +           |""".
    +          stripMargin)
    +    }
    +    assert(exceptionCaught.getMessage.contains("both in partition and long_string_columns"))
    +  }
    +
    +  test("long_string_columns: columns cannot exist in no_inverted_index columns") {
    +    val exceptionCaught = intercept[Exception] {
    --- End diff --
    
    please be more specific about the Exception, do not just intercept all Exception


---

[GitHub] carbondata pull request #2436: [CARBONDATA-2682][32K] fix create table with ...

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

    https://github.com/apache/carbondata/pull/2436#discussion_r202009599
  
    --- Diff: integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala ---
    @@ -486,6 +504,60 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
         }
       }
     
    +  /**
    +   * This method validates the long string columns
    +   *
    +   * @param fields
    +   * @param varcharCols
    +   * @return
    +   */
    +  private def validateLongStringColumns(fields: Seq[Field],
    +      varcharCols: Seq[String]): Unit = {
    +    var longStringColumnsMap: Map[String, Field] = Map[String, Field]()
    +    fields.foreach(field =>
    +      longStringColumnsMap.put(field.column.toUpperCase, field)
    +    )
    +    var dataTypeErr: Set[String] = Set[String]()
    +    var duplicateColumnErr: Map[String, Int] = Map[String, Int]()
    +    var nullColumnErr: Set[String] = Set[String]()
    +    var tmpStr: String = ""
    +    varcharCols.foreach {
    +      column =>
    +        tmpStr = column.toUpperCase
    +        duplicateColumnErr.get(tmpStr) match {
    +          case None => duplicateColumnErr.put(tmpStr, 1)
    +          case Some(count) => duplicateColumnErr.put(tmpStr, count + 1)
    +        }
    +        longStringColumnsMap.get(tmpStr) match {
    +          case None => nullColumnErr += column
    +          case Some(field) => if (!DataTypes.STRING.getName.equalsIgnoreCase(field.dataType.get)) {
    +            dataTypeErr += column
    +          }
    +        }
    +    }
    +    if (!nullColumnErr.isEmpty) {
    +      val errMsg = "long_string_columns:" +
    +                   nullColumnErr.mkString(",") +
    +                   " does not exist in table. Please check create table statement."
    +      throw new MalformedCarbonCommandException(errMsg)
    +    }
    +
    +    var duplicateColumns = duplicateColumnErr.filter(kv => kv._2 != 1).keySet
    --- End diff --
    
    to check if exists duplicate  columns in long_string_columns property


---

[GitHub] carbondata pull request #2436: [CARBONDATA-2682][32K] fix create table with ...

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

    https://github.com/apache/carbondata/pull/2436#discussion_r201542948
  
    --- Diff: integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala ---
    @@ -391,6 +391,24 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
             tableProperties.get(CarbonCommonConstants.CACHE_LEVEL).get,
             tableProperties)
         }
    +    // validate long_string_columns from tale properties
    +    var longStringColumns = varcharColumns.map(column => column.toUpperCase)
    --- End diff --
    
    ok


---

[GitHub] carbondata pull request #2436: [CARBONDATA-2682][32K] fix create table with ...

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

    https://github.com/apache/carbondata/pull/2436#discussion_r200896218
  
    --- Diff: integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/longstring/VarcharDataTypesBasicTestCase.scala ---
    @@ -133,6 +133,64 @@ class VarcharDataTypesBasicTestCase extends QueryTest with BeforeAndAfterEach wi
         assert(exceptionCaught.getMessage.contains("its data type is not string"))
       }
     
    +  test("long string columns cannot contain duplicate columns") {
    +    val exceptionCaught = intercept[Exception] {
    +      sql(
    +        s"""
    +           | CREATE TABLE if not exists $longStringTable(
    +           | id INT, name STRING, description STRING, address STRING, note STRING
    +           | ) STORED BY 'carbondata'
    +           | TBLPROPERTIES('LONG_STRING_COLUMNS'='address, note, Note')
    +           |""".
    +          stripMargin)
    +    }
    +    assert(exceptionCaught.getMessage.contains("long_string_columns"))
    +    assert(exceptionCaught.getMessage.contains("Duplicate columns are not allowed"))
    +  }
    +
    +  test("long_string_columns: column does not exist in table ") {
    +    val exceptionCaught = intercept[Exception] {
    +      sql(
    +        s"""
    +           | CREATE TABLE if not exists $longStringTable(
    +           | id INT, name STRING, description STRING, address STRING, note STRING
    +           | ) STORED BY 'carbondata'
    +           | TBLPROPERTIES('LONG_STRING_COLUMNS'='address, note, NoteS')
    +           |""".
    +          stripMargin)
    +    }
    +    assert(exceptionCaught.getMessage.contains("long_string_columns"))
    +    assert(exceptionCaught.getMessage.contains("does not exist in table"))
    +  }
    +
    +  test("long_string_columns: columns cannot exist in patitions columns") {
    +    val exceptionCaught = intercept[Exception] {
    --- End diff --
    
    please be more specific about the Exception, do not just intercept all Exception


---

[GitHub] carbondata pull request #2436: [CARBONDATA-2682][32K] fix create table with ...

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

    https://github.com/apache/carbondata/pull/2436#discussion_r201542881
  
    --- Diff: integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/longstring/VarcharDataTypesBasicTestCase.scala ---
    @@ -133,6 +133,64 @@ class VarcharDataTypesBasicTestCase extends QueryTest with BeforeAndAfterEach wi
         assert(exceptionCaught.getMessage.contains("its data type is not string"))
       }
     
    +  test("long string columns cannot contain duplicate columns") {
    +    val exceptionCaught = intercept[Exception] {
    +      sql(
    +        s"""
    +           | CREATE TABLE if not exists $longStringTable(
    +           | id INT, name STRING, description STRING, address STRING, note STRING
    +           | ) STORED BY 'carbondata'
    +           | TBLPROPERTIES('LONG_STRING_COLUMNS'='address, note, Note')
    +           |""".
    +          stripMargin)
    +    }
    +    assert(exceptionCaught.getMessage.contains("long_string_columns"))
    +    assert(exceptionCaught.getMessage.contains("Duplicate columns are not allowed"))
    +  }
    +
    +  test("long_string_columns: column does not exist in table ") {
    +    val exceptionCaught = intercept[Exception] {
    --- End diff --
    
    ok


---

[GitHub] carbondata issue #2436: [CARBONDATA-2682]fix create table with long_string_c...

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

    https://github.com/apache/carbondata/pull/2436
  
    @jackylk can you help to review this


---

[GitHub] carbondata pull request #2436: [CARBONDATA-2682][32K] fix create table with ...

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

    https://github.com/apache/carbondata/pull/2436#discussion_r201544064
  
    --- Diff: integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala ---
    @@ -486,6 +504,60 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
         }
       }
     
    +  /**
    +   * This method validates the long string columns
    +   *
    +   * @param fields
    --- End diff --
    
    ok


---

[GitHub] carbondata pull request #2436: [CARBONDATA-2682][32K] fix create table with ...

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

    https://github.com/apache/carbondata/pull/2436#discussion_r200898371
  
    --- Diff: integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala ---
    @@ -486,6 +504,60 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
         }
       }
     
    +  /**
    +   * This method validates the long string columns
    +   *
    +   * @param fields
    --- End diff --
    
    please add comment for all parameters


---

[GitHub] carbondata issue #2436: [CARBONDATA-2682]fix create table with long_string_c...

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

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


---

[GitHub] carbondata pull request #2436: [CARBONDATA-2682][32K] fix create table with ...

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

    https://github.com/apache/carbondata/pull/2436#discussion_r200898644
  
    --- Diff: integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala ---
    @@ -486,6 +504,60 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
         }
       }
     
    +  /**
    +   * This method validates the long string columns
    +   *
    +   * @param fields
    +   * @param varcharCols
    +   * @return
    +   */
    +  private def validateLongStringColumns(fields: Seq[Field],
    +      varcharCols: Seq[String]): Unit = {
    +    var longStringColumnsMap: Map[String, Field] = Map[String, Field]()
    +    fields.foreach(field =>
    +      longStringColumnsMap.put(field.column.toUpperCase, field)
    +    )
    +    var dataTypeErr: Set[String] = Set[String]()
    +    var duplicateColumnErr: Map[String, Int] = Map[String, Int]()
    +    var nullColumnErr: Set[String] = Set[String]()
    +    var tmpStr: String = ""
    +    varcharCols.foreach {
    +      column =>
    +        tmpStr = column.toUpperCase
    +        duplicateColumnErr.get(tmpStr) match {
    +          case None => duplicateColumnErr.put(tmpStr, 1)
    +          case Some(count) => duplicateColumnErr.put(tmpStr, count + 1)
    +        }
    +        longStringColumnsMap.get(tmpStr) match {
    +          case None => nullColumnErr += column
    +          case Some(field) => if (!DataTypes.STRING.getName.equalsIgnoreCase(field.dataType.get)) {
    +            dataTypeErr += column
    +          }
    +        }
    +    }
    +    if (!nullColumnErr.isEmpty) {
    +      val errMsg = "long_string_columns:" +
    +                   nullColumnErr.mkString(",") +
    +                   " does not exist in table. Please check create table statement."
    +      throw new MalformedCarbonCommandException(errMsg)
    +    }
    +
    +    var duplicateColumns = duplicateColumnErr.filter(kv => kv._2 != 1).keySet
    --- End diff --
    
    I feel these logic can be optimized, can you describe what validation is done in this function?


---

[GitHub] carbondata pull request #2436: [CARBONDATA-2682][32K] fix create table with ...

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

    https://github.com/apache/carbondata/pull/2436#discussion_r201542863
  
    --- Diff: integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/longstring/VarcharDataTypesBasicTestCase.scala ---
    @@ -133,6 +133,64 @@ class VarcharDataTypesBasicTestCase extends QueryTest with BeforeAndAfterEach wi
         assert(exceptionCaught.getMessage.contains("its data type is not string"))
       }
     
    +  test("long string columns cannot contain duplicate columns") {
    +    val exceptionCaught = intercept[Exception] {
    --- End diff --
    
    ok


---