[jira] [Commented] (SPARK-13850) TimSort Comparison method violates its general contract

2016-05-26 Thread Apache Spark (JIRA)

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

Apache Spark commented on SPARK-13850:
--

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

> TimSort Comparison method violates its general contract
> ---
>
> Key: SPARK-13850
> URL: https://issues.apache.org/jira/browse/SPARK-13850
> Project: Spark
>  Issue Type: Bug
>  Components: Shuffle
>Affects Versions: 1.6.0
>Reporter: Sital Kedia
>
> While running a query which does a group by on a large dataset, the query 
> fails with following stack trace. 
> {code}
> Job aborted due to stage failure: Task 4077 in stage 1.3 failed 4 times, most 
> recent failure: Lost task 4077.3 in stage 1.3 (TID 88702, 
> hadoop3030.prn2.facebook.com): java.lang.IllegalArgumentException: Comparison 
> method violates its general contract!
>   at 
> org.apache.spark.util.collection.TimSort$SortState.mergeLo(TimSort.java:794)
>   at 
> org.apache.spark.util.collection.TimSort$SortState.mergeAt(TimSort.java:525)
>   at 
> org.apache.spark.util.collection.TimSort$SortState.mergeCollapse(TimSort.java:453)
>   at 
> org.apache.spark.util.collection.TimSort$SortState.access$200(TimSort.java:325)
>   at org.apache.spark.util.collection.TimSort.sort(TimSort.java:153)
>   at org.apache.spark.util.collection.Sorter.sort(Sorter.scala:37)
>   at 
> org.apache.spark.util.collection.unsafe.sort.UnsafeInMemorySorter.getSortedIterator(UnsafeInMemorySorter.java:228)
>   at 
> org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.spill(UnsafeExternalSorter.java:186)
>   at 
> org.apache.spark.memory.TaskMemoryManager.acquireExecutionMemory(TaskMemoryManager.java:175)
>   at 
> org.apache.spark.memory.TaskMemoryManager.allocatePage(TaskMemoryManager.java:249)
>   at 
> org.apache.spark.memory.MemoryConsumer.allocatePage(MemoryConsumer.java:112)
>   at 
> org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.acquireNewPageIfNecessary(UnsafeExternalSorter.java:318)
>   at 
> org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.insertRecord(UnsafeExternalSorter.java:333)
>   at 
> org.apache.spark.sql.execution.UnsafeExternalRowSorter.insertRow(UnsafeExternalRowSorter.java:91)
>   at 
> org.apache.spark.sql.execution.UnsafeExternalRowSorter.sort(UnsafeExternalRowSorter.java:168)
>   at org.apache.spark.sql.execution.Sort$$anonfun$1.apply(Sort.scala:90)
>   at org.apache.spark.sql.execution.Sort$$anonfun$1.apply(Sort.scala:64)
>   at 
> org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$21.apply(RDD.scala:728)
>   at 
> org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$21.apply(RDD.scala:728)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>   at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:66)
>   at org.apache.spark.scheduler.Task.run(Task.scala:89)
>   at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:214)
>   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}
> Please note that the same query used to succeed in Spark 1.5 so it seems like 
> a regression in 1.6.



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

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



[jira] [Commented] (SPARK-13850) TimSort Comparison method violates its general contract

2016-05-17 Thread Sital Kedia (JIRA)

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

Sital Kedia commented on SPARK-13850:
-

I am not 100% sure of the root cause, but I suspect this is happening when JVM 
is trying to allocate very large size buffer for pointer array. The issue might 
be because the JVM is not able to allocate large buffer in contiguous memory 
location on heap and since the unsafe operations assume contiguous memory 
location of the objects, any unsafe operation on large buffer results in memory 
corruption which manifests as TimSort issue.

