[ 
https://issues.apache.org/jira/browse/HIVE-8853?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14270418#comment-14270418
 ] 

Brock Noland commented on HIVE-8853:
------------------------------------

[~jxiang] I took some thread dumps of an executor JVM during execution with 
vectorization turned on and I saw a ton of thread dumps here like the ones 
below.

{noformat}
"Executor task launch worker-4" daemon prio=10 tid=0x00007f8394048800 
nid=0x707a runnable [0x00007f8457dfb000]
   java.lang.Thread.State: RUNNABLE
        at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
        at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
        at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:79)
        at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:87)
        - locked <0x0000000281c57b70> (a sun.nio.ch.Util$2)
        - locked <0x0000000281c57b80> (a java.util.Collections$UnmodifiableSet)
        - locked <0x0000000281c57b28> (a sun.nio.ch.EPollSelectorImpl)
        at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:98)
        at 
org.apache.hadoop.net.SocketIOWithTimeout$SelectorPool.select(SocketIOWithTimeout.java:335)
        at 
org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:157)
        at 
org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:161)
        at 
org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver.readChannelFully(PacketReceiver.java:258)
        at 
org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver.doReadFully(PacketReceiver.java:209)
        at 
org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver.doRead(PacketReceiver.java:171)
        at 
org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver.receiveNextPacket(PacketReceiver.java:102)
        at 
org.apache.hadoop.hdfs.RemoteBlockReader2.readNextPacket(RemoteBlockReader2.java:186)
        at 
org.apache.hadoop.hdfs.RemoteBlockReader2.read(RemoteBlockReader2.java:146)
        - locked <0x0000000718424118> (a 
org.apache.hadoop.hdfs.RemoteBlockReader2)
        at 
org.apache.hadoop.hdfs.DFSInputStream$ByteArrayStrategy.doRead(DFSInputStream.java:693)
        at 
org.apache.hadoop.hdfs.DFSInputStream.readBuffer(DFSInputStream.java:749)
        - eliminated <0x00000007184169b8> (a 
org.apache.hadoop.hdfs.DFSInputStream)
        at 
org.apache.hadoop.hdfs.DFSInputStream.readWithStrategy(DFSInputStream.java:806)
        at org.apache.hadoop.hdfs.DFSInputStream.read(DFSInputStream.java:847)
        - locked <0x00000007184169b8> (a org.apache.hadoop.hdfs.DFSInputStream)
        at java.io.DataInputStream.read(DataInputStream.java:149)
        at 
org.apache.hive.com.esotericsoftware.kryo.io.Input.fill(Input.java:146)
        at 
org.apache.hive.com.esotericsoftware.kryo.io.Input.require(Input.java:178)
        at 
org.apache.hive.com.esotericsoftware.kryo.io.Input.readUtf8_slow(Input.java:542)
        at 
org.apache.hive.com.esotericsoftware.kryo.io.Input.readUtf8(Input.java:535)
        at 
org.apache.hive.com.esotericsoftware.kryo.io.Input.readString(Input.java:465)
        at 
org.apache.hive.com.esotericsoftware.kryo.serializers.DefaultSerializers$StringSerializer.read(DefaultSerializers.java:171)
        at 
org.apache.hive.com.esotericsoftware.kryo.serializers.DefaultSerializers$StringSerializer.read(DefaultSerializers.java:160)
        at 
org.apache.hive.com.esotericsoftware.kryo.Kryo.readClassAndObject(Kryo.java:776)
        at 
org.apache.hive.com.esotericsoftware.kryo.serializers.MapSerializer.read(MapSerializer.java:139)
        at 
org.apache.hive.com.esotericsoftware.kryo.serializers.MapSerializer.read(MapSerializer.java:17)
        at 
org.apache.hive.com.esotericsoftware.kryo.Kryo.readObject(Kryo.java:694)
        at 
org.apache.hive.com.esotericsoftware.kryo.serializers.ObjectField.read(ObjectField.java:106)
        at 
