[jira] [Commented] (SPARK-20356) Spark sql group by returns incorrect results after join + distinct transformations

2017-04-18 Thread Apache Spark (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-20356?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15973996#comment-15973996
 ] 

Apache Spark commented on SPARK-20356:
--

User 'viirya' has created a pull request for this issue:
https://github.com/apache/spark/pull/17679

> Spark sql group by returns incorrect results after join + distinct 
> transformations
> --
>
> Key: SPARK-20356
> URL: https://issues.apache.org/jira/browse/SPARK-20356
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 2.2.0
> Environment: Linux mint 18
> Python 3.5
>Reporter: Chris Kipers
>
> I'm experiencing a bug with the head version of spark as of 4/17/2017. After 
> joining to dataframes, renaming a column and invoking distinct, the results 
> of the aggregation is incorrect after caching the dataframe. The following 
> code snippet consistently reproduces the error.
> from pyspark.sql import SparkSession
> import pyspark.sql.functions as sf
> import pandas as pd
> spark = SparkSession.builder.master("local").appName("Word 
> Count").getOrCreate()
> mapping_sdf = spark.createDataFrame(pd.DataFrame([
> {"ITEM": "a", "GROUP": 1},
> {"ITEM": "b", "GROUP": 1},
> {"ITEM": "c", "GROUP": 2}
> ]))
> items_sdf = spark.createDataFrame(pd.DataFrame([
> {"ITEM": "a", "ID": 1},
> {"ITEM": "b", "ID": 2},
> {"ITEM": "c", "ID": 3}
> ]))
> mapped_sdf = \
> items_sdf.join(mapping_sdf, on='ITEM').select("ID", 
> sf.col("GROUP").alias('ITEM')).distinct()
> print(mapped_sdf.groupBy("ITEM").count().count())  # Prints 2, correct
> mapped_sdf.cache()
> print(mapped_sdf.groupBy("ITEM").count().count())  # Prints 3, incorrect
> The next code snippet is almost the same after the first except I don't call 
> distinct on the dataframe. This snippet performs as expected:
> mapped_sdf = \
> items_sdf.join(mapping_sdf, on='ITEM').select("ID", 
> sf.col("GROUP").alias('ITEM'))
> print(mapped_sdf.groupBy("ITEM").count().count())  # Prints 2, correct
> mapped_sdf.cache()
> print(mapped_sdf.groupBy("ITEM").count().count())  # Prints 2, correct
> I don't experience this bug with spark 2.1 or event earlier versions for 2.2



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-20356) Spark sql group by returns incorrect results after join + distinct transformations

2017-04-18 Thread Liang-Chi Hsieh (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-20356?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15973936#comment-15973936
 ] 

Liang-Chi Hsieh commented on SPARK-20356:
-

[~dkbiswal] Yeah, right. Thanks. We need to force the partition to reproduce 
this.

> Spark sql group by returns incorrect results after join + distinct 
> transformations
> --
>
> Key: SPARK-20356
> URL: https://issues.apache.org/jira/browse/SPARK-20356
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 2.2.0
> Environment: Linux mint 18
> Python 3.5
>Reporter: Chris Kipers
>
> I'm experiencing a bug with the head version of spark as of 4/17/2017. After 
> joining to dataframes, renaming a column and invoking distinct, the results 
> of the aggregation is incorrect after caching the dataframe. The following 
> code snippet consistently reproduces the error.
> from pyspark.sql import SparkSession
> import pyspark.sql.functions as sf
> import pandas as pd
> spark = SparkSession.builder.master("local").appName("Word 
> Count").getOrCreate()
> mapping_sdf = spark.createDataFrame(pd.DataFrame([
> {"ITEM": "a", "GROUP": 1},
> {"ITEM": "b", "GROUP": 1},
> {"ITEM": "c", "GROUP": 2}
> ]))
> items_sdf = spark.createDataFrame(pd.DataFrame([
> {"ITEM": "a", "ID": 1},
> {"ITEM": "b", "ID": 2},
> {"ITEM": "c", "ID": 3}
> ]))
> mapped_sdf = \
> items_sdf.join(mapping_sdf, on='ITEM').select("ID", 
> sf.col("GROUP").alias('ITEM')).distinct()
> print(mapped_sdf.groupBy("ITEM").count().count())  # Prints 2, correct
> mapped_sdf.cache()
> print(mapped_sdf.groupBy("ITEM").count().count())  # Prints 3, incorrect
> The next code snippet is almost the same after the first except I don't call 
> distinct on the dataframe. This snippet performs as expected:
> mapped_sdf = \
> items_sdf.join(mapping_sdf, on='ITEM').select("ID", 
> sf.col("GROUP").alias('ITEM'))
> print(mapped_sdf.groupBy("ITEM").count().count())  # Prints 2, correct
> mapped_sdf.cache()
> print(mapped_sdf.groupBy("ITEM").count().count())  # Prints 2, correct
> I don't experience this bug with spark 2.1 or event earlier versions for 2.2



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-20356) Spark sql group by returns incorrect results after join + distinct transformations