> TimSort Comparison method violates its general contract
> ---
>
> Key: SPARK-13850
> URL: https://issues.apache.org/jira/browse/SPARK-13850
> Project: Spark
>  Issue Type: Bug
>  Components: Shuffle
>Affects Versions: 1.6.0
>Reporter: Sital Kedia
>
> While running a query which does a group by on a large dataset, the query 
> fails with following stack trace. 
> {code}
> Job aborted due to stage failure: Task 4077 in stage 1.3 failed 4 times, most 
> recent failure: Lost task 4077.3 in stage 1.3 (TID 88702, 
> hadoop3030.prn2.facebook.com): java.lang.IllegalArgumentException: Comparison 
> method violates its general contract!
>   at 
> org.apache.spark.util.collection.TimSort$SortState.mergeLo(TimSort.java:794)
>   at 
> org.apache.spark.util.collection.TimSort$SortState.mergeAt(TimSort.java:525)
>   at 
> org.apache.spark.util.collection.TimSort$SortState.mergeCollapse(TimSort.java:453)
>   at 
> org.apache.spark.util.collection.TimSort$SortState.access$200(TimSort.java:325)
>   at org.apache.spark.util.collection.TimSort.sort(TimSort.java:153)
>   at org.apache.spark.util.collection.Sorter.sort(Sorter.scala:37)
>   at 
> org.apache.spark.util.collection.unsafe.sort.UnsafeInMemorySorter.getSortedIterator(UnsafeInMemorySorter.java:228)
>   at 
> org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.spill(UnsafeExternalSorter.java:186)
>   at 
> org.apache.spark.memory.TaskMemoryManager.acquireExecutionMemory(TaskMemoryManager.java:175)
>   at 
> org.apache.spark.memory.TaskMemoryManager.allocatePage(TaskMemoryManager.java:249)
>   at 
> org.apache.spark.memory.MemoryConsumer.allocatePage(MemoryConsumer.java:112)
>   at 
> org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.acquireNewPageIfNecessary(UnsafeExternalSorter.java:318)
>   at 
> org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.insertRecord(UnsafeExternalSorter.java:333)
>   at 
> org.apache.spark.sql.execution.UnsafeExternalRowSorter.insertRow(UnsafeExternalRowSorter.java:91)
>   at 
> org.apache.spark.sql.execution.UnsafeExternalRowSorter.sort(UnsafeExternalRowSorter.java:168)
>   at org.apache.spark.sql.execution.Sort$$anonfun$1.apply(Sort.scala:90)
>   at org.apache.spark.sql.execution.Sort$$anonfun$1.apply(Sort.scala:64)
>   at 
> org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$21.apply(RDD.scala:728)
>   at 
> org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$21.apply(RDD.scala:728)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>   at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:66)
>   at org.apache.spark.scheduler.Task.run(Task.scala:89)
>   at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:214)
>   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}
> Please note that the same query used to succeed in 

[jira] [Commented] (SPARK-13850) TimSort Comparison method violates its general contract

2016-05-16 Thread Yin Huai (JIRA)

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

Yin Huai commented on SPARK-13850:
--

Can you explain the root cause at here?