org.apache.hive.com.esotericsoftware.kryo.serializers.FieldSerializer.read(FieldSerializer.java:507)
        at 
org.apache.hive.com.esotericsoftware.kryo.Kryo.readObject(Kryo.java:694)
        at 
org.apache.hive.com.esotericsoftware.kryo.serializers.ObjectField.read(ObjectField.java:106)
        at 
org.apache.hive.com.esotericsoftware.kryo.serializers.FieldSerializer.read(FieldSerializer.java:507)
        at 
org.apache.hive.com.esotericsoftware.kryo.Kryo.readClassAndObject(Kryo.java:776)
        at 
org.apache.hive.com.esotericsoftware.kryo.serializers.MapSerializer.read(MapSerializer.java:139)
        at 
org.apache.hive.com.esotericsoftware.kryo.serializers.MapSerializer.read(MapSerializer.java:17)
        at 
org.apache.hive.com.esotericsoftware.kryo.Kryo.readObject(Kryo.java:694)
        at 
org.apache.hive.com.esotericsoftware.kryo.serializers.ObjectField.read(ObjectField.java:106)
        at 
org.apache.hive.com.esotericsoftware.kryo.serializers.FieldSerializer.read(FieldSerializer.java:507)
        at 
org.apache.hive.com.esotericsoftware.kryo.Kryo.readObject(Kryo.java:672)
        at 
org.apache.hadoop.hive.ql.exec.Utilities.deserializeObjectByKryo(Utilities.java:1050)
        at 
org.apache.hadoop.hive.ql.exec.Utilities.deserializePlan(Utilities.java:941)
        at 
org.apache.hadoop.hive.ql.exec.Utilities.deserializePlan(Utilities.java:955)
        at 
org.apache.hadoop.hive.ql.exec.Utilities.getBaseWork(Utilities.java:396)
        at 
org.apache.hadoop.hive.ql.exec.Utilities.getMapWork(Utilities.java:287)
        at 
org.apache.hadoop.hive.ql.exec.Utilities.getMapWorkAllScratchColumnVectorTypeMaps(Utilities.java:442)
        at 
org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatchCtx.init(VectorizedRowBatchCtx.java:173)
        at 
org.apache.hadoop.hive.ql.io.orc.VectorizedOrcInputFormat$VectorizedOrcRecordReader.<init>(VectorizedOrcInputFormat.java:71)
        at 
org.apache.hadoop.hive.ql.io.orc.VectorizedOrcInputFormat.getRecordReader(VectorizedOrcInputFormat.java:153)
        at 
org.apache.hadoop.hive.ql.io.orc.OrcInputFormat.createVectorizedReader(OrcInputFormat.java:1014)
        at 
org.apache.hadoop.hive.ql.io.orc.OrcInputFormat.getRecordReader(OrcInputFormat.java:1028)
        at 
org.apache.hadoop.hive.ql.io.CombineHiveRecordReader.<init>(CombineHiveRecordReader.java:66)
        at sun.reflect.GeneratedConstructorAccessor56.newInstance(Unknown 
Source)
        at 
sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
        at java.lang.reflect.Constructor.newInstance(Constructor.java:526)
        at 
org.apache.hadoop.hive.shims.HadoopShimsSecure$CombineFileRecordReader.initNextRecordReader(HadoopShimsSecure.java:266)
        at 
org.apache.hadoop.hive.shims.HadoopShimsSecure$CombineFileRecordReader.next(HadoopShimsSecure.java:154)
        at org.apache.spark.rdd.HadoopRDD$$anon$1.getNext(HadoopRDD.scala:244)
        at org.apache.spark.rdd.HadoopRDD$$anon$1.getNext(HadoopRDD.scala:210)
        at org.apache.spark.util.NextIterator.hasNext(NextIterator.scala:71)
        at 
org.apache.spark.InterruptibleIterator.hasNext(InterruptibleIterator.scala:39)
        at 
scala.collection.convert.Wrappers$IteratorWrapper.hasNext(Wrappers.scala:29)
        at 
