[ https://issues.apache.org/jira/browse/SPARK-3098?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14110463#comment-14110463 ]
Sean Owen commented on SPARK-3098: ---------------------------------- [~matei] The question isn't whether distinct returns a particular ordering, or whether zipWithIndex assigns particular indices, but whether they would result in the same ordering and same assignments every time the RDD is evaluated: {code} val c = {...}.distinct().zipWithIndex() c.join(c).filter(t => t._2._1 != t._2._2) {code} If so, then the same values should map to the same indices, and the self-join of c to itself should always pair the same value with itself. Regardless of what those un-guaranteed values are they should be the same since it's the very same RDD. If not, obviously that explains the behavior then. The behavior at first glance had also surprised me, since I had taken RDDs to be "deterministic" and transparently recomputable on demand. That is the important first question -- is that supposed to be so or not? > In some cases, operation zipWithIndex get a wrong results > ---------------------------------------------------------- > > Key: SPARK-3098 > URL: https://issues.apache.org/jira/browse/SPARK-3098 > Project: Spark > Issue Type: Bug > Components: Spark Core > Affects Versions: 1.0.1 > Reporter: Guoqiang Li > Priority: Critical > > The reproduce code: > {code} > val c = sc.parallelize(1 to 7899).flatMap { i => > (1 to 10000).toSeq.map(p => i * 6000 + p) > }.distinct().zipWithIndex() > c.join(c).filter(t => t._2._1 != t._2._2).take(3) > {code} > => > {code} > Array[(Int, (Long, Long))] = Array((1732608,(11,12)), (45515264,(12,13)), > (36579712,(13,14))) > {code} -- This message was sent by Atlassian JIRA (v6.2#6252) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org