> TimSort Comparison method violates its general contract
> ---
>
> Key: SPARK-13850
> URL: https://issues.apache.org/jira/browse/SPARK-13850
> Project: Spark
>  Issue Type: Bug
>  Components: Shuffle
>Affects Versions: 1.6.0
>Reporter: Sital Kedia
>
> While running a query which does a group by on a large dataset, the query 
> fails with following stack trace. 
> {code}
> Job aborted due to stage failure: Task 4077 in stage 1.3 failed 4 times, most 
> recent failure: Lost task 4077.3 in stage 1.3 (TID 88702, 
> hadoop3030.prn2.facebook.com): java.lang.IllegalArgumentException: Comparison 
> method violates its general contract!
>   at 
> org.apache.spark.util.collection.TimSort$SortState.mergeLo(TimSort.java:794)
>   at 
> org.apache.spark.util.collection.TimSort$SortState.mergeAt(TimSort.java:525)
>   at 
> org.apache.spark.util.collection.TimSort$SortState.mergeCollapse(TimSort.java:453)
>   at 
> org.apache.spark.util.collection.TimSort$SortState.access$200(TimSort.java:325)
>   at org.apache.spark.util.collection.TimSort.sort(TimSort.java:153)
>   at org.apache.spark.util.collection.Sorter.sort(Sorter.scala:37)
>   at 
> org.apache.spark.util.collection.unsafe.sort.UnsafeInMemorySorter.getSortedIterator(UnsafeInMemorySorter.java:228)
>   at 
> org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.spill(UnsafeExternalSorter.java:186)
>   at 
> org.apache.spark.memory.TaskMemoryManager.acquireExecutionMemory(TaskMemoryManager.java:175)
>   at 
> org.apache.spark.memory.TaskMemoryManager.allocatePage(TaskMemoryManager.java:249)
>   at 
> org.apache.spark.memory.MemoryConsumer.allocatePage(MemoryConsumer.java:112)
>   at 
> org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.acquireNewPageIfNecessary(UnsafeExternalSorter.java:318)
>   at 
> org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.insertRecord(UnsafeExternalSorter.java:333)
>   at 
> org.apache.spark.sql.execution.UnsafeExternalRowSorter.insertRow(UnsafeExternalRowSorter.java:91)
>   at 
> org.apache.spark.sql.execution.UnsafeExternalRowSorter.sort(UnsafeExternalRowSorter.java:168)
>   at org.apache.spark.sql.execution.Sort$$anonfun$1.apply(Sort.scala:90)
>   at org.apache.spark.sql.execution.Sort$$anonfun$1.apply(Sort.scala:64)
>   at 
> org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$21.apply(RDD.scala:728)
>   at 
> org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$21.apply(RDD.scala:728)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>   at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:66)
>   at org.apache.spark.scheduler.Task.run(Task.scala:89)
>   at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:214)
>   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}
> Please note that the same query used to succeed in Spark 1.5 so it seems like 
> a regression in 1.6.



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

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



[jira] [Commented] (SPARK-13850) TimSort Comparison method violates its general contract

2016-05-13 Thread Sital Kedia (JIRA)

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

Sital Kedia commented on SPARK-13850:
-

I have found a workaround for this issue. Please take a look at the PR. 

> TimSort Comparison method violates its general contract
> ---
>
> Key: SPARK-13850
> URL: https://issues.apache.org/jira/browse/SPARK-13850
> Project: Spark
>  Issue Type: Bug
>  Components: Shuffle
>Affects Versions: 1.6.0
>Reporter: Sital Kedia
>
> While running a query which does a group by on a large dataset, the query 
> fails with following stack trace. 
> {code}
> Job aborted due to stage failure: Task 4077 in stage 1.3 failed 4 times, most 
> recent failure: Lost task 4077.3 in stage 1.3 (TID 88702, 
> hadoop3030.prn2.facebook.com): java.lang.IllegalArgumentException: Comparison 
> method violates its general contract!
>   at 
> org.apache.spark.util.collection.TimSort$SortState.mergeLo(TimSort.java:794)
>   at 
> org.apache.spark.util.collection.TimSort$SortState.mergeAt(TimSort.java:525)
>   at 
> org.apache.spark.util.collection.TimSort$SortState.mergeCollapse(TimSort.java:453)
>   at 
> org.apache.spark.util.collection.TimSort$SortState.access$200(TimSort.java:325)
>   at org.apache.spark.util.collection.TimSort.sort(TimSort.java:153)
>   at org.apache.spark.util.collection.Sorter.sort(Sorter.scala:37)
>   at 
> org.apache.spark.util.collection.unsafe.sort.UnsafeInMemorySorter.getSortedIterator(UnsafeInMemorySorter.java:228)
>   at 
> org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.spill(UnsafeExternalSorter.java:186)
>   at 
> org.apache.spark.memory.TaskMemoryManager.acquireExecutionMemory(TaskMemoryManager.java:175)
>   at 
> org.apache.spark.memory.TaskMemoryManager.allocatePage(TaskMemoryManager.java:249)
>   at 
> org.apache.spark.memory.MemoryConsumer.allocatePage(MemoryConsumer.java:112)
>   at 
> org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.acquireNewPageIfNecessary(UnsafeExternalSorter.java:318)
>   at 
> org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.insertRecord(UnsafeExternalSorter.java:333)
>   at 
> org.apache.spark.sql.execution.UnsafeExternalRowSorter.insertRow(UnsafeExternalRowSorter.java:91)
>   at 
> org.apache.spark.sql.execution.UnsafeExternalRowSorter.sort(UnsafeExternalRowSorter.java:168)
>   at org.apache.spark.sql.execution.Sort$$anonfun$1.apply(Sort.scala:90)
>   at org.apache.spark.sql.execution.Sort$$anonfun$1.apply(Sort.scala:64)
>   at 
> org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$21.apply(RDD.scala:728)
>   at 
> org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$21.apply(RDD.scala:728)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>   at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:66)
>   at org.apache.spark.scheduler.Task.run(Task.scala:89)
>   at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:214)
>   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}
> Please note that the same query used to succeed in Spark 1.5 so it seems like 
> a regression in 1.6.



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

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