org.apache.hadoop.hive.ql.exec.spark.HiveBaseFunctionResultList$ResultIterator.hasNext(HiveBaseFunctionResultList.java:96)
        at 
scala.collection.convert.Wrappers$JIteratorWrapper.hasNext(Wrappers.scala:41)
        at 
org.apache.spark.util.collection.ExternalSorter.insertAll(ExternalSorter.scala:210)
        at 
org.apache.spark.shuffle.sort.SortShuffleWriter.write(SortShuffleWriter.scala:65)
        at 
org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:68)
        at 
org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41)
        at org.apache.spark.scheduler.Task.run(Task.scala:56)
        at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:196)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
        at java.lang.Thread.run(Thread.java:745)


"Executor task launch worker-0" daemon prio=10 tid=0x00007f839402a000 
nid=0x7076 runnable [0x00007f845cfa8000]
   java.lang.Thread.State: RUNNABLE
        at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
        at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
        at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:79)
        at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:87)
        - locked <0x0000000285a78858> (a sun.nio.ch.Util$2)
        - locked <0x0000000285a78848> (a java.util.Collections$UnmodifiableSet)
        - locked <0x0000000285a78630> (a sun.nio.ch.EPollSelectorImpl)
        at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:98)
        at 
org.apache.hadoop.net.SocketIOWithTimeout$SelectorPool.select(SocketIOWithTimeout.java:335)
        at 
org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:157)
        at 
org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:161)
        at 
org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:131)
        at 
org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:118)
        at java.io.FilterInputStream.read(FilterInputStream.java:83)
        at 
org.apache.hadoop.hdfs.protocolPB.PBHelper.vintPrefixed(PBHelper.java:2201)
        at 
org.apache.hadoop.hdfs.RemoteBlockReader2.newBlockReader(RemoteBlockReader2.java:408)
        at 
org.apache.hadoop.hdfs.BlockReaderFactory.getRemoteBlockReader(BlockReaderFactory.java:796)
        at 
org.apache.hadoop.hdfs.BlockReaderFactory.getRemoteBlockReaderFromTcp(BlockReaderFactory.java:674)
        at 
org.apache.hadoop.hdfs.BlockReaderFactory.build(BlockReaderFactory.java:337)
        at 
org.apache.hadoop.hdfs.DFSInputStream.blockSeekTo(DFSInputStream.java:576)
        - locked <0x0000000642757968> (a org.apache.hadoop.hdfs.DFSInputStream)
        at 
org.apache.hadoop.hdfs.DFSInputStream.readWithStrategy(DFSInputStream.java:800)
        at org.apache.hadoop.hdfs.DFSInputStream.read(DFSInputStream.java:847)
        - locked <0x0000000642757968> (a org.apache.hadoop.hdfs.DFSInputStream)
        at java.io.DataInputStream.read(DataInputStream.java:149)
        at 
org.apache.hive.com.esotericsoftware.kryo.io.Input.fill(Input.java:146)
        at 
org.apache.hive.com.esotericsoftware.kryo.io.Input.require(Input.java:178)
        at 
org.apache.hive.com.esotericsoftware.kryo.io.Input.readVarInt(Input.java:355)
        at 
org.apache.hive.com.esotericsoftware.kryo.Kryo.readReferenceOrNull(Kryo.java:809)
        at 
org.apache.hive.com.esotericsoftware.kryo.Kryo.readObject(Kryo.java:670)
        at 
org.apache.hadoop.hive.ql.exec.Utilities.deserializeObjectByKryo(Utilities.java:1050)
        at 
org.apache.hadoop.hive.ql.exec.Utilities.deserializePlan(Utilities.java:941)
        at 
org.apache.hadoop.hive.ql.exec.Utilities.deserializePlan(Utilities.java:955)
        at 
org.apache.hadoop.hive.ql.exec.Utilities.getBaseWork(Utilities.java:396)
        at 
org.apache.hadoop.hive.ql.exec.Utilities.getMapWork(Utilities.java:287)
        at 
