You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Virgil Palanciuc (JIRA)" <ji...@apache.org> on 2016/09/19 18:13:20 UTC

[jira] [Updated] (SPARK-17594) Bug in left-outer join

     [ https://issues.apache.org/jira/browse/SPARK-17594?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Virgil Palanciuc updated SPARK-17594:
-------------------------------------
    Description: 
I have a bug where I think a left-join returns wrong results, by mistakenly matching long values that are identical on 32bits (differ in their upper halves). 

Instructions to reproduce

{code}
scala> val x = Seq((100L,17179869185L), (102L,17179869185L), (101L,17179869186L), (200L,17179869186L), (401L,1L), (500L,1L), (600L,8589934593L), (700L,8589934593L), (800L,8589934593L), (900L,8589934594L), (1000L,8589934594L), (1200L,2L), (1300L,2L), (1301L,2L), (1400L,17179869187L), (1500L,17179869188L), (1600L,8589934595L)).toDF("u","x1")
x: org.apache.spark.sql.DataFrame = [u: bigint, x1: bigint]

scala> val y = Seq((17179869187L,-8589934595L), (17179869188L,-8589934595L), (17179869185L,-8589934593L)).toDF("x2","y")
y: org.apache.spark.sql.DataFrame = [x2: bigint, y: bigint]

scala> x.join(y,'x1 === 'x2, "left_outer").show()
{code}
|   u|         x1|         x2|          y|
| 100|17179869185|17179869185|-8589934593|
| 102|17179869185|17179869185|-8589934593|
| 101|17179869186|       null|       null|
| 200|17179869186|       null|       null|
| 401|          1|17179869185|-8589934593|
| 500|          1|17179869185|-8589934593|
| 600| 8589934593|17179869185|-8589934593|
| 700| 8589934593|17179869185|-8589934593|
| 800| 8589934593|17179869185|-8589934593|
| 900| 8589934594|       null|       null|
|1000| 8589934594|       null|       null|
|1200|          2|       null|       null|
|1300|          2|       null|       null|
|1301|          2|       null|       null|
|1400|17179869187|17179869187|-8589934595|
|1500|17179869188|17179869188|-8589934595|
|1600| 8589934595|17179869187|-8589934595|




  was:
I have a bug where I think a left-join returns wrong results, by mistakenly matching long values that are identical on 32bits (differ in their upper halves). 

Trying to reproduce it in the console - I get "ArrayIndexOutOfBoundsException" - not identical, but it may be related:

{code}
scala> val b = Seq(1L, 3L).toDF("x")
b: org.apache.spark.sql.DataFrame = [x: bigint]

scala> val a = Seq(17179869185L, 17179869186L).toDF("x")
a: org.apache.spark.sql.DataFrame = [x: bigint]

scala> b.join(a,Seq("x"), "left_outer").show()
16/09/19 16:34:42 ERROR Executor: Exception in task 6.0 in stage 3.0 (TID 15)
java.lang.ArrayIndexOutOfBoundsException: 2
	at org.apache.spark.sql.execution.joins.LongToUnsafeRowMap.getValue(HashedRelation.scala:463)
	at org.apache.spark.sql.execution.joins.LongHashedRelation.getValue(HashedRelation.scala:762)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown Source)
	at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
	at org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$8$$anon$1.hasNext(WholeStageCodegenExec.scala:370)
	at org.apache.spark.sql.execution.SparkPlan$$anonfun$4.apply(SparkPlan.scala:246)
	at org.apache.spark.sql.execution.SparkPlan$$anonfun$4.apply(SparkPlan.scala:240)
	at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:784)
	at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:784)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:319)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:283)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:70)
	at org.apache.spark.scheduler.Task.run(Task.scala:85)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
	at java.lang.Thread.run(Thread.java:745)