[jira] [Commented] (SPARK-13850) TimSort Comparison method violates its general contract

2016-05-13 Thread Apache Spark (JIRA)

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

Apache Spark commented on SPARK-13850:
--

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

> TimSort Comparison method violates its general contract
> ---
>
> Key: SPARK-13850
> URL: https://issues.apache.org/jira/browse/SPARK-13850
> Project: Spark
>  Issue Type: Bug
>  Components: Shuffle
>Affects Versions: 1.6.0
>Reporter: Sital Kedia
>
> While running a query which does a group by on a large dataset, the query 
> fails with following stack trace. 
> {code}
> Job aborted due to stage failure: Task 4077 in stage 1.3 failed 4 times, most 
> recent failure: Lost task 4077.3 in stage 1.3 (TID 88702, 
> hadoop3030.prn2.facebook.com): java.lang.IllegalArgumentException: Comparison 
> method violates its general contract!
>   at 
> org.apache.spark.util.collection.TimSort$SortState.mergeLo(TimSort.java:794)
>   at 
> org.apache.spark.util.collection.TimSort$SortState.mergeAt(TimSort.java:525)
>   at 
> org.apache.spark.util.collection.TimSort$SortState.mergeCollapse(TimSort.java:453)
>   at 
> org.apache.spark.util.collection.TimSort$SortState.access$200(TimSort.java:325)
>   at org.apache.spark.util.collection.TimSort.sort(TimSort.java:153)
>   at org.apache.spark.util.collection.Sorter.sort(Sorter.scala:37)
>   at 
> org.apache.spark.util.collection.unsafe.sort.UnsafeInMemorySorter.getSortedIterator(UnsafeInMemorySorter.java:228)
>   at 
> org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.spill(UnsafeExternalSorter.java:186)
>   at 
> org.apache.spark.memory.TaskMemoryManager.acquireExecutionMemory(TaskMemoryManager.java:175)
>   at 
> org.apache.spark.memory.TaskMemoryManager.allocatePage(TaskMemoryManager.java:249)
>   at 
> org.apache.spark.memory.MemoryConsumer.allocatePage(MemoryConsumer.java:112)
>   at 
> org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.acquireNewPageIfNecessary(UnsafeExternalSorter.java:318)
>   at 
> org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.insertRecord(UnsafeExternalSorter.java:333)
>   at 
> org.apache.spark.sql.execution.UnsafeExternalRowSorter.insertRow(UnsafeExternalRowSorter.java:91)
>   at 
> org.apache.spark.sql.execution.UnsafeExternalRowSorter.sort(UnsafeExternalRowSorter.java:168)
>   at org.apache.spark.sql.execution.Sort$$anonfun$1.apply(Sort.scala:90)
>   at org.apache.spark.sql.execution.Sort$$anonfun$1.apply(Sort.scala:64)
>   at 
> org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$21.apply(RDD.scala:728)
>   at 
> org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$21.apply(RDD.scala:728)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>   at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:66)
>   at org.apache.spark.scheduler.Task.run(Task.scala:89)
>   at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:214)
>   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}
> Please note that the same query used to succeed in Spark 1.5 so it seems like 
> a regression in 1.6.



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

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



[jira] [Commented] (SPARK-13850) TimSort Comparison method violates its general contract

2016-05-11 Thread Yin Huai (JIRA)

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

Yin Huai commented on SPARK-13850:
--