org.apache.hadoop.hive.ql.exec.Utilities.getMapRedWork(Utilities.java:273)
        at 
org.apache.hadoop.hive.ql.exec.Utilities.isVectorMode(Utilities.java:3618)
        at 
org.apache.hadoop.hive.ql.io.orc.OrcInputFormat.getRecordReader(OrcInputFormat.java:1022)
        at 
org.apache.hadoop.hive.ql.io.CombineHiveRecordReader.<init>(CombineHiveRecordReader.java:66)
        at sun.reflect.GeneratedConstructorAccessor56.newInstance(Unknown 
Source)
        at 
sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
        at java.lang.reflect.Constructor.newInstance(Constructor.java:526)
        at 
org.apache.hadoop.hive.shims.HadoopShimsSecure$CombineFileRecordReader.initNextRecordReader(HadoopShimsSecure.java:266)
        at 
org.apache.hadoop.hive.shims.HadoopShimsSecure$CombineFileRecordReader.next(HadoopShimsSecure.java:154)
        at org.apache.spark.rdd.HadoopRDD$$anon$1.getNext(HadoopRDD.scala:244)
        at org.apache.spark.rdd.HadoopRDD$$anon$1.getNext(HadoopRDD.scala:210)
        at org.apache.spark.util.NextIterator.hasNext(NextIterator.scala:71)
        at 
org.apache.spark.InterruptibleIterator.hasNext(InterruptibleIterator.scala:39)
        at 
scala.collection.convert.Wrappers$IteratorWrapper.hasNext(Wrappers.scala:29)
        at 
org.apache.hadoop.hive.ql.exec.spark.HiveBaseFunctionResultList$ResultIterator.hasNext(HiveBaseFunctionResultList.java:96)
        at 
scala.collection.convert.Wrappers$JIteratorWrapper.hasNext(Wrappers.scala:41)
        at 
org.apache.spark.util.collection.ExternalSorter.insertAll(ExternalSorter.scala:210)
        at 
org.apache.spark.shuffle.sort.SortShuffleWriter.write(SortShuffleWriter.scala:65)
        at 
org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:68)
        at 
org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41)
        at org.apache.spark.scheduler.Task.run(Task.scala:56)
        at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:196)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
        at java.lang.Thread.run(Thread.java:745)

"Executor task launch worker-0" daemon prio=10 tid=0x00007f839402a000 
nid=0x7076 runnable [0x00007f845cfa8000]
   java.lang.Thread.State: RUNNABLE
        at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
        at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
        at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:79)
        at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:87)
        - locked <0x0000000285a81a90> (a sun.nio.ch.Util$2)
        - locked <0x0000000285a81a80> (a java.util.Collections$UnmodifiableSet)
        - locked <0x0000000285a74618> (a sun.nio.ch.EPollSelectorImpl)
        at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:98)
        at 
org.apache.hadoop.net.SocketIOWithTimeout$SelectorPool.select(SocketIOWithTimeout.java:335)
        at 
org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:157)
        at 
org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:161)
        at 
org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver.readChannelFully(PacketReceiver.java:258)
        at 
org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver.doReadFully(PacketReceiver.java:209)
        at 
org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver.doRead(PacketReceiver.java:171)
        at 
org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver.receiveNextPacket(PacketReceiver.java:102)
        at 
org.apache.hadoop.hdfs.RemoteBlockReader2.readNextPacket(RemoteBlockReader2.java:186)
        at 
org.apache.hadoop.hdfs.RemoteBlockReader2.read(RemoteBlockReader2.java:146)
        - locked <0x000000073f52a768> (a 
org.apache.hadoop.hdfs.RemoteBlockReader2)
        at 
org.apache.hadoop.hdfs.DFSInputStream$ByteArrayStrategy.doRead(DFSInputStream.java:693)
        at 
