You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@carbondata.apache.org by anubhav100 <gi...@git.apache.org> on 2017/03/07 11:25:44 UTC

[GitHub] incubator-carbondata pull request #630: [CARBONDATA-730]added decimal type i...

GitHub user anubhav100 opened a pull request:

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

    [CARBONDATA-730]added decimal type in carbon data frame writer

    Below exception is thrown while trying to save dataframe with a decimal column type.
    
    scala> df.printSchema
    \u2013 account: integer (nullable = true)
    \u2013 currency: integer (nullable = true)
    \u2013 branch: integer (nullable = true)
    \u2013 country: integer (nullable = true)
    \u2013 date: date (nullable = true)
    \u2013 fcbalance: decimal(16,3) (nullable = true)
    \u2013 lcbalance: decimal(16,3) (nullable = true)
    
    scala> df.write.format("carbondata").option("tableName", "accBal").option("compress", "true").mode(SaveMode.Overwrite).save()
    
    java.lang.RuntimeException: unsupported type: DecimalType(16,3)
    at scala.sys.package$.error(package.scala:27)
               


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

    $ git pull https://github.com/anubhav100/incubator-carbondata CARBONDATA-730

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

    https://github.com/apache/incubator-carbondata/pull/630.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 #630
    
----
commit acd67332af60ad78dd93c1897ad31af4621c237e
Author: anubhav100 <an...@knoldus.in>
Date:   2017-03-06T12:39:55Z

    added decimal type in carbon data frame writer

----


---
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 #630: [CARBONDATA-730] added decimal type ...

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

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


---
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 #630: [CARBONDATA-730] added decimal type in carb...

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

    https://github.com/apache/incubator-carbondata/pull/630
  
    @jackylk i think the ci  issue is resolved that was because of test case failure in spark package 


---
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 #630: [CARBONDATA-730] added decimal type in carb...

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

    https://github.com/apache/incubator-carbondata/pull/630
  
    @jackylk all changes done can you review please?