SPARK-12055  duplicates https://issues.apache.org/jira/browse/SPARK-12030, 
which was merged in both 1.5 branch and 1.6.0.

> TimSort Comparison method violates its general contract
> ---
>
> Key: SPARK-13850
> URL: https://issues.apache.org/jira/browse/SPARK-13850
> Project: Spark
>  Issue Type: Bug
>  Components: Shuffle
>Affects Versions: 1.6.0
>Reporter: Sital Kedia
>
> While running a query which does a group by on a large dataset, the query 
> fails with following stack trace. 
> {code}
> Job aborted due to stage failure: Task 4077 in stage 1.3 failed 4 times, most 
> recent failure: Lost task 4077.3 in stage 1.3 (TID 88702, 
> hadoop3030.prn2.facebook.com): java.lang.IllegalArgumentException: Comparison 
> method violates its general contract!
>   at 
> org.apache.spark.util.collection.TimSort$SortState.mergeLo(TimSort.java:794)
>   at 
> org.apache.spark.util.collection.TimSort$SortState.mergeAt(TimSort.java:525)
>   at 
> org.apache.spark.util.collection.TimSort$SortState.mergeCollapse(TimSort.java:453)
>   at 
> org.apache.spark.util.collection.TimSort$SortState.access$200(TimSort.java:325)
>   at org.apache.spark.util.collection.TimSort.sort(TimSort.java:153)
>   at org.apache.spark.util.collection.Sorter.sort(Sorter.scala:37)
>   at 
> org.apache.spark.util.collection.unsafe.sort.UnsafeInMemorySorter.getSortedIterator(UnsafeInMemorySorter.java:228)
>   at 
> org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.spill(UnsafeExternalSorter.java:186)
>   at 
> org.apache.spark.memory.TaskMemoryManager.acquireExecutionMemory(TaskMemoryManager.java:175)
>   at 
> org.apache.spark.memory.TaskMemoryManager.allocatePage(TaskMemoryManager.java:249)
>   at 
> org.apache.spark.memory.MemoryConsumer.allocatePage(MemoryConsumer.java:112)
>   at 
> org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.acquireNewPageIfNecessary(UnsafeExternalSorter.java:318)
>   at 
> org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.insertRecord(UnsafeExternalSorter.java:333)
>   at 
> org.apache.spark.sql.execution.UnsafeExternalRowSorter.insertRow(UnsafeExternalRowSorter.java:91)
>   at 
> org.apache.spark.sql.execution.UnsafeExternalRowSorter.sort(UnsafeExternalRowSorter.java:168)
>   at org.apache.spark.sql.execution.Sort$$anonfun$1.apply(Sort.scala:90)
>   at org.apache.spark.sql.execution.Sort$$anonfun$1.apply(Sort.scala:64)
>   at 
> org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$21.apply(RDD.scala:728)
>   at 
> org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$21.apply(RDD.scala:728)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>   at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:66)
>   at org.apache.spark.scheduler.Task.run(Task.scala:89)
>   at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:214)
>   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}
> Please note that the same query used to succeed in Spark 1.5 so it seems like 
> a regression in 1.6.



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

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



[jira] [Commented] (SPARK-13850) TimSort Comparison method violates its general contract

2016-03-30 Thread Regan Dvoskin (JIRA)

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

Regan Dvoskin commented on SPARK-13850:
---

We're having a query fail on an inner join of two large HIVE tables with the 
same stack trace. The query worked on 1.5, and works on 1.6.0 if 
spark.memory.useLegacyMode is true, but fails on 1.6.0 when useLegacyMode is 
not enabled. 