2017-04-18 Thread Dilip Biswal (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-20356?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15973929#comment-15973929
 ] 

Dilip Biswal commented on SPARK-20356:
--

[~viirya] Did you try from spark-shell or from one of our query suites ? I 
could reproduce it from spark-shell fine. From our query suites i had to force 
the number of shuffle partitions to reproduce it.
{code}
test("cache defect") {
withSQLConf("spark.sql.shuffle.partitions" -> "200") {
  val df1 = Seq(("a", 1), ("b", 1), ("c", 2)).toDF("item", "group")
  val df2 = Seq(("a", 1), ("b", 2), ("c", 3)).toDF("item", "id")
  val df3 = df1.join(df2, Seq("item")).select($"id", 
$"group".as("item")).distinct()

  df3.explain(true)

  df3.unpersist()
  val agg_without_cache = df3.groupBy($"item").count()
  agg_without_cache.show()

  df3.cache()
  val agg_with_cache = df3.groupBy($"item").count()
  agg_with_cache.explain(true)
  agg_with_cache.show()
}
  }
{code}

> Spark sql group by returns incorrect results after join + distinct 
> transformations
> --
>
> Key: SPARK-20356
> URL: https://issues.apache.org/jira/browse/SPARK-20356
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 2.2.0
> Environment: Linux mint 18
> Python 3.5
>Reporter: Chris Kipers
>
> I'm experiencing a bug with the head version of spark as of 4/17/2017. After 
> joining to dataframes, renaming a column and invoking distinct, the results 
> of the aggregation is incorrect after caching the dataframe. The following 
> code snippet consistently reproduces the error.
> from pyspark.sql import SparkSession
> import pyspark.sql.functions as sf
> import pandas as pd
> spark = SparkSession.builder.master("local").appName("Word 
> Count").getOrCreate()
> mapping_sdf = spark.createDataFrame(pd.DataFrame([
> {"ITEM": "a", "GROUP": 1},
> {"ITEM": "b", "GROUP": 1},
> {"ITEM": "c", "GROUP": 2}
> ]))
> items_sdf = spark.createDataFrame(pd.DataFrame([
> {"ITEM": "a", "ID": 1},
> {"ITEM": "b", "ID": 2},
> {"ITEM": "c", "ID": 3}
> ]))
> mapped_sdf = \
> items_sdf.join(mapping_sdf, on='ITEM').select("ID", 
> sf.col("GROUP").alias('ITEM')).distinct()
> print(mapped_sdf.groupBy("ITEM").count().count())  # Prints 2, correct
> mapped_sdf.cache()
> print(mapped_sdf.groupBy("ITEM").count().count())  # Prints 3, incorrect
> The next code snippet is almost the same after the first except I don't call 
> distinct on the dataframe. This snippet performs as expected:
> mapped_sdf = \
> items_sdf.join(mapping_sdf, on='ITEM').select("ID", 
> sf.col("GROUP").alias('ITEM'))
> print(mapped_sdf.groupBy("ITEM").count().count())  # Prints 2, correct
> mapped_sdf.cache()
> print(mapped_sdf.groupBy("ITEM").count().count())  # Prints 2, correct
> I don't experience this bug with spark 2.1 or event earlier versions for 2.2



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-20356) Spark sql group by returns incorrect results after join + distinct transformations

2017-04-18 Thread Liang-Chi Hsieh (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-20356?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15973928#comment-15973928
 ] 

Liang-Chi Hsieh commented on SPARK-20356:
-

I think I found the reason of the issue. I am working on it.

