[GitHub] incubator-carbondata pull request #624: [WIP] Add performance test

classic Classic list List threaded Threaded
34 messages Options
12
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata pull request #624: [WIP] Add performance test

qiuchenjian-2
GitHub user jackylk opened a pull request:

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

    [WIP] Add performance test

    1. add a CompareTest for spark-2.1 integration
    2. add single pass support for dataframe.write to carbon

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

    $ git pull https://github.com/jackylk/incubator-carbondata comparetest

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

    https://github.com/apache/incubator-carbondata/pull/624.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 #624
   
----
commit 394112f72c87375965a279762d743dc279695f0d
Author: jackylk <[hidden email]>
Date:   2017-03-04T02:57:31Z

    add compareTest

commit 07aa5f98e4cfb6021913ab671d66a48a5420ee19
Author: jackylk <[hidden email]>
Date:   2017-03-04T15:17:40Z

    change compare test

commit ea6e88567ee139954cb7a4303e5c2cf13316f83b
Author: jackylk <[hidden email]>
Date:   2017-03-05T16:02:27Z

    add singlepass

----


---
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 [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata issue #624: [WIP] Add simple performance test for spark...

qiuchenjian-2
Github user CarbonDataQA commented on the issue:

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



---
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 [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata issue #624: [WIP] Add simple performance test for spark...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user CarbonDataQA commented on the issue:

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



---
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 [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata pull request #624: [CARBONDATA-747][WIP] Add simple per...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user jarray888 commented on a diff in the pull request:

    https://github.com/apache/incubator-carbondata/pull/624#discussion_r104369679
 
    --- Diff: examples/spark2/src/main/scala/org/apache/carbondata/examples/CompareTest.scala ---
    @@ -0,0 +1,122 @@
    +/*
    + * 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.examples
    +
    +import org.apache.spark.sql.{DataFrame, Row, SaveMode, SparkSession}
    +
    +import org.apache.carbondata.core.util.CarbonProperties
    +
    +// scalastyle:off println
    +object CompareTest {
    +
    +  val parquetTableName = "comparetest_parquet"
    +  val carbonTableName = "comparetest_carbon"
    +
    +  private def generateDataFrame(spark: SparkSession): DataFrame = {
    +    import spark.implicits._
    +    spark.sparkContext.parallelize(1 to 10 * 1000 * 1000, 4)
    +        .map(x => ("i" + x, "p" + x % 10, "j" + x % 100, x, x + 1, (x + 7) % 21, (x + 5) / 43, x
    +            * 5))
    +        .toDF("id", "country", "city", "c4", "c5", "c6", "c7", "c8")
    +  }
    +
    +  private def loadParquetTable(spark: SparkSession, input: DataFrame): Long = timeit {
    +    input.write.mode(SaveMode.Overwrite).parquet(parquetTableName)
    +  }
    +
    +  private def loadCarbonTable(spark: SparkSession, input: DataFrame): Long = {
    +    spark.sql(s"drop table if exists $carbonTableName")
    +    timeit {
    +      input.write
    +          .format("carbondata")
    +          .option("tableName", carbonTableName)
    +          .option("tempCSV", "false")
    +          .option("single_pass", "true")
    +          .option("dictionary_exclude", "id") // id is high cardinality column
    +          .mode(SaveMode.Overwrite)
    +          .save()
    +    }
    +  }
    +
    +  private def prepareTable(spark: SparkSession): Unit = {
    +    val df = generateDataFrame(spark).cache()
    +    println(s"loading dataframe into table, schema: ${df.schema}")
    +    val loadParquetTime = loadParquetTable(spark, df)
    +    val loadCarbonTime = loadCarbonTable(spark, df)
    +    println(s"load completed, time: $loadParquetTime, $loadCarbonTime")
    +    spark.read.parquet(parquetTableName).registerTempTable(parquetTableName)
    +  }
    +
    +  private def runQuery(spark: SparkSession): Unit = {
    +    val test = Array(
    +      "select count(*) from $table",
    +      "select sum(c4) from $table",
    +      "select sum(c4), sum(c5) from $table",
    +      "select sum(c4), sum(c5), sum(c6) from $table",
    +      "select sum(c4), sum(c5), sum(c6), sum(c7) from $table",
    +      "select sum(c4), sum(c5), sum(c6), sum(c7), avg(c8) from $table",
    +      "select * from $table where id = 'i9999999' ",
    +      "select * from $table where country = 'p9' ",
    +      "select * from $table where city = 'j99' ",
    +      "select * from $table where c4 < 1000 "
    --- End diff --
   
    please add more testcase, for example:
     "select sum(c4) from $table where id like 'i1%' "
     "select sum(c4) from $table where id like '%10' "
     "select sum(c4) from $table where id like '%xyz%' "



---
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 [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata pull request #624: [CARBONDATA-747][WIP] Add simple per...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user jarray888 commented on a diff in the pull request:

    https://github.com/apache/incubator-carbondata/pull/624#discussion_r104369883
 
    --- Diff: examples/spark2/src/main/scala/org/apache/carbondata/examples/CompareTest.scala ---
    @@ -0,0 +1,122 @@
    +/*
    + * 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.examples
    +
    +import org.apache.spark.sql.{DataFrame, Row, SaveMode, SparkSession}
    +
    +import org.apache.carbondata.core.util.CarbonProperties
    +
    +// scalastyle:off println
    +object CompareTest {
    +
    +  val parquetTableName = "comparetest_parquet"
    +  val carbonTableName = "comparetest_carbon"
    +
    +  private def generateDataFrame(spark: SparkSession): DataFrame = {
    +    import spark.implicits._
    +    spark.sparkContext.parallelize(1 to 10 * 1000 * 1000, 4)
    +        .map(x => ("i" + x, "p" + x % 10, "j" + x % 100, x, x + 1, (x + 7) % 21, (x + 5) / 43, x
    +            * 5))
    +        .toDF("id", "country", "city", "c4", "c5", "c6", "c7", "c8")
    +  }
    +
    +  private def loadParquetTable(spark: SparkSession, input: DataFrame): Long = timeit {
    +    input.write.mode(SaveMode.Overwrite).parquet(parquetTableName)
    --- End diff --
   
    suggest to use last char of id column to do partition on parquet, so the comparison is fare.


---
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 [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata pull request #624: [CARBONDATA-747][WIP] Add simple per...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user jarray888 commented on a diff in the pull request:

    https://github.com/apache/incubator-carbondata/pull/624#discussion_r104370904
 
    --- Diff: examples/spark2/src/main/scala/org/apache/carbondata/examples/CompareTest.scala ---
    @@ -0,0 +1,122 @@
    +/*
    + * 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.examples
    +
    +import org.apache.spark.sql.{DataFrame, Row, SaveMode, SparkSession}
    +
    +import org.apache.carbondata.core.util.CarbonProperties
    +
    +// scalastyle:off println
    +object CompareTest {
    +
    +  val parquetTableName = "comparetest_parquet"
    +  val carbonTableName = "comparetest_carbon"
    +
    +  private def generateDataFrame(spark: SparkSession): DataFrame = {
    +    import spark.implicits._
    +    spark.sparkContext.parallelize(1 to 10 * 1000 * 1000, 4)
    +        .map(x => ("i" + x, "p" + x % 10, "j" + x % 100, x, x + 1, (x + 7) % 21, (x + 5) / 43, x
    --- End diff --
   
    To simulate a real-life data, please make the data unsorted, like
    `map(x => ("i" + randon number, "p" + x % 13, "j" + x % 97, ...)`


---
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 [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata pull request #624: [CARBONDATA-747][WIP] Add simple per...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user jarray888 commented on a diff in the pull request:

    https://github.com/apache/incubator-carbondata/pull/624#discussion_r104374666
 
    --- Diff: examples/spark2/src/main/scala/org/apache/carbondata/examples/CompareTest.scala ---
    @@ -0,0 +1,122 @@
    +/*
    + * 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.examples
    +
    +import org.apache.spark.sql.{DataFrame, Row, SaveMode, SparkSession}
    +
    +import org.apache.carbondata.core.util.CarbonProperties
    +
    +// scalastyle:off println
    +object CompareTest {
    +
    +  val parquetTableName = "comparetest_parquet"
    +  val carbonTableName = "comparetest_carbon"
    +
    +  private def generateDataFrame(spark: SparkSession): DataFrame = {
    +    import spark.implicits._
    +    spark.sparkContext.parallelize(1 to 10 * 1000 * 1000, 4)
    +        .map(x => ("i" + x, "p" + x % 10, "j" + x % 100, x, x + 1, (x + 7) % 21, (x + 5) / 43, x
    +            * 5))
    +        .toDF("id", "country", "city", "c4", "c5", "c6", "c7", "c8")
    --- End diff --
   
    can you add a column using decimal data type?


---
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 [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata issue #624: [CARBONDATA-747][WIP] Add simple performanc...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user CarbonDataQA commented on the issue:

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



---
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 [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata issue #624: [CARBONDATA-747][WIP] Add simple performanc...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user CarbonDataQA commented on the issue:

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



---
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 [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata pull request #624: [CARBONDATA-747][WIP] Add simple per...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user chenliang613 commented on a diff in the pull request:

    https://github.com/apache/incubator-carbondata/pull/624#discussion_r104542895
 
    --- Diff: examples/spark2/src/main/scala/org/apache/carbondata/examples/CompareTest.scala ---
    @@ -0,0 +1,347 @@
    +/*
    + * 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.examples
    +
    +import org.apache.spark.sql.{DataFrame, Row, SaveMode, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +import org.apache.carbondata.core.constants.CarbonCommonConstants
    +import org.apache.carbondata.core.util.CarbonProperties
    +
    +/**
    + * A query test case
    + * @param sqlText SQL statement
    + * @param queryType type of query: scan, filter, aggregate, topN
    + * @param desc description of the goal of this test case
    + */
    +case class Query(sqlText: String, queryType: String, desc: String)
    +
    +// scalastyle:off println
    +object CompareTest {
    +
    +  def parquetTableName: String = "comparetest_parquet"
    +  def carbonTableName(version: String): String = s"comparetest_carbonV$version"
    +
    +  // Table schema:
    +  // +-------------+-----------+-------------+-------------+------------+
    +  // | Column name | Data type | Cardinality | Column type | Dictionary |
    +  // +-------------+-----------+-------------+-------------+------------+
    +  // | id          | string    | 10,000,000  | dimension   | no         |
    +  // +-------------+-----------+-------------+-------------+------------+
    +  // | country     | string    | 1103        | dimension   | yes        |
    +  // +-------------+-----------+-------------+-------------+------------+
    +  // | city        | string    | 13          | dimension   | yes        |
    +  // +-------------+-----------+-------------+-------------+------------+
    +  // | c4          | short     | NA          | measure     | no         |
    +  // +-------------+-----------+-------------+-------------+------------+
    +  // | c5          | int       | NA          | measure     | no         |
    +  // +-------------+-----------+-------------+-------------+------------+
    +  // | c6          | big int   | NA          | measure     | no         |
    +  // +-------------+-----------+-------------+-------------+------------+
    +  // | c7          | double    | NA          | measure     | no         |
    +  // +-------------+-----------+-------------+-------------+------------+
    +  // | c8          | double    | NA          | measure     | no         |
    +  // +-------------+-----------+-------------+-------------+------------+
    +  private def generateDataFrame(spark: SparkSession): DataFrame = {
    +    val rdd = spark.sparkContext
    +        .parallelize(1 to 10 * 1000 * 1000, 4)
    +        .map { x =>
    +          (x.toString, "p" + x % 1103, "j" + x % 13, (x % 31).toShort, x, x.toLong * 1000,
    +              x.toDouble / 13, x.toDouble / 71 )
    +        }.map { x =>
    +          Row(x._1, x._2, x._3, x._4, x._5, x._6, x._7, x._8)
    +        }
    +    val schema = StructType(
    +      Seq(
    +        StructField("id", StringType, nullable = false),
    +        StructField("country", StringType, nullable = false),
    +        StructField("city", StringType, nullable = false),
    +        StructField("c4", ShortType, nullable = true),
    +        StructField("c5", IntegerType, nullable = true),
    +        StructField("c6", LongType, nullable = true),
    +        StructField("c7", DoubleType, nullable = true),
    +        StructField("c8", DoubleType, nullable = true)
    +      )
    +    )
    +    spark.createDataFrame(rdd, schema)
    +  }
    +
    +  // performance test queries
    +  val queries: Array[Query] = Array(
    +    Query(
    +      "select count(*) from $table",
    +      "warm up",
    +      "warm up query"
    +    ),
    +    // ===========================================================================
    +    // ==                               FULL SCAN                               ==
    +    // ===========================================================================
    +    Query(
    +      "select sum(c4) from $table",
    +      "full scan",
    +      "full scan query, 1 aggregate"
    +    ),
    +    Query(
    +      "select sum(c4), sum(c5) from $table",
    +      "full scan",
    +      "full scan query, 2 aggregate"
    +    ),
    +    Query(
    +      "select sum(c4), sum(c5), sum(c6) from $table",
    +      "full scan",
    +      "full scan query, 3 aggregate"
    +    ),
    +    Query(
    +      "select sum(c4), sum(c5), sum(c6), sum(c7) from $table",
    +      "full scan",
    +      "full scan query, 4 aggregate"
    +    ),
    +    Query(
    +      "select sum(c4), sum(c5), sum(c6), sum(c7), avg(c8) from $table",
    +      "full scan",
    +      "full scan query, 4 aggregate"
    +    ),
    +//    Query(
    +//      "select * from $table",
    +//      "full scan",
    +//      "full scan query, big result set"
    +//    ),
    +    Query(
    +      "select count(distinct id) from $table",
    +      "full scan",
    +      "full scan and count distinct of high card column"
    +    ),
    +    Query(
    +      "select count(distinct country) from $table",
    +      "full scan",
    +      "full scan and count distinct of medium card column"
    +    ),
    +    Query(
    +      "select count(distinct city) from $table",
    +      "full scan",
    +      "full scan and count distinct of low card column"
    +    ),
    +    // ===========================================================================
    +    // ==                               AGGREGATE                               ==
    +    // ===========================================================================
    +//    Query(
    +//      "select id, sum(c4) from $table group by id",
    +//      "aggregate",
    +//      "group by on big data, on high card column, big result set"
    +//    ),
    +    Query(
    +      "select country, sum(c4) from $table group by country",
    +      "aggregate",
    +      "group by on big data, on medium card column, medium result set,"
    +    ),
    +    Query(
    +      "select city, sum(c4) from $table group by city",
    +      "aggregate",
    +      "group by on big data, on low card column, small result set,"
    +    ),
    +    Query(
    +      "select id, sum(c4) as metric from $table group by id order by metric desc limit 100",
    +      "topN",
    +      "top N on high card column"
    +    ),
    +    Query(
    +      "select country,sum(c4) as metric from $table group by country order by metric desc limit 10",
    +      "topN",
    +      "top N on medium card column"
    +    ),
    +    Query(
    +      "select city,sum(c4) as metric from $table group by city order by metric desc limit 10",
    +      "topN",
    +      "top N on low card column"
    +    ),
    +    // ===========================================================================
    +    // ==                             FILTER SCAN                               ==
    +    // ===========================================================================
    +    Query(
    +      "select id, sum(c4) from $table where city='j9' group by id",
    +      "filter scan and aggregate",
    +      "group by on medium data, large result set"
    +    ),
    +    Query(
    +      "select country, sum(c4) from $table where city='j8' group by country ",
    +      "filter scan and aggregate",
    +      "group by on medium data, medium result set"
    +    ),
    +    Query(
    +      "select city, sum(c4) from $table where city='j7' group by city ",
    +      "filter scan and aggregate",
    +      "group by on medium data, small result set"
    +    ),
    +    Query(
    +      "select * from $table where id = '408938' ",
    +      "primary key filter",
    +      "primary key filtering"
    +    ),
    +    Query(
    +      "select * from $table where country = 'p99' ",
    +      "filter scan",
    +      "filter on c2, small result set, fetch all columns"
    +    ),
    +    Query(
    +      "select * from $table where country='p10000'  ",
    +      "filter scan",
    +      "filter on c2, not exist"
    +    ),
    +    Query(
    +      "select * from $table where city = 'j3' limit 100000",
    +      "filter scan",
    +      "filter on c3, medium result set, fetch all columns"
    +    ),
    +    Query(
    +      "select * from $table where country='p88' and city ='j8' ",
    +      "filter scan",
    +      "filter on c2 and c3, small result set, fetch all columns"
    +    ),
    +    Query(
    +      "select * from $table where id='1' and country='p1' and city ='j1' ",
    +      "filter scan",
    +      "filter on c1, c2 and c3, small result set, fetch all columns"
    +    ),
    +    Query(
    +      "select * from $table where c4 < 3",
    +      "filter scan",
    +      "filter on measure, small result set, fetch all columns"
    +    ),
    +    Query(
    +      "select * from $table where id like '1%' ",
    +      "fuzzy filter scan",
    +      "like filter, big result set"
    +    ),
    +    Query(
    +      "select * from $table where id like '%111'",
    +      "fuzzy filter scan",
    +      "like filter, medium result set"
    +    ),
    +    Query(
    +      "select * from $table where id like 'xyz%' ",
    +      "fuzzy filter scan",
    +      "like filter, full scan but not exist"
    +    )
    +  )
    +
    +  private def loadParquetTable(spark: SparkSession, input: DataFrame): Double = timeit {
    +    input.write.mode(SaveMode.Overwrite).parquet(parquetTableName)
    +  }
    +
    +  private def loadCarbonTable(spark: SparkSession, input: DataFrame, version: String): Double = {
    +    CarbonProperties.getInstance().addProperty(
    +      CarbonCommonConstants.CARBON_DATA_FILE_VERSION,
    +      version
    +    )
    +    spark.sql(s"drop table if exists ${carbonTableName(version)}")
    +    timeit {
    +      input.write
    +          .format("carbondata")
    +          .option("tableName", carbonTableName(version))
    +          .option("tempCSV", "false")
    +          .option("single_pass", "true")
    +          .option("dictionary_exclude", "id") // id is high cardinality column
    +          .mode(SaveMode.Overwrite)
    +          .save()
    +    }
    +  }
    +
    +  // load data into parquet, carbonV2, carbonV3
    +  private def prepareTable(spark: SparkSession): Unit = {
    +    val df = generateDataFrame(spark)
    +    println(s"loading ${df.count} records, schema: ${df.schema}")
    +    val loadParquetTime = loadParquetTable(spark, df)
    +    val loadCarbonV3Time = loadCarbonTable(spark, df, version = "3")
    +    val loadCarbonV2Time = loadCarbonTable(spark, df, version = "2")
    +    println(s"load completed, time: $loadParquetTime, $loadCarbonV3Time, $loadCarbonV2Time")
    +    spark.read.parquet(parquetTableName).registerTempTable(parquetTableName)
    +  }
    +
    +  private def runQuery(spark: SparkSession): Unit = {
    +    println("[")
    +    queries.zipWithIndex.foreach {
    +      case (query, index) =>
    +        var result1: Array[Row] = null
    +        var result2: Array[Row] = null
    +
    +        // query parquet, carbonV3, carbonV2, and carbonV2 usinga non-vector reader
    +        CarbonProperties.getInstance().addProperty("carbon.enable.vector.reader", "false")
    +        var sqlText = query.sqlText.replace("$table", carbonTableName("2"))
    +        val carbonV2NonVectorQueryTime = timeit {
    +          spark.sql(sqlText).collect()
    +        }
    +        CarbonProperties.getInstance().addProperty("carbon.enable.vector.reader", "true")
    +        sqlText = query.sqlText.replace("$table", carbonTableName("2"))
    +        val carbonV2QueryTime = timeit {
    +          spark.sql(sqlText).collect()
    +        }
    +        CarbonProperties.getInstance().addProperty("carbon.enable.vector.reader", "true")
    +        sqlText = query.sqlText.replace("$table", carbonTableName("3"))
    +        val carbonV3QueryTime = timeit {
    +          result2 = spark.sql(sqlText).collect()
    +        }
    +        sqlText = query.sqlText.replace("$table", parquetTableName)
    +        val parquetQueryTime = timeit {
    +          result1 = spark.sql(sqlText).collect()
    +        }
    +        checkResult(result1, result2)
    +        println("\t" +
    +            s"""{ "query":"$index", """ +
    +            s""""parquet"=$parquetQueryTime, """ +
    +            s""""carbonV3"=$carbonV3QueryTime, """ +
    +            s""""carbonV2"=$carbonV2QueryTime, """ +
    +            s""""carbonV2nonVector"=$carbonV2NonVectorQueryTime, """ +
    +            s""""fetched"=${result1.length}", """ +
    +            s""""desc"="${query.desc}" }, """ )
    +    }
    +    println("]")
    +  }
    +
    +  private def checkResult(result1: Array[Row], result2: Array[Row]): Unit = {
    +    if (result1.length != result2.length) {
    +      sys.error(s"result not matching: ${result1.length} and ${result2.length}")
    +    }
    +  }
    +
    +  def main(args: Array[String]): Unit = {
    +    CarbonProperties.getInstance()
    +        .addProperty("carbon.number.of.page.in.blocklet.column", "10")
    +        .addProperty("carbon.enable.vector.reader", "true")
    +        .addProperty("enable.unsafe.sort", "true")
    +    import org.apache.spark.sql.CarbonSession._
    +    val spark = SparkSession
    +        .builder()
    +        .master("local")
    +        .enableHiveSupport()
    +        .getOrCreateCarbonSession("./carbonstore")
    --- End diff --
   
    Please put carbonstore to target directory.


---
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 [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata pull request #624: [CARBONDATA-747][WIP] Add simple per...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user jackylk commented on a diff in the pull request:

    https://github.com/apache/incubator-carbondata/pull/624#discussion_r104564727
 
    --- Diff: examples/spark2/src/main/scala/org/apache/carbondata/examples/CompareTest.scala ---
    @@ -0,0 +1,347 @@
    +/*
    + * 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.examples
    +
    +import org.apache.spark.sql.{DataFrame, Row, SaveMode, SparkSession}
    +import org.apache.spark.sql.types._
    +
    +import org.apache.carbondata.core.constants.CarbonCommonConstants
    +import org.apache.carbondata.core.util.CarbonProperties
    +
    +/**
    + * A query test case
    + * @param sqlText SQL statement
    + * @param queryType type of query: scan, filter, aggregate, topN
    + * @param desc description of the goal of this test case
    + */
    +case class Query(sqlText: String, queryType: String, desc: String)
    +
    +// scalastyle:off println
    +object CompareTest {
    +
    +  def parquetTableName: String = "comparetest_parquet"
    +  def carbonTableName(version: String): String = s"comparetest_carbonV$version"
    +
    +  // Table schema:
    +  // +-------------+-----------+-------------+-------------+------------+
    +  // | Column name | Data type | Cardinality | Column type | Dictionary |
    +  // +-------------+-----------+-------------+-------------+------------+
    +  // | id          | string    | 10,000,000  | dimension   | no         |
    +  // +-------------+-----------+-------------+-------------+------------+
    +  // | country     | string    | 1103        | dimension   | yes        |
    +  // +-------------+-----------+-------------+-------------+------------+
    +  // | city        | string    | 13          | dimension   | yes        |
    +  // +-------------+-----------+-------------+-------------+------------+
    +  // | c4          | short     | NA          | measure     | no         |
    +  // +-------------+-----------+-------------+-------------+------------+
    +  // | c5          | int       | NA          | measure     | no         |
    +  // +-------------+-----------+-------------+-------------+------------+
    +  // | c6          | big int   | NA          | measure     | no         |
    +  // +-------------+-----------+-------------+-------------+------------+
    +  // | c7          | double    | NA          | measure     | no         |
    +  // +-------------+-----------+-------------+-------------+------------+
    +  // | c8          | double    | NA          | measure     | no         |
    +  // +-------------+-----------+-------------+-------------+------------+
    +  private def generateDataFrame(spark: SparkSession): DataFrame = {
    +    val rdd = spark.sparkContext
    +        .parallelize(1 to 10 * 1000 * 1000, 4)
    +        .map { x =>
    +          (x.toString, "p" + x % 1103, "j" + x % 13, (x % 31).toShort, x, x.toLong * 1000,
    +              x.toDouble / 13, x.toDouble / 71 )
    +        }.map { x =>
    +          Row(x._1, x._2, x._3, x._4, x._5, x._6, x._7, x._8)
    +        }
    +    val schema = StructType(
    +      Seq(
    +        StructField("id", StringType, nullable = false),
    +        StructField("country", StringType, nullable = false),
    +        StructField("city", StringType, nullable = false),
    +        StructField("c4", ShortType, nullable = true),
    +        StructField("c5", IntegerType, nullable = true),
    +        StructField("c6", LongType, nullable = true),
    +        StructField("c7", DoubleType, nullable = true),
    +        StructField("c8", DoubleType, nullable = true)
    +      )
    +    )
    +    spark.createDataFrame(rdd, schema)
    +  }
    +
    +  // performance test queries
    +  val queries: Array[Query] = Array(
    +    Query(
    +      "select count(*) from $table",
    +      "warm up",
    +      "warm up query"
    +    ),
    +    // ===========================================================================
    +    // ==                               FULL SCAN                               ==
    +    // ===========================================================================
    +    Query(
    +      "select sum(c4) from $table",
    +      "full scan",
    +      "full scan query, 1 aggregate"
    +    ),
    +    Query(
    +      "select sum(c4), sum(c5) from $table",
    +      "full scan",
    +      "full scan query, 2 aggregate"
    +    ),
    +    Query(
    +      "select sum(c4), sum(c5), sum(c6) from $table",
    +      "full scan",
    +      "full scan query, 3 aggregate"
    +    ),
    +    Query(
    +      "select sum(c4), sum(c5), sum(c6), sum(c7) from $table",
    +      "full scan",
    +      "full scan query, 4 aggregate"
    +    ),
    +    Query(
    +      "select sum(c4), sum(c5), sum(c6), sum(c7), avg(c8) from $table",
    +      "full scan",
    +      "full scan query, 4 aggregate"
    +    ),
    +//    Query(
    +//      "select * from $table",
    +//      "full scan",
    +//      "full scan query, big result set"
    +//    ),
    +    Query(
    +      "select count(distinct id) from $table",
    +      "full scan",
    +      "full scan and count distinct of high card column"
    +    ),
    +    Query(
    +      "select count(distinct country) from $table",
    +      "full scan",
    +      "full scan and count distinct of medium card column"
    +    ),
    +    Query(
    +      "select count(distinct city) from $table",
    +      "full scan",
    +      "full scan and count distinct of low card column"
    +    ),
    +    // ===========================================================================
    +    // ==                               AGGREGATE                               ==
    +    // ===========================================================================
    +//    Query(
    +//      "select id, sum(c4) from $table group by id",
    +//      "aggregate",
    +//      "group by on big data, on high card column, big result set"
    +//    ),
    +    Query(
    +      "select country, sum(c4) from $table group by country",
    +      "aggregate",
    +      "group by on big data, on medium card column, medium result set,"
    +    ),
    +    Query(
    +      "select city, sum(c4) from $table group by city",
    +      "aggregate",
    +      "group by on big data, on low card column, small result set,"
    +    ),
    +    Query(
    +      "select id, sum(c4) as metric from $table group by id order by metric desc limit 100",
    +      "topN",
    +      "top N on high card column"
    +    ),
    +    Query(
    +      "select country,sum(c4) as metric from $table group by country order by metric desc limit 10",
    +      "topN",
    +      "top N on medium card column"
    +    ),
    +    Query(
    +      "select city,sum(c4) as metric from $table group by city order by metric desc limit 10",
    +      "topN",
    +      "top N on low card column"
    +    ),
    +    // ===========================================================================
    +    // ==                             FILTER SCAN                               ==
    +    // ===========================================================================
    +    Query(
    +      "select id, sum(c4) from $table where city='j9' group by id",
    +      "filter scan and aggregate",
    +      "group by on medium data, large result set"
    +    ),
    +    Query(
    +      "select country, sum(c4) from $table where city='j8' group by country ",
    +      "filter scan and aggregate",
    +      "group by on medium data, medium result set"
    +    ),
    +    Query(
    +      "select city, sum(c4) from $table where city='j7' group by city ",
    +      "filter scan and aggregate",
    +      "group by on medium data, small result set"
    +    ),
    +    Query(
    +      "select * from $table where id = '408938' ",
    +      "primary key filter",
    +      "primary key filtering"
    +    ),
    +    Query(
    +      "select * from $table where country = 'p99' ",
    +      "filter scan",
    +      "filter on c2, small result set, fetch all columns"
    +    ),
    +    Query(
    +      "select * from $table where country='p10000'  ",
    +      "filter scan",
    +      "filter on c2, not exist"
    +    ),
    +    Query(
    +      "select * from $table where city = 'j3' limit 100000",
    +      "filter scan",
    +      "filter on c3, medium result set, fetch all columns"
    +    ),
    +    Query(
    +      "select * from $table where country='p88' and city ='j8' ",
    +      "filter scan",
    +      "filter on c2 and c3, small result set, fetch all columns"
    +    ),
    +    Query(
    +      "select * from $table where id='1' and country='p1' and city ='j1' ",
    +      "filter scan",
    +      "filter on c1, c2 and c3, small result set, fetch all columns"
    +    ),
    +    Query(
    +      "select * from $table where c4 < 3",
    +      "filter scan",
    +      "filter on measure, small result set, fetch all columns"
    +    ),
    +    Query(
    +      "select * from $table where id like '1%' ",
    +      "fuzzy filter scan",
    +      "like filter, big result set"
    +    ),
    +    Query(
    +      "select * from $table where id like '%111'",
    +      "fuzzy filter scan",
    +      "like filter, medium result set"
    +    ),
    +    Query(
    +      "select * from $table where id like 'xyz%' ",
    +      "fuzzy filter scan",
    +      "like filter, full scan but not exist"
    +    )
    +  )
    +
    +  private def loadParquetTable(spark: SparkSession, input: DataFrame): Double = timeit {
    +    input.write.mode(SaveMode.Overwrite).parquet(parquetTableName)
    +  }
    +
    +  private def loadCarbonTable(spark: SparkSession, input: DataFrame, version: String): Double = {
    +    CarbonProperties.getInstance().addProperty(
    +      CarbonCommonConstants.CARBON_DATA_FILE_VERSION,
    +      version
    +    )
    +    spark.sql(s"drop table if exists ${carbonTableName(version)}")
    +    timeit {
    +      input.write
    +          .format("carbondata")
    +          .option("tableName", carbonTableName(version))
    +          .option("tempCSV", "false")
    +          .option("single_pass", "true")
    +          .option("dictionary_exclude", "id") // id is high cardinality column
    +          .mode(SaveMode.Overwrite)
    +          .save()
    +    }
    +  }
    +
    +  // load data into parquet, carbonV2, carbonV3
    +  private def prepareTable(spark: SparkSession): Unit = {
    +    val df = generateDataFrame(spark)
    +    println(s"loading ${df.count} records, schema: ${df.schema}")
    +    val loadParquetTime = loadParquetTable(spark, df)
    +    val loadCarbonV3Time = loadCarbonTable(spark, df, version = "3")
    +    val loadCarbonV2Time = loadCarbonTable(spark, df, version = "2")
    +    println(s"load completed, time: $loadParquetTime, $loadCarbonV3Time, $loadCarbonV2Time")
    +    spark.read.parquet(parquetTableName).registerTempTable(parquetTableName)
    +  }
    +
    +  private def runQuery(spark: SparkSession): Unit = {
    +    println("[")
    +    queries.zipWithIndex.foreach {
    +      case (query, index) =>
    +        var result1: Array[Row] = null
    +        var result2: Array[Row] = null
    +
    +        // query parquet, carbonV3, carbonV2, and carbonV2 usinga non-vector reader
    +        CarbonProperties.getInstance().addProperty("carbon.enable.vector.reader", "false")
    +        var sqlText = query.sqlText.replace("$table", carbonTableName("2"))
    +        val carbonV2NonVectorQueryTime = timeit {
    +          spark.sql(sqlText).collect()
    +        }
    +        CarbonProperties.getInstance().addProperty("carbon.enable.vector.reader", "true")
    +        sqlText = query.sqlText.replace("$table", carbonTableName("2"))
    +        val carbonV2QueryTime = timeit {
    +          spark.sql(sqlText).collect()
    +        }
    +        CarbonProperties.getInstance().addProperty("carbon.enable.vector.reader", "true")
    +        sqlText = query.sqlText.replace("$table", carbonTableName("3"))
    +        val carbonV3QueryTime = timeit {
    +          result2 = spark.sql(sqlText).collect()
    +        }
    +        sqlText = query.sqlText.replace("$table", parquetTableName)
    +        val parquetQueryTime = timeit {
    +          result1 = spark.sql(sqlText).collect()
    +        }
    +        checkResult(result1, result2)
    +        println("\t" +
    +            s"""{ "query":"$index", """ +
    +            s""""parquet"=$parquetQueryTime, """ +
    +            s""""carbonV3"=$carbonV3QueryTime, """ +
    +            s""""carbonV2"=$carbonV2QueryTime, """ +
    +            s""""carbonV2nonVector"=$carbonV2NonVectorQueryTime, """ +
    +            s""""fetched"=${result1.length}", """ +
    +            s""""desc"="${query.desc}" }, """ )
    +    }
    +    println("]")
    +  }
    +
    +  private def checkResult(result1: Array[Row], result2: Array[Row]): Unit = {
    +    if (result1.length != result2.length) {
    +      sys.error(s"result not matching: ${result1.length} and ${result2.length}")
    +    }
    +  }
    +
    +  def main(args: Array[String]): Unit = {
    +    CarbonProperties.getInstance()
    +        .addProperty("carbon.number.of.page.in.blocklet.column", "10")
    +        .addProperty("carbon.enable.vector.reader", "true")
    +        .addProperty("enable.unsafe.sort", "true")
    +    import org.apache.spark.sql.CarbonSession._
    +    val spark = SparkSession
    +        .builder()
    +        .master("local")
    +        .enableHiveSupport()
    +        .getOrCreateCarbonSession("./carbonstore")
    --- End diff --
   
    ok


---
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 [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata pull request #624: [CARBONDATA-747][WIP] Add simple per...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user jackylk commented on a diff in the pull request:

    https://github.com/apache/incubator-carbondata/pull/624#discussion_r104564975
 
    --- Diff: examples/spark2/src/main/scala/org/apache/carbondata/examples/CompareTest.scala ---
    @@ -0,0 +1,122 @@
    +/*
    + * 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.examples
    +
    +import org.apache.spark.sql.{DataFrame, Row, SaveMode, SparkSession}
    +
    +import org.apache.carbondata.core.util.CarbonProperties
    +
    +// scalastyle:off println
    +object CompareTest {
    +
    +  val parquetTableName = "comparetest_parquet"
    +  val carbonTableName = "comparetest_carbon"
    +
    +  private def generateDataFrame(spark: SparkSession): DataFrame = {
    +    import spark.implicits._
    +    spark.sparkContext.parallelize(1 to 10 * 1000 * 1000, 4)
    +        .map(x => ("i" + x, "p" + x % 10, "j" + x % 100, x, x + 1, (x + 7) % 21, (x + 5) / 43, x
    +            * 5))
    +        .toDF("id", "country", "city", "c4", "c5", "c6", "c7", "c8")
    +  }
    +
    +  private def loadParquetTable(spark: SparkSession, input: DataFrame): Long = timeit {
    +    input.write.mode(SaveMode.Overwrite).parquet(parquetTableName)
    +  }
    +
    +  private def loadCarbonTable(spark: SparkSession, input: DataFrame): Long = {
    +    spark.sql(s"drop table if exists $carbonTableName")
    +    timeit {
    +      input.write
    +          .format("carbondata")
    +          .option("tableName", carbonTableName)
    +          .option("tempCSV", "false")
    +          .option("single_pass", "true")
    +          .option("dictionary_exclude", "id") // id is high cardinality column
    +          .mode(SaveMode.Overwrite)
    +          .save()
    +    }
    +  }
    +
    +  private def prepareTable(spark: SparkSession): Unit = {
    +    val df = generateDataFrame(spark).cache()
    +    println(s"loading dataframe into table, schema: ${df.schema}")
    +    val loadParquetTime = loadParquetTable(spark, df)
    +    val loadCarbonTime = loadCarbonTable(spark, df)
    +    println(s"load completed, time: $loadParquetTime, $loadCarbonTime")
    +    spark.read.parquet(parquetTableName).registerTempTable(parquetTableName)
    +  }
    +
    +  private def runQuery(spark: SparkSession): Unit = {
    +    val test = Array(
    +      "select count(*) from $table",
    +      "select sum(c4) from $table",
    +      "select sum(c4), sum(c5) from $table",
    +      "select sum(c4), sum(c5), sum(c6) from $table",
    +      "select sum(c4), sum(c5), sum(c6), sum(c7) from $table",
    +      "select sum(c4), sum(c5), sum(c6), sum(c7), avg(c8) from $table",
    +      "select * from $table where id = 'i9999999' ",
    +      "select * from $table where country = 'p9' ",
    +      "select * from $table where city = 'j99' ",
    +      "select * from $table where c4 < 1000 "
    --- End diff --
   
    added


---
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 [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata pull request #624: [CARBONDATA-747][WIP] Add simple per...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user jackylk commented on a diff in the pull request:

    https://github.com/apache/incubator-carbondata/pull/624#discussion_r104565728
 
    --- Diff: examples/spark2/src/main/scala/org/apache/carbondata/examples/CompareTest.scala ---
    @@ -0,0 +1,122 @@
    +/*
    + * 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.examples
    +
    +import org.apache.spark.sql.{DataFrame, Row, SaveMode, SparkSession}
    +
    +import org.apache.carbondata.core.util.CarbonProperties
    +
    +// scalastyle:off println
    +object CompareTest {
    +
    +  val parquetTableName = "comparetest_parquet"
    +  val carbonTableName = "comparetest_carbon"
    +
    +  private def generateDataFrame(spark: SparkSession): DataFrame = {
    +    import spark.implicits._
    +    spark.sparkContext.parallelize(1 to 10 * 1000 * 1000, 4)
    +        .map(x => ("i" + x, "p" + x % 10, "j" + x % 100, x, x + 1, (x + 7) % 21, (x + 5) / 43, x
    --- End diff --
   
    ok


---
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 [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata pull request #624: [CARBONDATA-747][WIP] Add simple per...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user jackylk commented on a diff in the pull request:

    https://github.com/apache/incubator-carbondata/pull/624#discussion_r104566012
 
    --- Diff: examples/spark2/src/main/scala/org/apache/carbondata/examples/CompareTest.scala ---
    @@ -0,0 +1,122 @@
    +/*
    + * 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.examples
    +
    +import org.apache.spark.sql.{DataFrame, Row, SaveMode, SparkSession}
    +
    +import org.apache.carbondata.core.util.CarbonProperties
    +
    +// scalastyle:off println
    +object CompareTest {
    +
    +  val parquetTableName = "comparetest_parquet"
    +  val carbonTableName = "comparetest_carbon"
    +
    +  private def generateDataFrame(spark: SparkSession): DataFrame = {
    +    import spark.implicits._
    +    spark.sparkContext.parallelize(1 to 10 * 1000 * 1000, 4)
    +        .map(x => ("i" + x, "p" + x % 10, "j" + x % 100, x, x + 1, (x + 7) % 21, (x + 5) / 43, x
    +            * 5))
    +        .toDF("id", "country", "city", "c4", "c5", "c6", "c7", "c8")
    --- End diff --
   
    ok, I found decimal is not supported for dataframe.write, I will  raise a JIRA


---
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 [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata pull request #624: [CARBONDATA-747][WIP] Add simple per...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user jackylk commented on a diff in the pull request:

    https://github.com/apache/incubator-carbondata/pull/624#discussion_r104642435
 
    --- Diff: examples/spark2/src/main/scala/org/apache/carbondata/examples/CompareTest.scala ---
    @@ -0,0 +1,122 @@
    +/*
    + * 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.examples
    +
    +import org.apache.spark.sql.{DataFrame, Row, SaveMode, SparkSession}
    +
    +import org.apache.carbondata.core.util.CarbonProperties
    +
    +// scalastyle:off println
    +object CompareTest {
    +
    +  val parquetTableName = "comparetest_parquet"
    +  val carbonTableName = "comparetest_carbon"
    +
    +  private def generateDataFrame(spark: SparkSession): DataFrame = {
    +    import spark.implicits._
    +    spark.sparkContext.parallelize(1 to 10 * 1000 * 1000, 4)
    +        .map(x => ("i" + x, "p" + x % 10, "j" + x % 100, x, x + 1, (x + 7) % 21, (x + 5) / 43, x
    +            * 5))
    +        .toDF("id", "country", "city", "c4", "c5", "c6", "c7", "c8")
    +  }
    +
    +  private def loadParquetTable(spark: SparkSession, input: DataFrame): Long = timeit {
    +    input.write.mode(SaveMode.Overwrite).parquet(parquetTableName)
    --- End diff --
   
    ok


---
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 [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata issue #624: [CARBONDATA-747][WIP] Add simple performanc...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user CarbonDataQA commented on the issue:

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



---
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 [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata issue #624: [CARBONDATA-747][WIP] Add simple performanc...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user CarbonDataQA commented on the issue:

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



---
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 [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata issue #624: [CARBONDATA-747] Add simple performance tes...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user CarbonDataQA commented on the issue:

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



---
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 [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata issue #624: [CARBONDATA-747] Add simple performance tes...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user CarbonDataQA commented on the issue:

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



---
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 [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata issue #624: [CARBONDATA-747] Add simple performance tes...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user CarbonDataQA commented on the issue:

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



---
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 [hidden email] or file a JIRA ticket
with INFRA.
---
12