> TimSort Comparison method violates its general contract
> ---
>
> Key: SPARK-13850
> URL: https://issues.apache.org/jira/browse/SPARK-13850
> Project: Spark
>  Issue Type: Bug
>  Components: Shuffle
>Affects Versions: 1.6.0
>Reporter: Sital Kedia
>
> While running a query which does a group by on a large dataset, the query 
> fails with following stack trace. 
> {code}
> Job aborted due to stage failure: Task 4077 in stage 1.3 failed 4 times, most 
> recent failure: Lost task 4077.3 in stage 1.3 (TID 88702, 
> hadoop3030.prn2.facebook.com): java.lang.IllegalArgumentException: Comparison 
> method violates its general contract!
>   at 
> org.apache.spark.util.collection.TimSort$SortState.mergeLo(TimSort.java:794)
>   at 
> org.apache.spark.util.collection.TimSort$SortState.mergeAt(TimSort.java:525)
>   at 
> org.apache.spark.util.collection.TimSort$SortState.mergeCollapse(TimSort.java:453)
>   at 
> org.apache.spark.util.collection.TimSort$SortState.access$200(TimSort.java:325)
>   at org.apache.spark.util.collection.TimSort.sort(TimSort.java:153)
>   at org.apache.spark.util.collection.Sorter.sort(Sorter.scala:37)
>   at 
> org.apache.spark.util.collection.unsafe.sort.UnsafeInMemorySorter.getSortedIterator(UnsafeInMemorySorter.java:228)
>   at 
> org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.spill(UnsafeExternalSorter.java:186)
>   at 
> org.apache.spark.memory.TaskMemoryManager.acquireExecutionMemory(TaskMemoryManager.java:175)
>   at 
> org.apache.spark.memory.TaskMemoryManager.allocatePage(TaskMemoryManager.java:249)
>   at 
> org.apache.spark.memory.MemoryConsumer.allocatePage(MemoryConsumer.java:112)
>   at 
> org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.acquireNewPageIfNecessary(UnsafeExternalSorter.java:318)
>   at 
> org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.insertRecord(UnsafeExternalSorter.java:333)
>   at 
> org.apache.spark.sql.execution.UnsafeExternalRowSorter.insertRow(UnsafeExternalRowSorter.java:91)
>   at 
> org.apache.spark.sql.execution.UnsafeExternalRowSorter.sort(UnsafeExternalRowSorter.java:168)
>   at org.apache.spark.sql.execution.Sort$$anonfun$1.apply(Sort.scala:90)
>   at org.apache.spark.sql.execution.Sort$$anonfun$1.apply(Sort.scala:64)
>   at 
> org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$21.apply(RDD.scala:728)
>   at 
> org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$21.apply(RDD.scala:728)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>   at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:66)
>   at org.apache.spark.scheduler.Task.run(Task.scala:89)
>   at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:214)
>   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}
> Please note that the same query used to succeed in Spark 1.5 so it seems like 
> a regression in 1.6.



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

-
To unsubscribe, 

[jira] [Commented] (SPARK-13850) TimSort Comparison method violates its general contract

2016-03-14 Thread Sital Kedia (JIRA)

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

Sital Kedia commented on SPARK-13850:
-

Thanks [~srowen], The PR related to SPARK-12055 was merged in 1.5 and we are 
seeing this issue in 1.6. So unfortunately the issue is not resolved yet. 