> Spark sql group by returns incorrect results after join + distinct 
> transformations
> --
>
> Key: SPARK-20356
> URL: https://issues.apache.org/jira/browse/SPARK-20356
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 2.2.0
> Environment: Linux mint 18
> Python 3.5
>Reporter: Chris Kipers
>
> I'm experiencing a bug with the head version of spark as of 4/17/2017. After 
> joining to dataframes, renaming a column and invoking distinct, the results 
> of the aggregation is incorrect after caching the dataframe. The following 
> code snippet consistently reproduces the error.
> from pyspark.sql import SparkSession
> import pyspark.sql.functions as sf
> import pandas as pd
> spark = SparkSession.builder.master("local").appName("Word 
> Count").getOrCreate()
> mapping_sdf = spark.createDataFrame(pd.DataFrame([
> {"ITEM": "a", "GROUP": 1},
> {"ITEM": "b", "GROUP": 1},
> {"ITEM": "c", "GROUP": 2}
> ]))
> items_sdf = spark.createDataFrame(pd.DataFrame([
> {"ITEM": "a", "ID": 1},
> {"ITEM": "b", "ID": 2},
> {"ITEM": "c", "ID": 3}
> ]))
> mapped_sdf = \
> items_sdf.join(mapping_sdf, on='ITEM').select("ID", 
> sf.col("GROUP").alias('ITEM')).distinct()
> print(mapped_sdf.groupBy("ITEM").count().count())  # Prints 2, correct
> mapped_sdf.cache()
> print(mapped_sdf.groupBy("ITEM").count().count())  # Prints 3, incorrect
> The next code snippet is almost the same after the first except I don't call 
> distinct on the dataframe. This snippet performs as expected:
> mapped_sdf = \
> items_sdf.join(mapping_sdf, on='ITEM').select("ID", 
> sf.col("GROUP").alias('ITEM'))
> print(mapped_sdf.groupBy("ITEM").count().count())  # Prints 2, correct
> mapped_sdf.cache()
> print(mapped_sdf.groupBy("ITEM").count().count())  # Prints 2, correct
> I don't experience this bug with spark 2.1 or event earlier versions for 2.2



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-20356) Spark sql group by returns incorrect results after join + distinct transformations

2017-04-18 Thread Liang-Chi Hsieh (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-20356?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15973902#comment-15973902
 ] 

Liang-Chi Hsieh commented on SPARK-20356:
-

[~hvanhovell] I can't reproduce it with your example code. They both output:
{code}
++-+
|item|count|
++-+
|   1|2|
|   2|1|
++-+
{code}

> Spark sql group by returns incorrect results after join + distinct 
> transformations
> --
>
> Key: SPARK-20356
> URL: https://issues.apache.org/jira/browse/SPARK-20356
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 2.2.0
> Environment: Linux mint 18
> Python 3.5
>Reporter: Chris Kipers
>
> I'm experiencing a bug with the head version of spark as of 4/17/2017. After 
> joining to dataframes, renaming a column and invoking distinct, the results 
> of the aggregation is incorrect after caching the dataframe. The following 
> code snippet consistently reproduces the error.
> from pyspark.sql import SparkSession
> import pyspark.sql.functions as sf
> import pandas as pd
> spark = SparkSession.builder.master("local").appName("Word 
> Count").getOrCreate()
> mapping_sdf = spark.createDataFrame(pd.DataFrame([
> {"ITEM": "a", "GROUP": 1},
> {"ITEM": "b", "GROUP": 1},
> {"ITEM": "c", "GROUP": 2}
> ]))
> items_sdf = spark.createDataFrame(pd.DataFrame([
> {"ITEM": "a", "ID": 1},
> {"ITEM": "b", "ID": 2},
> {"ITEM": "c", "ID": 3}
> ]))
> mapped_sdf = \
> items_sdf.join(mapping_sdf, on='ITEM').select("ID", 
> sf.col("GROUP").alias('ITEM')).distinct()
> print(mapped_sdf.groupBy("ITEM").count().count())  # Prints 2, correct
> mapped_sdf.cache()
> print(mapped_sdf.groupBy("ITEM").count().count())  # Prints 3, incorrect
> The next code snippet is almost the same after the first except I don't call 
> distinct on the dataframe. This snippet performs as expected:
> mapped_sdf = \
> items_sdf.join(mapping_sdf, on='ITEM').select("ID", 
> sf.col("GROUP").alias('ITEM'))
> print(mapped_sdf.groupBy("ITEM").count().count())  # Prints 2, correct
> mapped_sdf.cache()
> print(mapped_sdf.groupBy("ITEM").count().count())  # Prints 2, correct
> I don't experience this bug with spark 2.1 or event earlier versions for 2.2



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-20356) Spark sql group by returns incorrect results after join + distinct transformations