---
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 #630: [CARBONDATA-730] added decimal type ...

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/630#discussion_r105967586
  
    --- Diff: integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataframe/Decimal_DataFrameTest.scala ---
    @@ -0,0 +1,62 @@
    +/*
    + * 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.integration.spark.testsuite.dataframe
    +
    +import java.math.BigDecimal
    +
    +import org.apache.spark.sql.common.util.DataFrameUtils
    +import org.apache.spark.sql.types.{DecimalType, DoubleType, StringType, StructField, StructType}
    +import org.apache.spark.sql.{Row, SaveMode}
    +import org.scalatest.FunSuiteLike
    +
    +class Decimal_DataFrameTest extends FunSuiteLike {
    +
    +  val carbonContext = DataFrameUtils.createCarbonContext("Decimal_DataFrameTest")
    +
    +  carbonContext.sql("DROP TABLE IF EXISTS carbon3")
    +
    +  val rdd = carbonContext.sc.parallelize(
    +    Row(52.23, BigDecimal.valueOf(1234.4440), "Warsaw") ::
    +    Row(42.30, BigDecimal.valueOf(9999.9990), "Corte") :: Nil)
    +
    +  val schema = StructType(
    +    StructField("double", DoubleType, nullable = false) ::
    +    StructField("decimal", DecimalType(9, 2), nullable = false) ::
    +    StructField("string", StringType, nullable = false) :: Nil)
    +
    +  val dataFrame = carbonContext.createDataFrame(rdd, schema)
    +
    +  dataFrame.printSchema()
    +
    +  dataFrame.write
    +    .format("carbondata")
    +    .option("tableName", "carbon3")
    +    .option("compress", "true")
    +    .mode(SaveMode.Overwrite)
    +    .save()
    +
    +  test("select salary from table") {
    +    val actualDataFrame = carbonContext.sql("SELECT decimal FROM carbon3")
    +    actualDataFrame.show()
    +    val actualResult = actualDataFrame.collect().toList
    +    val expectedResult = List(Row(10000.00) :: Nil, Row(1234.44) :: Nil)
    +    assert((actualResult diff expectedResult).isEmpty)
    +
    --- End diff --
    
    drop the table created by dataframe.write also


---
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 #630: [CARBONDATA-730]added decimal type in carbo...

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

    https://github.com/apache/incubator-carbondata/pull/630
  
    @jackylk 
    
    i tried this with decimal value 
    val employee = List(Employee("anubhav", 20000.5))
    val employeeRDD = cc.sc.parallelize(employee)
    val employeeDataFrame = employeeRDD.toDF("name", "salary")
    
    employeeDataFrame.printSchema()
    employeeDataFrame.write
      .format("carbondata")
      .option("tableName", "carbon3")
      .option("compress", "true")
      .mode(SaveMode.Overwrite)
      .save()
    
    
    
    cc.sql("SELECT * FROM carbon3").show()
    
    |   name|salary|
    +-------+------+
    |anubhav| 20001|
    +-------+------+
    
    it seems like it is rounding off the values can you suggest something



---
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 #630: [CARBONDATA-730]added decimal type in carbo...

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

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



---
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 #630: [CARBONDATA-730] added decimal type in carb...

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

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



---
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 #630: [CARBONDATA-730]added decimal type i...

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/630#discussion_r105579644
  
    --- Diff: examples/spark/src/main/scala/org/apache/carbondata/examples/CaseClassDataFrameAPIExample.scala ---
    @@ -42,8 +45,25 @@ object CaseClassDataFrameAPIExample {
           .mode(SaveMode.Overwrite)
           .save()
     
    +    val employee = List(Employee("anubhav", 20000))
    --- End diff --
    
    And can you add a testcase for both spark1 and spark2 integration?


---
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 #630: [CARBONDATA-730] added decimal type in carb...

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

    https://github.com/apache/incubator-carbondata/pull/630
  
    LGTM
    please fix CI failing


---
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 #630: [CARBONDATA-730]added decimal type in carbo...

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

    https://github.com/apache/incubator-carbondata/pull/630
  
    @jackylk i added a example in  CaseClassDataFrameAPIExample,i think its working correct
    
    case class Employee(name: String, salary: BigDecimal)
    
     val employee = List(Employee("anubhav", 20000))
        val employeeRDD = cc.sc.parallelize(employee)
        val employeeDataFrame = employeeRDD.toDF("name", "salary")
    
        employeeDataFrame.printSchema()
        employeeDataFrame.write
          .format("carbondata")
          .option("tableName", "carbon3")
          .option("compress", "true")
          .mode(SaveMode.Overwrite)
          .save()
    
     cc.sql("SELECT * FROM carbon2").show()
    
    here is result
    
    +-------+------+
    |   name|salary|
    +-------+------+
    |anubhav| 20000|
    +-------+------+


---
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 #630: [CARBONDATA-730]added decimal type i...

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/630#discussion_r105579596
  
    --- Diff: examples/spark/src/main/scala/org/apache/carbondata/examples/CaseClassDataFrameAPIExample.scala ---
    @@ -42,8 +45,25 @@ object CaseClassDataFrameAPIExample {
           .mode(SaveMode.Overwrite)
           .save()
     
    +    val employee = List(Employee("anubhav", 20000))
    --- End diff --
    
    I tried with `    val employee = List(Employee("anubhav", BigDecimal(324234353, 4)))`
    the output is:
    ```
    +-------+------+
    |   name|salary|
    +-------+------+
    |anubhav| 32423|
    +-------+------+
    ```
    It is not handling correctly


---
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 #630: [CARBONDATA-730] added decimal type ...

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/630#discussion_r105967226
  
    --- Diff: integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataframe/Decimal_DataFrameTest.scala ---
    @@ -0,0 +1,62 @@
    +/*
    + * 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.integration.spark.testsuite.dataframe
    +
    +import java.math.BigDecimal
    +
    +import org.apache.spark.sql.common.util.DataFrameUtils
    +import org.apache.spark.sql.types.{DecimalType, DoubleType, StringType, StructField, StructType}
    +import org.apache.spark.sql.{Row, SaveMode}
    +import org.scalatest.FunSuiteLike
    +
    +class Decimal_DataFrameTest extends FunSuiteLike {
    +
    +  val carbonContext = DataFrameUtils.createCarbonContext("Decimal_DataFrameTest")
    +
    +  carbonContext.sql("DROP TABLE IF EXISTS carbon3")
    +
    +  val rdd = carbonContext.sc.parallelize(
    +    Row(52.23, BigDecimal.valueOf(1234.4440), "Warsaw") ::
    +    Row(42.30, BigDecimal.valueOf(9999.9990), "Corte") :: Nil)
    +
    +  val schema = StructType(
    +    StructField("double", DoubleType, nullable = false) ::
    +    StructField("decimal", DecimalType(9, 2), nullable = false) ::
    +    StructField("string", StringType, nullable = false) :: Nil)
    +
    +  val dataFrame = carbonContext.createDataFrame(rdd, schema)
    +
    +  dataFrame.printSchema()
    +
    +  dataFrame.write
    --- End diff --
    
    put the dataframe creation and write inside the test 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 #630: [CARBONDATA-730] added decimal type ...

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

    https://github.com/apache/incubator-carbondata/pull/630#discussion_r105830283
  
    --- Diff: examples/spark/src/main/scala/org/apache/carbondata/examples/CaseClassDataFrameAPIExample.scala ---
    @@ -42,8 +45,25 @@ object CaseClassDataFrameAPIExample {
           .mode(SaveMode.Overwrite)
           .save()
     
    +    val employee = List(Employee("anubhav", 20000))
    --- End diff --
    
    @jackylk i added test cases in integration package for both spark version can you review?


---
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 #630: [CARBONDATA-730]added decimal type i...

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

    https://github.com/apache/incubator-carbondata/pull/630#discussion_r105590219
  
    --- Diff: examples/spark/src/main/scala/org/apache/carbondata/examples/CaseClassDataFrameAPIExample.scala ---
    @@ -42,8 +45,25 @@ object CaseClassDataFrameAPIExample {
           .mode(SaveMode.Overwrite)
           .save()
     
    +    val employee = List(Employee("anubhav", 20000))
    --- End diff --
    
    @anubhav100 @jackylk  I am new to carbondata, but any reason for not adding the precision as below? 
    
    case dt: DecimalType => s"${CarbonType.DECIMAL.getName}(${dt.precision}, ${dt.scale})


---
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 #630: [CARBONDATA-730] added decimal type ...

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/630#discussion_r105968595
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDataFrameWriter.scala ---
    @@ -34,12 +34,37 @@ class CarbonDataFrameWriter(sqlContext: SQLContext, val dataFrame: DataFrame) {
       def saveAsCarbonFile(parameters: Map[String, String] = Map()): Unit = {
         // create a new table using dataframe's schema and write its content into the table
         sqlContext.sparkSession.sql(makeCreateTableString(dataFrame.schema,
    -    new CarbonOption(parameters)))
    +      new CarbonOption(parameters)))
         writeToCarbonFile(parameters)
       }
     
    -  def appendToCarbonFile(parameters: Map[String, String] = Map()): Unit = {
    -    writeToCarbonFile(parameters)
    +  private def makeCreateTableString(schema: StructType, options: CarbonOption): String = {
    --- End diff --
    
    It seems this function is moved from line 144 to here? It is better not to move it so that it is easier to review


---
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 #630: [CARBONDATA-730]added decimal type i...

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

    https://github.com/apache/incubator-carbondata/pull/630#discussion_r105590522
  
    --- Diff: examples/spark/src/main/scala/org/apache/carbondata/examples/CaseClassDataFrameAPIExample.scala ---
    @@ -42,8 +45,25 @@ object CaseClassDataFrameAPIExample {
           .mode(SaveMode.Overwrite)
           .save()
     
    +    val employee = List(Employee("anubhav", 20000))
    --- End diff --
    
    No I think it's alright I changed my code as u suggested and when I tested it ,it is working correctly, I am adding the test cases and will be pushing it 


---
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 #630: [CARBONDATA-730]added decimal type in carbo...

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

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



---
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 #630: [CARBONDATA-730] added decimal type ...

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/630#discussion_r105968680
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDataFrameWriter.scala ---
    @@ -34,12 +34,37 @@ class CarbonDataFrameWriter(sqlContext: SQLContext, val dataFrame: DataFrame) {
       def saveAsCarbonFile(parameters: Map[String, String] = Map()): Unit = {
         // create a new table using dataframe's schema and write its content into the table
         sqlContext.sparkSession.sql(makeCreateTableString(dataFrame.schema,
    -    new CarbonOption(parameters)))
    +      new CarbonOption(parameters)))
         writeToCarbonFile(parameters)
       }
     
    -  def appendToCarbonFile(parameters: Map[String, String] = Map()): Unit = {
    -    writeToCarbonFile(parameters)
    +  private def makeCreateTableString(schema: StructType, options: CarbonOption): String = {
    +    val carbonSchema = schema.map { field =>
    +      s"${ field.name } ${ convertToCarbonType(field.dataType) }"
    +    }
    +    s"""
    +          CREATE TABLE IF NOT EXISTS ${ options.dbName }.${ options.tableName }
    +          (${ carbonSchema.mkString(", ") })
    +          using org.apache.spark.sql.CarbonSource
    +          OPTIONS('dbName'='${ options.dbName }', 'tableName'='${ options.tableName }')
    +      """
    +  }
    +
    +  private def convertToCarbonType(sparkType: DataType): String = {
    --- End diff --
    
    It seems this function is moved from line 144 to here? It is better not to move it so that it is easier to review


---
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 #630: [CARBONDATA-730]added decimal type in carbo...

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

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



---
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 #630: [CARBONDATA-730]added decimal type in carbo...

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

    https://github.com/apache/incubator-carbondata/pull/630
  
    You can use the code snippet that I had pasted in the JIRA issue. As I mentioned it resolved my issue. 


---
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 #630: [CARBONDATA-730]added decimal type in carbo...

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

    https://github.com/apache/incubator-carbondata/pull/630
  
    Can you add a testcase to write datafame containing a decimal column and query it? I think it needs more implementation for load and read


---
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 #630: [CARBONDATA-730] added decimal type in carb...

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

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



---
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 #630: [CARBONDATA-730] added decimal type ...

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/630#discussion_r105966957
  
    --- Diff: integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataframe/Decimal_DataFrameTest.scala ---
    @@ -0,0 +1,62 @@
    +/*
    + * 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.integration.spark.testsuite.dataframe
    +
    +import java.math.BigDecimal
    +
    +import org.apache.spark.sql.common.util.DataFrameUtils
    +import org.apache.spark.sql.types.{DecimalType, DoubleType, StringType, StructField, StructType}
    +import org.apache.spark.sql.{Row, SaveMode}
    +import org.scalatest.FunSuiteLike
    +
    +class Decimal_DataFrameTest extends FunSuiteLike {
    --- End diff --
    
    Please add it in `TestLoadDataFrame` instead of creating a new class


---
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 #630: [CARBONDATA-730] added decimal type ...

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/630#discussion_r105967481
  
    --- Diff: integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataframe/Decimal_DataFrameTest.scala ---
    @@ -0,0 +1,62 @@
    +/*
    + * 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.integration.spark.testsuite.dataframe
    +
    +import java.math.BigDecimal
    +
    +import org.apache.spark.sql.common.util.DataFrameUtils
    +import org.apache.spark.sql.types.{DecimalType, DoubleType, StringType, StructField, StructType}
    +import org.apache.spark.sql.{Row, SaveMode}
    +import org.scalatest.FunSuiteLike
    +
    +class Decimal_DataFrameTest extends FunSuiteLike {
    +
    +  val carbonContext = DataFrameUtils.createCarbonContext("Decimal_DataFrameTest")
    +
    +  carbonContext.sql("DROP TABLE IF EXISTS carbon3")
    +
    +  val rdd = carbonContext.sc.parallelize(
    +    Row(52.23, BigDecimal.valueOf(1234.4440), "Warsaw") ::
    +    Row(42.30, BigDecimal.valueOf(9999.9990), "Corte") :: Nil)
    +
    +  val schema = StructType(
    +    StructField("double", DoubleType, nullable = false) ::
    +    StructField("decimal", DecimalType(9, 2), nullable = false) ::
    +    StructField("string", StringType, nullable = false) :: Nil)
    +
    +  val dataFrame = carbonContext.createDataFrame(rdd, schema)
    +
    +  dataFrame.printSchema()
    +
    +  dataFrame.write
    +    .format("carbondata")
    +    .option("tableName", "carbon3")
    +    .option("compress", "true")
    +    .mode(SaveMode.Overwrite)
    +    .save()
    +
    +  test("select salary from table") {
    +    val actualDataFrame = carbonContext.sql("SELECT decimal FROM carbon3")
    +    actualDataFrame.show()
    +    val actualResult = actualDataFrame.collect().toList
    +    val expectedResult = List(Row(10000.00) :: Nil, Row(1234.44) :: Nil)
    +    assert((actualResult diff expectedResult).isEmpty)
    --- End diff --
    
    use `checkAnswer` like in the `TestLoadDataFrame`


---
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 #630: [CARBONDATA-730]added decimal type in carbo...

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

    https://github.com/apache/incubator-carbondata/pull/630
  
    I think adding test case is better instead of example as it can be verified for every build
    And please verify declaring decimal with precision and scale parameter


---
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 #630: [CARBONDATA-730] added decimal type in carb...

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

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



---
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 #630: [CARBONDATA-730] added decimal type in carb...

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

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



---
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 #630: [CARBONDATA-730] added decimal type ...

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/630#discussion_r105967718
  
    --- Diff: integration/spark/src/test/scala/org/apache/spark/sql/common/util/DataFrameUtils.scala ---
    @@ -0,0 +1,43 @@
    +package org.apache.spark.sql.common.util
    +
    +import java.io.File
    +
    +import org.apache.spark.sql.CarbonContext
    +import org.apache.spark.{SparkConf, SparkContext}
    +
    +import org.apache.carbondata.core.util.CarbonProperties
    +
    +
    +object DataFrameUtils {
    --- End diff --
    
    I think this is not required, if you reuse `TestLoadDataFrame`


---
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 #630: [CARBONDATA-730] added decimal type in carb...

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

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



---
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 #630: [CARBONDATA-730] added decimal type in carb...

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

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



---
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 #630: [CARBONDATA-730]added decimal type i...

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

    https://github.com/apache/incubator-carbondata/pull/630#discussion_r105582287
  
    --- Diff: examples/spark/src/main/scala/org/apache/carbondata/examples/CaseClassDataFrameAPIExample.scala ---
    @@ -42,8 +45,25 @@ object CaseClassDataFrameAPIExample {
           .mode(SaveMode.Overwrite)
           .save()
     
    +    val employee = List(Employee("anubhav", 20000))
    --- End diff --
    
    I change the code as suggested in code snippet,earlier I am not giving precision and scale factor,ya sure I am adding test cases


---
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 #630: [CARBONDATA-730]added decimal type in carbo...

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

    https://github.com/apache/incubator-carbondata/pull/630
  
    please change the title as per the format: [CARBONDATA-issue number>] Title of the pull request (need to add a blank)


---
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 #630: [CARBONDATA-730]added decimal type in carbo...

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

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



---
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 #630: [CARBONDATA-730] added decimal type ...

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/630#discussion_r105968216
  
    --- Diff: integration/spark2/src/test/scala/org/apache/spark/carbondata/dataframe/Decimal_DataFrameTest.scala ---
    @@ -0,0 +1,59 @@
    +/*
    + * 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.spark.carbondata.dataframe
    +
    +import org.apache.spark.sql.types.{DecimalType, DoubleType, StringType, StructField, StructType}
    +import org.apache.spark.sql.{Row, SaveMode}
    +import org.apache.spark.util.DataFrameUtils
    +import org.scalatest.FunSuiteLike
    +
    +class Decimal_DataFrameTest extends FunSuiteLike {
    --- End diff --
    
    This is not required, resuse `TestLoadDataFrame` in spark-common-test module


---
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 #630: [CARBONDATA-730] added decimal type in carb...

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

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



---
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 #630: [CARBONDATA-730] added decimal type in carb...

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

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



---
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.
---