> TimSort Comparison method violates its general contract
> ---
>
> Key: SPARK-13850
> URL: https://issues.apache.org/jira/browse/SPARK-13850
> Project: Spark
>  Issue Type: Bug
>  Components: Shuffle
>Affects Versions: 1.6.0
>Reporter: Sital Kedia
>
> While running a query which does a group by on a large dataset, the query 
> fails with following stack trace. 
> {code}
> Job aborted due to stage failure: Task 4077 in stage 1.3 failed 4 times, most 
> recent failure: Lost task 4077.3 in stage 1.3 (TID 88702, 
> hadoop3030.prn2.facebook.com): java.lang.IllegalArgumentException: Comparison 
> method violates its general contract!
>   at 
> org.apache.spark.util.collection.TimSort$SortState.mergeLo(TimSort.java:794)
>   at 
> org.apache.spark.util.collection.TimSort$SortState.mergeAt(TimSort.java:525)
>   at 
> org.apache.spark.util.collection.TimSort$SortState.mergeCollapse(TimSort.java:453)
>   at 
> org.apache.spark.util.collection.TimSort$SortState.access$200(TimSort.java:325)
>   at org.apache.spark.util.collection.TimSort.sort(TimSort.java:153)
>   at org.apache.spark.util.collection.Sorter.sort(Sorter.scala:37)
>   at 
> org.apache.spark.util.collection.unsafe.sort.UnsafeInMemorySorter.getSortedIterator(UnsafeInMemorySorter.java:228)
>   at 
> org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.spill(UnsafeExternalSorter.java:186)
>   at 
> org.apache.spark.memory.TaskMemoryManager.acquireExecutionMemory(TaskMemoryManager.java:175)
>   at 
> org.apache.spark.memory.TaskMemoryManager.allocatePage(TaskMemoryManager.java:249)
>   at 
> org.apache.spark.memory.MemoryConsumer.allocatePage(MemoryConsumer.java:112)
>   at 
> org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.acquireNewPageIfNecessary(UnsafeExternalSorter.java:318)
>   at 
> org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.insertRecord(UnsafeExternalSorter.java:333)
>   at 
> org.apache.spark.sql.execution.UnsafeExternalRowSorter.insertRow(UnsafeExternalRowSorter.java:91)
>   at 
> org.apache.spark.sql.execution.UnsafeExternalRowSorter.sort(UnsafeExternalRowSorter.java:168)
>   at org.apache.spark.sql.execution.Sort$$anonfun$1.apply(Sort.scala:90)
>   at org.apache.spark.sql.execution.Sort$$anonfun$1.apply(Sort.scala:64)
>   at 
> org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$21.apply(RDD.scala:728)
>   at 
> org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$21.apply(RDD.scala:728)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>   at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:66)
>   at org.apache.spark.scheduler.Task.run(Task.scala:89)
>   at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:214)
>   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}
> Please note that the same query used to succeed in Spark 1.5 so it seems like 
> a regression in 1.6.



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

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

[jira] [Commented] (SPARK-13850) TimSort Comparison method violates its general contract

2016-03-14 Thread Sean Owen (JIRA)

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

Sean Owen commented on SPARK-13850:
---

Better, but this still looks like SPARK-12055 for example

> TimSort Comparison method violates its general contract
> ---
>
> Key: SPARK-13850
> URL: https://issues.apache.org/jira/browse/SPARK-13850
> Project: Spark
>  Issue Type: Bug
>  Components: Shuffle
>Affects Versions: 1.6.0
>Reporter: Sital Kedia
>
> While running a query which does a group by on a large dataset, the query 
> fails with following stack trace. 
> {code}
> Job aborted due to stage failure: Task 4077 in stage 1.3 failed 4 times, most 
> recent failure: Lost task 4077.3 in stage 1.3 (TID 88702, 
> hadoop3030.prn2.facebook.com): java.lang.IllegalArgumentException: Comparison 
> method violates its general contract!
>   at 
> org.apache.spark.util.collection.TimSort$SortState.mergeLo(TimSort.java:794)
>   at 
> org.apache.spark.util.collection.TimSort$SortState.mergeAt(TimSort.java:525)
>   at 
> org.apache.spark.util.collection.TimSort$SortState.mergeCollapse(TimSort.java:453)
>   at 
> org.apache.spark.util.collection.TimSort$SortState.access$200(TimSort.java:325)
>   at org.apache.spark.util.collection.TimSort.sort(TimSort.java:153)
>   at org.apache.spark.util.collection.Sorter.sort(Sorter.scala:37)
>   at 
> org.apache.spark.util.collection.unsafe.sort.UnsafeInMemorySorter.getSortedIterator(UnsafeInMemorySorter.java:228)
>   at 
> org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.spill(UnsafeExternalSorter.java:186)
>   at 
> org.apache.spark.memory.TaskMemoryManager.acquireExecutionMemory(TaskMemoryManager.java:175)
>   at 
> org.apache.spark.memory.TaskMemoryManager.allocatePage(TaskMemoryManager.java:249)
>   at 
> org.apache.spark.memory.MemoryConsumer.allocatePage(MemoryConsumer.java:112)
>   at 
> org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.acquireNewPageIfNecessary(UnsafeExternalSorter.java:318)
>   at 
> org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.insertRecord(UnsafeExternalSorter.java:333)
>   at 
> org.apache.spark.sql.execution.UnsafeExternalRowSorter.insertRow(UnsafeExternalRowSorter.java:91)
>   at 
> org.apache.spark.sql.execution.UnsafeExternalRowSorter.sort(UnsafeExternalRowSorter.java:168)
>   at org.apache.spark.sql.execution.Sort$$anonfun$1.apply(Sort.scala:90)
>   at org.apache.spark.sql.execution.Sort$$anonfun$1.apply(Sort.scala:64)
>   at 
> org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$21.apply(RDD.scala:728)
>   at 
> org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$21.apply(RDD.scala:728)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>   at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:66)
>   at org.apache.spark.scheduler.Task.run(Task.scala:89)
>   at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:214)
>   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}
> Please note that the same query used to succeed in Spark 1.5 so it seems like 
> a regression in 1.6.



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

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