16/09/19 16:34:42 WARN TaskSetManager: Lost task 6.0 in stage 3.0 (TID 15, localhost): java.lang.ArrayIndexOutOfBoundsException: 2
	at org.apache.spark.sql.execution.joins.LongToUnsafeRowMap.getValue(HashedRelation.scala:463)
	at org.apache.spark.sql.execution.joins.LongHashedRelation.getValue(HashedRelation.scala:762)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown Source)
	at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
	at org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$8$$anon$1.hasNext(WholeStageCodegenExec.scala:370)
	at org.apache.spark.sql.execution.SparkPlan$$anonfun$4.apply(SparkPlan.scala:246)
	at org.apache.spark.sql.execution.SparkPlan$$anonfun$4.apply(SparkPlan.scala:240)
	at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:784)
	at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:784)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:319)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:283)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:70)
	at org.apache.spark.scheduler.Task.run(Task.scala:85)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
	at java.lang.Thread.run(Thread.java:745)

16/09/19 16:34:42 ERROR TaskSetManager: Task 6 in stage 3.0 failed 1 times; aborting job
org.apache.spark.SparkException: Job aborted due to stage failure: Task 6 in stage 3.0 failed 1 times, most recent failure: Lost task 6.0 in stage 3.0 (TID 15, localhost): java.lang.ArrayIndexOutOfBoundsException: 2
	at org.apache.spark.sql.execution.joins.LongToUnsafeRowMap.getValue(HashedRelation.scala:463)
	at org.apache.spark.sql.execution.joins.LongHashedRelation.getValue(HashedRelation.scala:762)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown Source)
	at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
	at org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$8$$anon$1.hasNext(WholeStageCodegenExec.scala:370)
	at org.apache.spark.sql.execution.SparkPlan$$anonfun$4.apply(SparkPlan.scala:246)
	at org.apache.spark.sql.execution.SparkPlan$$anonfun$4.apply(SparkPlan.scala:240)
	at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:784)
	at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:784)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:319)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:283)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:70)
	at org.apache.spark.scheduler.Task.run(Task.scala:85)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
	at java.lang.Thread.run(Thread.java:745)

Driver stacktrace:
  at org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1450)
  at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1438)
  at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1437)
  at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
  at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
  at org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1437)
  at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:811)
  at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:811)
  at scala.Option.foreach(Option.scala:257)
  at org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:811)
  at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:1659)
  at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1618)
  at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1607)
  at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:48)
  at org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:632)
  at org.apache.spark.SparkContext.runJob(SparkContext.scala:1871)
  at org.apache.spark.SparkContext.runJob(SparkContext.scala:1884)
  at org.apache.spark.SparkContext.runJob(SparkContext.scala:1897)
  at org.apache.spark.sql.execution.SparkPlan.executeTake(SparkPlan.scala:347)
  at org.apache.spark.sql.execution.CollectLimitExec.executeCollect(limit.scala:39)
  at org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1.apply(Dataset.scala:2183)
  at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:57)
  at org.apache.spark.sql.Dataset.withNewExecutionId(Dataset.scala:2532)
  at org.apache.spark.sql.Dataset.org$apache$spark$sql$Dataset$$execute$1(Dataset.scala:2182)
  at org.apache.spark.sql.Dataset.org$apache$spark$sql$Dataset$$collect(Dataset.scala:2189)
  at org.apache.spark.sql.Dataset$$anonfun$head$1.apply(Dataset.scala:1925)
  at org.apache.spark.sql.Dataset$$anonfun$head$1.apply(Dataset.scala:1924)
  at org.apache.spark.sql.Dataset.withTypedCallback(Dataset.scala:2562)
  at org.apache.spark.sql.Dataset.head(Dataset.scala:1924)
  at org.apache.spark.sql.Dataset.take(Dataset.scala:2139)
  at org.apache.spark.sql.Dataset.showString(Dataset.scala:239)
  at org.apache.spark.sql.Dataset.show(Dataset.scala:526)
  at org.apache.spark.sql.Dataset.show(Dataset.scala:486)
  at org.apache.spark.sql.Dataset.show(Dataset.scala:495)
  ... 48 elided