org.apache.hadoop.hdfs.DFSInputStream.readBuffer(DFSInputStream.java:749)
        - eliminated <0x000000073f51d068> (a 
org.apache.hadoop.hdfs.DFSInputStream)
        at 
org.apache.hadoop.hdfs.DFSInputStream.readWithStrategy(DFSInputStream.java:806)
        at org.apache.hadoop.hdfs.DFSInputStream.read(DFSInputStream.java:847)
        - locked <0x000000073f51d068> (a org.apache.hadoop.hdfs.DFSInputStream)
        at java.io.DataInputStream.read(DataInputStream.java:149)
        at 
org.apache.hive.com.esotericsoftware.kryo.io.Input.fill(Input.java:146)
        at 
org.apache.hive.com.esotericsoftware.kryo.io.Input.require(Input.java:178)
        at 
org.apache.hive.com.esotericsoftware.kryo.io.Input.readUtf8_slow(Input.java:542)
        at 
org.apache.hive.com.esotericsoftware.kryo.io.Input.readUtf8(Input.java:535)
        at 
org.apache.hive.com.esotericsoftware.kryo.io.Input.readString(Input.java:465)
        at 
org.apache.hive.com.esotericsoftware.kryo.serializers.DefaultSerializers$StringSerializer.read(DefaultSerializers.java:171)
        at 
org.apache.hive.com.esotericsoftware.kryo.serializers.DefaultSerializers$StringSerializer.read(DefaultSerializers.java:160)
        at 
org.apache.hive.com.esotericsoftware.kryo.Kryo.readClassAndObject(Kryo.java:776)
        at 
org.apache.hive.com.esotericsoftware.kryo.serializers.MapSerializer.read(MapSerializer.java:139)
        at 
org.apache.hive.com.esotericsoftware.kryo.serializers.MapSerializer.read(MapSerializer.java:17)
        at 
org.apache.hive.com.esotericsoftware.kryo.Kryo.readObject(Kryo.java:694)
        at 
org.apache.hive.com.esotericsoftware.kryo.serializers.ObjectField.read(ObjectField.java:106)
        at 
org.apache.hive.com.esotericsoftware.kryo.serializers.FieldSerializer.read(FieldSerializer.java:507)
        at 
org.apache.hive.com.esotericsoftware.kryo.Kryo.readObject(Kryo.java:694)
        at 
org.apache.hive.com.esotericsoftware.kryo.serializers.ObjectField.read(ObjectField.java:106)
        at 
org.apache.hive.com.esotericsoftware.kryo.serializers.FieldSerializer.read(FieldSerializer.java:507)
        at 
org.apache.hive.com.esotericsoftware.kryo.Kryo.readClassAndObject(Kryo.java:776)
        at 
org.apache.hive.com.esotericsoftware.kryo.serializers.MapSerializer.read(MapSerializer.java:139)
        at 
org.apache.hive.com.esotericsoftware.kryo.serializers.MapSerializer.read(MapSerializer.java:17)
        at 
org.apache.hive.com.esotericsoftware.kryo.Kryo.readObject(Kryo.java:694)
        at 
org.apache.hive.com.esotericsoftware.kryo.serializers.ObjectField.read(ObjectField.java:106)
        at 
org.apache.hive.com.esotericsoftware.kryo.serializers.FieldSerializer.read(FieldSerializer.java:507)
        at 
org.apache.hive.com.esotericsoftware.kryo.Kryo.readObject(Kryo.java:672)
        at 
org.apache.hadoop.hive.ql.exec.Utilities.deserializeObjectByKryo(Utilities.java:1050)
        at 
org.apache.hadoop.hive.ql.exec.Utilities.deserializePlan(Utilities.java:941)
        at 
org.apache.hadoop.hive.ql.exec.Utilities.deserializePlan(Utilities.java:955)
        at 
org.apache.hadoop.hive.ql.exec.Utilities.getBaseWork(Utilities.java:396)
        at 
org.apache.hadoop.hive.ql.exec.Utilities.getMapWork(Utilities.java:287)
        at 