[jira] [Commented] (SPARK-13850) TimSort Comparison method violates its general contract

2016-03-14 Thread Sean Owen (JIRA)

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

Sean Owen commented on SPARK-13850:
---

Duplicate of https://issues.apache.org/jira/browse/SPARK-8428, but you gave no 
detail

> TimSort Comparison method violates its general contract
> ---
>
> Key: SPARK-13850
> URL: https://issues.apache.org/jira/browse/SPARK-13850
> Project: Spark
>  Issue Type: Bug
>  Components: Shuffle
>Affects Versions: 1.6.0
>Reporter: Sital Kedia
>
> While running a query which does a group by on a large dataset, the query 
> fails with following stack trace. 
> {code}
> Job aborted due to stage failure: Task 4077 in stage 1.3 failed 4 times, most 
> recent failure: Lost task 4077.3 in stage 1.3 (TID 88702, 
> hadoop3030.prn2.facebook.com): java.lang.IllegalArgumentException: Comparison 
> method violates its general contract!
>   at 
> org.apache.spark.util.collection.TimSort$SortState.mergeLo(TimSort.java:794)
>   at 
> org.apache.spark.util.collection.TimSort$SortState.mergeAt(TimSort.java:525)
>   at 
> org.apache.spark.util.collection.TimSort$SortState.mergeCollapse(TimSort.java:453)
>   at 
> org.apache.spark.util.collection.TimSort$SortState.access$200(TimSort.java:325)
>   at org.apache.spark.util.collection.TimSort.sort(TimSort.java:153)
>   at org.apache.spark.util.collection.Sorter.sort(Sorter.scala:37)
>   at 
> org.apache.spark.util.collection.unsafe.sort.UnsafeInMemorySorter.getSortedIterator(UnsafeInMemorySorter.java:228)
>   at 
> org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.spill(UnsafeExternalSorter.java:186)
>   at 
> org.apache.spark.memory.TaskMemoryManager.acquireExecutionMemory(TaskMemoryManager.java:175)
>   at 
> org.apache.spark.memory.TaskMemoryManager.allocatePage(TaskMemoryManager.java:249)
>   at 
> org.apache.spark.memory.MemoryConsumer.allocatePage(MemoryConsumer.java:112)
>   at 
> org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.acquireNewPageIfNecessary(UnsafeExternalSorter.java:318)
>   at 
> org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.insertRecord(UnsafeExternalSorter.java:333)
>   at 
> org.apache.spark.sql.execution.UnsafeExternalRowSorter.insertRow(UnsafeExternalRowSorter.java:91)
>   at 
> org.apache.spark.sql.execution.UnsafeExternalRowSorter.sort(UnsafeExternalRowSorter.java:168)
>   at org.apache.spark.sql.execution.Sort$$anonfun$1.apply(Sort.scala:90)
>   at org.apache.spark.sql.execution.Sort$$anonfun$1.apply(Sort.scala:64)
>   at 
> org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$21.apply(RDD.scala:728)
>   at 
> org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$21.apply(RDD.scala:728)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>   at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>   at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:66)
>   at org.apache.spark.scheduler.Task.run(Task.scala:89)
>   at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:214)
>   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}
> Please note that the same query used to succeed in Spark 1.5 so it seems like 
> a regression in 1.6.



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

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