Caused by: java.lang.ArrayIndexOutOfBoundsException: 2
  at org.apache.spark.sql.execution.joins.LongToUnsafeRowMap.getValue(HashedRelation.scala:463)
  at org.apache.spark.sql.execution.joins.LongHashedRelation.getValue(HashedRelation.scala:762)
  at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown Source)
  at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
  at org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$8$$anon$1.hasNext(WholeStageCodegenExec.scala:370)
  at org.apache.spark.sql.execution.SparkPlan$$anonfun$4.apply(SparkPlan.scala:246)
  at org.apache.spark.sql.execution.SparkPlan$$anonfun$4.apply(SparkPlan.scala:240)
  at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:784)
  at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:784)
  at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
  at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:319)
  at org.apache.spark.rdd.RDD.iterator(RDD.scala:283)
  at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:70)
  at org.apache.spark.scheduler.Task.run(Task.scala:85)
  at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274)
  at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
  at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
  at java.lang.Thread.run(Thread.java:745)

{code}

My issue may or may not be related with this exception that I've reproduced in the console..... here is what I'm getting in my program:

{code}
scala> df1.join(df2, 'x1 === 'x2, "left_outer").show()
{code}
| uuid| x1| x2| y|
| 100|  17179869185|             17179869185|         -8589934593|
| 102|  17179869185|             17179869185|         -8589934593|
| 101|  17179869186|                    null|                null|
| 200|  17179869186|                    null|                null|
| 401|            1|             17179869185|         -8589934593|
| 500|            1|             17179869185|         -8589934593|
| 600|   8589934593|             17179869185|         -8589934593|
| 700|   8589934593|             17179869185|         -8589934593|
| 800|   8589934593|             17179869185|         -8589934593|
| 900|   8589934594|                    null|                null|
|1000|   8589934594|                    null|                null|
|1200|            2|                    null|                null|
|1300|            2|                    null|                null|
|1301|            2|                    null|                null|
|1400|  17179869187|             17179869187|         -8589934595|
|1500|  17179869188|             17179869188|         -8589934595|
|1600|   8589934595|             17179869187|         -8589934595|



> Bug in left-outer join
> ----------------------
>
>                 Key: SPARK-17594
>                 URL: https://issues.apache.org/jira/browse/SPARK-17594
>             Project: Spark
>          Issue Type: Bug
>          Components: SQL
>    Affects Versions: 2.0.0
>            Reporter: Virgil Palanciuc
>
> I have a bug where I think a left-join returns wrong results, by mistakenly matching long values that are identical on 32bits (differ in their upper halves). 
> Instructions to reproduce
> {code}
> scala> val x = Seq((100L,17179869185L), (102L,17179869185L), (101L,17179869186L), (200L,17179869186L), (401L,1L), (500L,1L), (600L,8589934593L), (700L,8589934593L), (800L,8589934593L), (900L,8589934594L), (1000L,8589934594L), (1200L,2L), (1300L,2L), (1301L,2L), (1400L,17179869187L), (1500L,17179869188L), (1600L,8589934595L)).toDF("u","x1")
> x: org.apache.spark.sql.DataFrame = [u: bigint, x1: bigint]
> scala> val y = Seq((17179869187L,-8589934595L), (17179869188L,-8589934595L), (17179869185L,-8589934593L)).toDF("x2","y")
> y: org.apache.spark.sql.DataFrame = [x2: bigint, y: bigint]
> scala> x.join(y,'x1 === 'x2, "left_outer").show()
> {code}
> |   u|         x1|         x2|          y|
> | 100|17179869185|17179869185|-8589934593|
> | 102|17179869185|17179869185|-8589934593|
> | 101|17179869186|       null|       null|
> | 200|17179869186|       null|       null|
> | 401|          1|17179869185|-8589934593|
> | 500|          1|17179869185|-8589934593|
> | 600| 8589934593|17179869185|-8589934593|
> | 700| 8589934593|17179869185|-8589934593|
> | 800| 8589934593|17179869185|-8589934593|
> | 900| 8589934594|       null|       null|
> |1000| 8589934594|       null|       null|
> |1200|          2|       null|       null|
> |1300|          2|       null|       null|
> |1301|          2|       null|       null|
> |1400|17179869187|17179869187|-8589934595|
> |1500|17179869188|17179869188|-8589934595|
> |1600| 8589934595|17179869187|-8589934595|



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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