2017-04-18 Thread Liang-Chi Hsieh (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-20356?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15973793#comment-15973793
 ] 

Liang-Chi Hsieh commented on SPARK-20356:
-

[~dkbiswal] Thanks for pinging me. I will look into this.

> Spark sql group by returns incorrect results after join + distinct 
> transformations
> --
>
> Key: SPARK-20356
> URL: https://issues.apache.org/jira/browse/SPARK-20356
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 2.2.0
> Environment: Linux mint 18
> Python 3.5
>Reporter: Chris Kipers
>
> I'm experiencing a bug with the head version of spark as of 4/17/2017. After 
> joining to dataframes, renaming a column and invoking distinct, the results 
> of the aggregation is incorrect after caching the dataframe. The following 
> code snippet consistently reproduces the error.
> from pyspark.sql import SparkSession
> import pyspark.sql.functions as sf
> import pandas as pd
> spark = SparkSession.builder.master("local").appName("Word 
> Count").getOrCreate()
> mapping_sdf = spark.createDataFrame(pd.DataFrame([
> {"ITEM": "a", "GROUP": 1},
> {"ITEM": "b", "GROUP": 1},
> {"ITEM": "c", "GROUP": 2}
> ]))
> items_sdf = spark.createDataFrame(pd.DataFrame([
> {"ITEM": "a", "ID": 1},
> {"ITEM": "b", "ID": 2},
> {"ITEM": "c", "ID": 3}
> ]))
> mapped_sdf = \
> items_sdf.join(mapping_sdf, on='ITEM').select("ID", 
> sf.col("GROUP").alias('ITEM')).distinct()
> print(mapped_sdf.groupBy("ITEM").count().count())  # Prints 2, correct
> mapped_sdf.cache()
> print(mapped_sdf.groupBy("ITEM").count().count())  # Prints 3, incorrect
> The next code snippet is almost the same after the first except I don't call 
> distinct on the dataframe. This snippet performs as expected:
> mapped_sdf = \
> items_sdf.join(mapping_sdf, on='ITEM').select("ID", 
> sf.col("GROUP").alias('ITEM'))
> print(mapped_sdf.groupBy("ITEM").count().count())  # Prints 2, correct
> mapped_sdf.cache()
> print(mapped_sdf.groupBy("ITEM").count().count())  # Prints 2, correct
> I don't experience this bug with spark 2.1 or event earlier versions for 2.2



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-20356) Spark sql group by returns incorrect results after join + distinct transformations

2017-04-18 Thread Dilip Biswal (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-20356?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15973455#comment-15973455
 ] 

Dilip Biswal commented on SPARK-20356:
--

[~viirya] [~hvanhovell] [~cloud_fan] [~smilegator]
I took a quick look and it seems the issue started happening after 
[pr|https://github.com/apache/spark/pull/17175]. We are
changing the output partitioning information as part of the fix ( id, item -> 
item, item) causing a missing shuffle in the operators
above InMemoryTableScan. Changing to use the child's output partitioning like 
before fixes the issue. 

I am a little new to this code :-) And this is what i have found so far. Hope 
this helps.


> Spark sql group by returns incorrect results after join + distinct 
> transformations
> --
>
> Key: SPARK-20356
> URL: https://issues.apache.org/jira/browse/SPARK-20356
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 2.2.0
> Environment: Linux mint 18
> Python 3.5
>Reporter: Chris Kipers
>
> I'm experiencing a bug with the head version of spark as of 4/17/2017. After 
> joining to dataframes, renaming a column and invoking distinct, the results 
> of the aggregation is incorrect after caching the dataframe. The following 
> code snippet consistently reproduces the error.
> from pyspark.sql import SparkSession
> import pyspark.sql.functions as sf
> import pandas as pd
> spark = SparkSession.builder.master("local").appName("Word 
> Count").getOrCreate()
> mapping_sdf = spark.createDataFrame(pd.DataFrame([
> {"ITEM": "a", "GROUP": 1},
> {"ITEM": "b", "GROUP": 1},
> {"ITEM": "c", "GROUP": 2}
> ]))
> items_sdf = spark.createDataFrame(pd.DataFrame([
> {"ITEM": "a", "ID": 1},
> {"ITEM": "b", "ID": 2},
> {"ITEM": "c", "ID": 3}
> ]))
> mapped_sdf = \
> items_sdf.join(mapping_sdf, on='ITEM').select("ID", 
> sf.col("GROUP").alias('ITEM')).distinct()
> print(mapped_sdf.groupBy("ITEM").count().count())  # Prints 2, correct
> mapped_sdf.cache()
> print(mapped_sdf.groupBy("ITEM").count().count())  # Prints 3, incorrect
> The next code snippet is almost the same after the first except I don't call 
> distinct on the dataframe. This snippet performs as expected:
> mapped_sdf = \
> items_sdf.join(mapping_sdf, on='ITEM').select("ID", 
> sf.col("GROUP").alias('ITEM'))
> print(mapped_sdf.groupBy("ITEM").count().count())  # Prints 2, correct
> mapped_sdf.cache()
> print(mapped_sdf.groupBy("ITEM").count().count())  # Prints 2, correct
> I don't experience this bug with spark 2.1 or event earlier versions for 2.2



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-20356) Spark sql group by returns incorrect results after join + distinct transformations