org.apache.hadoop.hive.ql.exec.Utilities.getMapRedWork(Utilities.java:273)
        at 
org.apache.hadoop.hive.ql.exec.Utilities.isVectorMode(Utilities.java:3618)
        at 
org.apache.hadoop.hive.ql.io.orc.OrcInputFormat.getRecordReader(OrcInputFormat.java:1022)
        at 
org.apache.hadoop.hive.ql.io.CombineHiveRecordReader.<init>(CombineHiveRecordReader.java:66)
        at sun.reflect.GeneratedConstructorAccessor56.newInstance(Unknown 
Source)
        at 
sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
        at java.lang.reflect.Constructor.newInstance(Constructor.java:526)
        at 
org.apache.hadoop.hive.shims.HadoopShimsSecure$CombineFileRecordReader.initNextRecordReader(HadoopShimsSecure.java:266)
        at 
org.apache.hadoop.hive.shims.HadoopShimsSecure$CombineFileRecordReader.next(HadoopShimsSecure.java:154)
        at org.apache.spark.rdd.HadoopRDD$$anon$1.getNext(HadoopRDD.scala:244)
        at org.apache.spark.rdd.HadoopRDD$$anon$1.getNext(HadoopRDD.scala:210)
        at org.apache.spark.util.NextIterator.hasNext(NextIterator.scala:71)
        at 
org.apache.spark.InterruptibleIterator.hasNext(InterruptibleIterator.scala:39)
        at 
scala.collection.convert.Wrappers$IteratorWrapper.hasNext(Wrappers.scala:29)
        at 
org.apache.hadoop.hive.ql.exec.spark.HiveBaseFunctionResultList$ResultIterator.hasNext(HiveBaseFunctionResultList.java:96)
        at 
scala.collection.convert.Wrappers$JIteratorWrapper.hasNext(Wrappers.scala:41)
        at 
org.apache.spark.util.collection.ExternalSorter.insertAll(ExternalSorter.scala:210)
        at 
org.apache.spark.shuffle.sort.SortShuffleWriter.write(SortShuffleWriter.scala:65)
        at 
org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:68)
        at 
org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41)
        at org.apache.spark.scheduler.Task.run(Task.scala:56)
        at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:196)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
        at java.lang.Thread.run(Thread.java:745)
{noformat}

I feel like there might be something wrong here:

{noformat}
        at 
org.apache.hadoop.hive.ql.io.orc.OrcInputFormat.getRecordReader(OrcInputFormat.java:1028)
        at 
org.apache.hadoop.hive.ql.io.CombineHiveRecordReader.<init>(CombineHiveRecordReader.java:66)
        at sun.reflect.GeneratedConstructorAccessor56.newInstance(Unknown 
Source)
        at 
sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
        at java.lang.reflect.Constructor.newInstance(Constructor.java:526)
        at 
org.apache.hadoop.hive.shims.HadoopShimsSecure$CombineFileRecordReader.initNextRecordReader(HadoopShimsSecure.java:266)
        at 
org.apache.hadoop.hive.shims.HadoopShimsSecure$CombineFileRecordReader.next(HadoopShimsSecure.java:154)
        at org.apache.spark.rdd.HadoopRDD$$anon$1.getNext(HadoopRDD.scala:244)
{noformat}

> Make vectorization work with Spark [Spark Branch]
> -------------------------------------------------
>
>                 Key: HIVE-8853
>                 URL: https://issues.apache.org/jira/browse/HIVE-8853
>             Project: Hive
>          Issue Type: Sub-task
>          Components: Spark
>            Reporter: Xuefu Zhang
>            Assignee: Jimmy Xiang
>
> In Hive to make vectorization work, the reader needs to be also vectorized, 
> which means that the reader can read a chunk of rows (or a list of column 
> chunks) instead of one row at a time. However, we use Spark RDD for reading, 
> which again utilized the underlying inputformat to read. Subsequent 
> processing also needs to hapen in batches. We need to make sure that 
> vectorizatoin is working as expected.



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

Reply via email to