[ https://issues.apache.org/jira/browse/SPARK-15822?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15324506#comment-15324506 ]
Pete Robbins commented on SPARK-15822: -------------------------------------- generated SMJ code from the stack: {code} public Object generate(Object[] references) { return new GeneratedIterator(references); } /*wholestagecodegen_c1*/ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator { private Object[] references; private scala.collection.Iterator smj_leftInput; private scala.collection.Iterator smj_rightInput; private InternalRow smj_leftRow; private InternalRow smj_rightRow; private UTF8String smj_value4; private UTF8String smj_value5; private java.util.ArrayList smj_matches; private UTF8String smj_value6; private UTF8String smj_value7; private UTF8String smj_value8; private boolean smj_isNull4; private UTF8String smj_value9; private boolean smj_isNull5; private long smj_value10; private org.apache.spark.sql.execution.metric.SQLMetric smj_numOutputRows; private UnsafeRow smj_result; private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder smj_holder; private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter smj_rowWriter; private UnsafeRow project_result; private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder project_holder; private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter project_rowWriter; public GeneratedIterator(Object[] references) { this.references = references; } public void init(int index, scala.collection.Iterator inputs[]) { partitionIndex = index; smj_leftInput = inputs[0]; smj_rightInput = inputs[1]; smj_rightRow = null; smj_matches = new java.util.ArrayList(); this.smj_numOutputRows = (org.apache.spark.sql.execution.metric.SQLMetric) references[0]; smj_result = new UnsafeRow(6); this.smj_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(smj_result, 128); this.smj_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(smj_holder, 6); project_result = new UnsafeRow(3); this.project_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(project_result, 64); this.project_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(project_holder, 3); } private boolean findNextInnerJoinRows( scala.collection.Iterator leftIter, scala.collection.Iterator rightIter) { smj_leftRow = null; int comp = 0; while (smj_leftRow == null) { if (!leftIter.hasNext()) return false; smj_leftRow = (InternalRow) leftIter.next(); /*smj_c1*/ boolean smj_isNull = smj_leftRow.isNullAt(0); UTF8String smj_value = smj_isNull ? null : (smj_leftRow.getUTF8String(0)); /*smj_c2*/ boolean smj_isNull1 = smj_leftRow.isNullAt(1); UTF8String smj_value1 = smj_isNull1 ? null : (smj_leftRow.getUTF8String(1)); if (smj_isNull || smj_isNull1) { smj_leftRow = null; continue; } if (!smj_matches.isEmpty()) { comp = 0; if (comp == 0) { comp = smj_value.compare(smj_value6); } if (comp == 0) { comp = smj_value1.compare(smj_value7); } if (comp == 0) { return true; } smj_matches.clear(); } do { if (smj_rightRow == null) { if (!rightIter.hasNext()) { smj_value6 = smj_value; smj_value7 = smj_value1; return !smj_matches.isEmpty(); } smj_rightRow = (InternalRow) rightIter.next(); /*smj_c3*/ boolean smj_isNull2 = smj_rightRow.isNullAt(0); UTF8String smj_value2 = smj_isNull2 ? null : (smj_rightRow.getUTF8String(0)); /*smj_c4*/ boolean smj_isNull3 = smj_rightRow.isNullAt(1); UTF8String smj_value3 = smj_isNull3 ? null : (smj_rightRow.getUTF8String(1)); if (smj_isNull2 || smj_isNull3) { smj_rightRow = null; continue; } smj_value4 = smj_value2; smj_value5 = smj_value3; } comp = 0; if (comp == 0) { comp = smj_value.compare(smj_value4); } if (comp == 0) { comp = smj_value1.compare(smj_value5); } if (comp > 0) { smj_rightRow = null; } else if (comp < 0) { if (!smj_matches.isEmpty()) { smj_value6 = smj_value; smj_value7 = smj_value1; return true; } smj_leftRow = null; } else { smj_matches.add(smj_rightRow.copy()); smj_rightRow = null;; } } while (smj_leftRow != null); } return false; // unreachable } protected void processNext() throws java.io.IOException { /*project_c*/ /*smj_c*/ while (findNextInnerJoinRows(smj_leftInput, smj_rightInput)) { int smj_size = smj_matches.size(); smj_isNull4 = smj_leftRow.isNullAt(0); smj_value8 = smj_isNull4 ? null : (smj_leftRow.getUTF8String(0)); smj_isNull5 = smj_leftRow.isNullAt(1); smj_value9 = smj_isNull5 ? null : (smj_leftRow.getUTF8String(1)); smj_value10 = smj_leftRow.getLong(2); for (int smj_i = 0; smj_i < smj_size; smj_i ++) { InternalRow smj_rightRow1 = (InternalRow) smj_matches.get(smj_i); smj_numOutputRows.add(1); /*project_c1*/ /*wholestagecodegen_c*/ /*project_c7*/ /*smj_c7*/ long smj_value13 = smj_rightRow1.getLong(2); boolean project_isNull8 = false; double project_value8 = -1.0; if (!false) { project_value8 = (double) smj_value13; } boolean project_isNull3 = false; double project_value3 = -1.0; if (project_value8 == 0) { project_isNull3 = true; } else { /*project_c3*/ long project_value5 = -1L; project_value5 = smj_value10 * 100L; boolean project_isNull4 = false; double project_value4 = -1.0; if (!false) { project_value4 = (double) project_value5; } project_value3 = (double)(project_value4 / project_value8); } boolean project_isNull2 = project_isNull3; double project_value2 = -1.0; if (!project_isNull2) { if (Double.isNaN(project_value3) || Double.isInfinite(project_value3)) { project_value2 = project_value3; } else { project_value2 = java.math.BigDecimal.valueOf(project_value3). setScale(2, java.math.BigDecimal.ROUND_HALF_UP).doubleValue(); } } project_holder.reset(); project_rowWriter.zeroOutNullBytes(); if (smj_isNull4) { project_rowWriter.setNullAt(0); } else { project_rowWriter.write(0, smj_value8); } if (smj_isNull5) { project_rowWriter.setNullAt(1); } else { project_rowWriter.write(1, smj_value9); } if (project_isNull2) { project_rowWriter.setNullAt(2); } else { project_rowWriter.write(2, project_value2); } project_result.setTotalSize(project_holder.totalSize()); append(project_result.copy()); } if (shouldStop()) return; } } } {code} > segmentation violation in o.a.s.unsafe.types.UTF8String > -------------------------------------------------------- > > Key: SPARK-15822 > URL: https://issues.apache.org/jira/browse/SPARK-15822 > Project: Spark > Issue Type: Bug > Affects Versions: 2.0.0 > Environment: linux amd64 > openjdk version "1.8.0_91" > OpenJDK Runtime Environment (build 1.8.0_91-b14) > OpenJDK 64-Bit Server VM (build 25.91-b14, mixed mode) > Reporter: Pete Robbins > Assignee: Herman van Hovell > Priority: Blocker > > Executors fail with segmentation violation while running application with > spark.memory.offHeap.enabled true > spark.memory.offHeap.size 512m > Also now reproduced with > spark.memory.offHeap.enabled false > {noformat} > # > # A fatal error has been detected by the Java Runtime Environment: > # > # SIGSEGV (0xb) at pc=0x00007f4559b4d4bd, pid=14182, tid=139935319750400 > # > # JRE version: OpenJDK Runtime Environment (8.0_91-b14) (build 1.8.0_91-b14) > # Java VM: OpenJDK 64-Bit Server VM (25.91-b14 mixed mode linux-amd64 > compressed oops) > # Problematic frame: > # J 4816 C2 > org.apache.spark.unsafe.types.UTF8String.compareTo(Lorg/apache/spark/unsafe/types/UTF8String;)I > (64 bytes) @ 0x00007f4559b4d4bd [0x00007f4559b4d460+0x5d] > {noformat} > We initially saw this on IBM java on PowerPC box but is recreatable on linux > with OpenJDK. On linux with IBM Java 8 we see a null pointer exception at the > same code point: > {noformat} > 16/06/08 11:14:58 ERROR Executor: Exception in task 1.0 in stage 5.0 (TID 48) > java.lang.NullPointerException > at > org.apache.spark.unsafe.types.UTF8String.compareTo(UTF8String.java:831) > at org.apache.spark.unsafe.types.UTF8String.compare(UTF8String.java:844) > at > org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.findNextInnerJoinRows$(Unknown > Source) > 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$doExecute$2$$anon$2.hasNext(WholeStageCodegenExec.scala:377) > at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408) > at > scala.collection.convert.Wrappers$IteratorWrapper.hasNext(Wrappers.scala:30) > at org.spark_project.guava.collect.Ordering.leastOf(Ordering.java:664) > at org.apache.spark.util.collection.Utils$.takeOrdered(Utils.scala:37) > at > org.apache.spark.rdd.RDD$$anonfun$takeOrdered$1$$anonfun$30.apply(RDD.scala:1365) > at > org.apache.spark.rdd.RDD$$anonfun$takeOrdered$1$$anonfun$30.apply(RDD.scala:1362) > at > org.apache.spark.rdd.RDD$$anonfun$mapPartitions$1$$anonfun$apply$23.apply(RDD.scala:757) > at > org.apache.spark.rdd.RDD$$anonfun$mapPartitions$1$$anonfun$apply$23.apply(RDD.scala:757) > at > org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) > at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:318) > at org.apache.spark.rdd.RDD.iterator(RDD.scala:282) > 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:1153) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) > at java.lang.Thread.run(Thread.java:785) > {noformat} -- 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