2017-04-18 Thread Job Evers-Meltzer (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-20356?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15973100#comment-15973100
 ] 

Job Evers-Meltzer commented on SPARK-20356:
---

Commit 5ed397baa758c29c54a853d3f8fee0ad44e97c14 doesn't seem to have this issue.

> Spark sql group by returns incorrect results after join + distinct 
> transformations
> --
>
> Key: SPARK-20356
> URL: https://issues.apache.org/jira/browse/SPARK-20356
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 2.2.0
> Environment: Linux mint 18
> Python 3.5
>Reporter: Chris Kipers
>
> I'm experiencing a bug with the head version of spark as of 4/17/2017. After 
> joining to dataframes, renaming a column and invoking distinct, the results 
> of the aggregation is incorrect after caching the dataframe. The following 
> code snippet consistently reproduces the error.
> from pyspark.sql import SparkSession
> import pyspark.sql.functions as sf
> import pandas as pd
> spark = SparkSession.builder.master("local").appName("Word 
> Count").getOrCreate()
> mapping_sdf = spark.createDataFrame(pd.DataFrame([
> {"ITEM": "a", "GROUP": 1},
> {"ITEM": "b", "GROUP": 1},
> {"ITEM": "c", "GROUP": 2}
> ]))
> items_sdf = spark.createDataFrame(pd.DataFrame([
> {"ITEM": "a", "ID": 1},
> {"ITEM": "b", "ID": 2},
> {"ITEM": "c", "ID": 3}
> ]))
> mapped_sdf = \
> items_sdf.join(mapping_sdf, on='ITEM').select("ID", 
> sf.col("GROUP").alias('ITEM')).distinct()
> print(mapped_sdf.groupBy("ITEM").count().count())  # Prints 2, correct
> mapped_sdf.cache()
> print(mapped_sdf.groupBy("ITEM").count().count())  # Prints 3, incorrect
> The next code snippet is almost the same after the first except I don't call 
> distinct on the dataframe. This snippet performs as expected:
> mapped_sdf = \
> items_sdf.join(mapping_sdf, on='ITEM').select("ID", 
> sf.col("GROUP").alias('ITEM'))
> print(mapped_sdf.groupBy("ITEM").count().count())  # Prints 2, correct
> mapped_sdf.cache()
> print(mapped_sdf.groupBy("ITEM").count().count())  # Prints 2, correct
> I don't experience this bug with spark 2.1 or event earlier versions for 2.2



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-20356) Spark sql group by returns incorrect results after join + distinct transformations

2017-04-18 Thread Herman van Hovell (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-20356?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15972753#comment-15972753
 ] 

Herman van Hovell commented on SPARK-20356:
---

Here is a reproduction in scala:
{noformat}
val df1 = Seq(("a", 1), ("b", 1), ("c", 2)).toDF("item", "group")
val df2 = Seq(("a", 1), ("b", 2), ("c", 3)).toDF("item", "id")
val df3 = df1.join(df2, Seq("item")).select($"id", 
$"group".as("item")).distinct()

df3.unpersist()
val agg_without_cache = df3.groupBy($"item").count()
agg_without_cache.show()

df3.cache()
val agg_with_cache = df3.groupBy($"item").count()
agg_with_cache.show()
{noformat}

> Spark sql group by returns incorrect results after join + distinct 
> transformations
> --
>
> Key: SPARK-20356
> URL: https://issues.apache.org/jira/browse/SPARK-20356
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 2.2.0
> Environment: Linux mint 18
> Python 3.5
>Reporter: Chris Kipers
>
> I'm experiencing a bug with the head version of spark as of 4/17/2017. After 
> joining to dataframes, renaming a column and invoking distinct, the results 
> of the aggregation is incorrect after caching the dataframe. The following 
> code snippet consistently reproduces the error.
> from pyspark.sql import SparkSession
> import pyspark.sql.functions as sf
> import pandas as pd
> spark = SparkSession.builder.master("local").appName("Word 
> Count").getOrCreate()
> mapping_sdf = spark.createDataFrame(pd.DataFrame([
> {"ITEM": "a", "GROUP": 1},
> {"ITEM": "b", "GROUP": 1},
> {"ITEM": "c", "GROUP": 2}
> ]))
> items_sdf = spark.createDataFrame(pd.DataFrame([
> {"ITEM": "a", "ID": 1},
> {"ITEM": "b", "ID": 2},
> {"ITEM": "c", "ID": 3}
> ]))
> mapped_sdf = \
> items_sdf.join(mapping_sdf, on='ITEM').select("ID", 
> sf.col("GROUP").alias('ITEM')).distinct()
> print(mapped_sdf.groupBy("ITEM").count().count())  # Prints 2, correct
> mapped_sdf.cache()
> print(mapped_sdf.groupBy("ITEM").count().count())  # Prints 3, incorrect
> The next code snippet is almost the same after the first except I don't call 
> distinct on the dataframe. This snippet performs as expected:
> mapped_sdf = \
> items_sdf.join(mapping_sdf, on='ITEM').select("ID", 
> sf.col("GROUP").alias('ITEM'))
> print(mapped_sdf.groupBy("ITEM").count().count())  # Prints 2, correct
> mapped_sdf.cache()
> print(mapped_sdf.groupBy("ITEM").count().count())  # Prints 2, correct
> I don't experience this bug with spark 2.1 or event earlier versions for 2.2



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-20356) Spark sql group by returns incorrect results after join + distinct transformations

2017-04-18 Thread Ed Lee (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-20356?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15972544#comment-15972544
 ] 

Ed Lee commented on SPARK-20356:


really quite dangerous bug

> Spark sql group by returns incorrect results after join + distinct 
> transformations
> --
>
> Key: SPARK-20356
> URL: https://issues.apache.org/jira/browse/SPARK-20356
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 2.2.0
> Environment: Linux mint 18
> Python 3.5
>Reporter: Chris Kipers
>
> I'm experiencing a bug with the head version of spark as of 4/17/2017. After 
> joining to dataframes, renaming a column and invoking distinct, the results 
> of the aggregation is incorrect after caching the dataframe. The following 
> code snippet consistently reproduces the error.
> from pyspark.sql import SparkSession
> import pyspark.sql.functions as sf
> import pandas as pd
> spark = SparkSession.builder.master("local").appName("Word 
> Count").getOrCreate()
> mapping_sdf = spark.createDataFrame(pd.DataFrame([
> {"ITEM": "a", "GROUP": 1},
> {"ITEM": "b", "GROUP": 1},
> {"ITEM": "c", "GROUP": 2}
> ]))
> items_sdf = spark.createDataFrame(pd.DataFrame([
> {"ITEM": "a", "ID": 1},
> {"ITEM": "b", "ID": 2},
> {"ITEM": "c", "ID": 3}
> ]))
> mapped_sdf = \
> items_sdf.join(mapping_sdf, on='ITEM').select("ID", 
> sf.col("GROUP").alias('ITEM')).distinct()
> print(mapped_sdf.groupBy("ITEM").count().count())  # Prints 2, correct
> mapped_sdf.cache()
> print(mapped_sdf.groupBy("ITEM").count().count())  # Prints 3, incorrect
> The next code snippet is almost the same after the first except I don't call 
> distinct on the dataframe. This snippet performs as expected:
> mapped_sdf = \
> items_sdf.join(mapping_sdf, on='ITEM').select("ID", 
> sf.col("GROUP").alias('ITEM'))
> print(mapped_sdf.groupBy("ITEM").count().count())  # Prints 2, correct
> mapped_sdf.cache()
> print(mapped_sdf.groupBy("ITEM").count().count())  # Prints 2, correct
> I don't experience this bug with spark 2.1 or event earlier versions for 2.2



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org