[
https://issues.apache.org/jira/browse/HIVE-15237?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Xuefu Zhang updated HIVE-15237:
-------------------------------
Attachment: HIVE-15237.patch
Thanks, Rui. For your reference, I attached a POC patch that was based on CDH
5.7. With the patch, I got the following on the console, which seems having too
much and duplicated info. Please feel free to adapt and modify.
{code}
2016-11-18 00:10:17,732 Stage-0_0: 1(+12,-41)/23 Stage-1_0: 0/28
Status: Failed: org.apache.spark.SparkException:
java.util.concurrent.ExecutionException: Exception thrown by job
at
org.apache.spark.JavaFutureActionWrapper.getImpl(FutureAction.scala:311)
at org.apache.spark.JavaFutureActionWrapper.get(FutureAction.scala:316)
at
org.apache.hive.spark.client.RemoteDriver$JobWrapper.call(RemoteDriver.java:382)
at
org.apache.hive.spark.client.RemoteDriver$JobWrapper.call(RemoteDriver.java:335)
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
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)
Caused by: org.apache.spark.SparkException: Job aborted due to stage failure:
Task 9 in stage 0.0 failed 4 times, most recent failure: Lost task 9.3 in stage
0.0 (TID 39, hadoopworker863-sjc1.prod.uber.internal): java.io.IOException:
java.lang.reflect.InvocationTargetException
at
org.apache.hadoop.hive.io.HiveIOExceptionHandlerChain.handleRecordReaderCreationException(HiveIOExceptionHandlerChain.java:97)
at
org.apache.hadoop.hive.io.HiveIOExceptionHandlerUtil.handleRecordReaderCreationException(HiveIOExceptionHandlerUtil.java:57)
at
org.apache.hadoop.hive.shims.HadoopShimsSecure$CombineFileRecordReader.initNextRecordReader(HadoopShimsSecure.java:265)
at
org.apache.hadoop.hive.shims.HadoopShimsSecure$CombineFileRecordReader.<init>(HadoopShimsSecure.java:212)
at
org.apache.hadoop.hive.shims.HadoopShimsSecure$CombineFileInputFormatShim.getRecordReader(HadoopShimsSecure.java:332)
at
com.uber.hoodie.hadoop.hive.HoodieCombineHiveInputFormat.getRecordReader(HoodieCombineHiveInputFormat.java:741)
at org.apache.spark.rdd.HadoopRDD$$anon$1.<init>(HadoopRDD.scala:237)
at org.apache.spark.rdd.HadoopRDD.compute(HadoopRDD.scala:208)
at org.apache.spark.rdd.HadoopRDD.compute(HadoopRDD.scala:101)
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.ShuffleMapTask.runTask(ShuffleMapTask.scala:73)
at
org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41)
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)
Caused by: java.lang.reflect.InvocationTargetException
at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
at
sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
at
sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
at
org.apache.hadoop.hive.shims.HadoopShimsSecure$CombineFileRecordReader.initNextRecordReader(HadoopShimsSecure.java:251)
... 18 more
Caused by: java.lang.OutOfMemoryError: Java heap space
at
parquet.hadoop.ParquetFileReader$ConsecutiveChunkList.readAll(ParquetFileReader.java:778)
at
parquet.hadoop.ParquetFileReader.readNextRowGroup(ParquetFileReader.java:511)
at
parquet.hadoop.InternalParquetRecordReader.checkRead(InternalParquetRecordReader.java:129)
at
parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:212)
at
parquet.hadoop.ParquetRecordReader.nextKeyValue(ParquetRecordReader.java:227)
at
org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper.<init>(ParquetRecordReaderWrapper.java:123)
at
org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper.<init>(ParquetRecordReaderWrapper.java:85)
at
org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat.getRecordReader(MapredParquetInputFormat.java:72)
at
org.apache.hadoop.hive.ql.io.CombineHiveRecordReader.<init>(CombineHiveRecordReader.java:67)
... 23 more
Driver stacktrace:
at
org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1431)
at
org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1419)
at
org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1418)
at
scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:47)
at
org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1418)
at
org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:799)
at
org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:799)
at scala.Option.foreach(Option.scala:236)
at
org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:799)
at
org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:1640)
at
org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1599)
at
org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1588)
at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:48)
Caused by: java.io.IOException: java.lang.reflect.InvocationTargetException
at
org.apache.hadoop.hive.io.HiveIOExceptionHandlerChain.handleRecordReaderCreationException(HiveIOExceptionHandlerChain.java:97)
at
org.apache.hadoop.hive.io.HiveIOExceptionHandlerUtil.handleRecordReaderCreationException(HiveIOExceptionHandlerUtil.java:57)
at
org.apache.hadoop.hive.shims.HadoopShimsSecure$CombineFileRecordReader.initNextRecordReader(HadoopShimsSecure.java:265)
at
org.apache.hadoop.hive.shims.HadoopShimsSecure$CombineFileRecordReader.<init>(HadoopShimsSecure.java:212)
at
org.apache.hadoop.hive.shims.HadoopShimsSecure$CombineFileInputFormatShim.getRecordReader(HadoopShimsSecure.java:332)
at
com.uber.hoodie.hadoop.hive.HoodieCombineHiveInputFormat.getRecordReader(HoodieCombineHiveInputFormat.java:741)
at org.apache.spark.rdd.HadoopRDD$$anon$1.<init>(HadoopRDD.scala:237)
at org.apache.spark.rdd.HadoopRDD.compute(HadoopRDD.scala:208)
at org.apache.spark.rdd.HadoopRDD.compute(HadoopRDD.scala:101)
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.ShuffleMapTask.runTask(ShuffleMapTask.scala:73)
at
org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41)
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)
Caused by: java.lang.reflect.InvocationTargetException
at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
at
sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
at
sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
at
org.apache.hadoop.hive.shims.HadoopShimsSecure$CombineFileRecordReader.initNextRecordReader(HadoopShimsSecure.java:251)
... 18 more
Caused by: java.lang.OutOfMemoryError: Java heap space
at
parquet.hadoop.ParquetFileReader$ConsecutiveChunkList.readAll(ParquetFileReader.java:778)
at
parquet.hadoop.ParquetFileReader.readNextRowGroup(ParquetFileReader.java:511)
at
parquet.hadoop.InternalParquetRecordReader.checkRead(InternalParquetRecordReader.java:129)
at
parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:212)
at
parquet.hadoop.ParquetRecordReader.nextKeyValue(ParquetRecordReader.java:227)
at
org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper.<init>(ParquetRecordReaderWrapper.java:123)
at
org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper.<init>(ParquetRecordReaderWrapper.java:85)
at
org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat.getRecordReader(MapredParquetInputFormat.java:72)
at
org.apache.hadoop.hive.ql.io.CombineHiveRecordReader.<init>(CombineHiveRecordReader.java:67)
... 23 more
FAILED: Execution Error, return code 3 from
org.apache.hadoop.hive.ql.exec.spark.SparkTask.
java.util.concurrent.ExecutionException: Exception thrown by job
at
org.apache.spark.JavaFutureActionWrapper.getImpl(FutureAction.scala:311)
at org.apache.spark.JavaFutureActionWrapper.get(FutureAction.scala:316)
at
org.apache.hive.spark.client.RemoteDriver$JobWrapper.call(RemoteDriver.java:382)
at
org.apache.hive.spark.client.RemoteDriver$JobWrapper.call(RemoteDriver.java:335)
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
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)
Caused by: org.apache.spark.SparkException: Job aborted due to stage failure:
Task 9 in stage 0.0 failed 4 times, most recent failure: Lost task 9.3 in stage
0.0 (TID 39, hadoopworker863-sjc1.prod.uber.internal): java.io.IOException:
java.lang.reflect.InvocationTargetException
at
org.apache.hadoop.hive.io.HiveIOExceptionHandlerChain.handleRecordReaderCreationException(HiveIOExceptionHandlerChain.java:97)
at
org.apache.hadoop.hive.io.HiveIOExceptionHandlerUtil.handleRecordReaderCreationException(HiveIOExceptionHandlerUtil.java:57)
at
org.apache.hadoop.hive.shims.HadoopShimsSecure$CombineFileRecordReader.initNextRecordReader(HadoopShimsSecure.java:265)
at
org.apache.hadoop.hive.shims.HadoopShimsSecure$CombineFileRecordReader.<init>(HadoopShimsSecure.java:212)
at
org.apache.hadoop.hive.shims.HadoopShimsSecure$CombineFileInputFormatShim.getRecordReader(HadoopShimsSecure.java:332)
at
com.uber.hoodie.hadoop.hive.HoodieCombineHiveInputFormat.getRecordReader(HoodieCombineHiveInputFormat.java:741)
at org.apache.spark.rdd.HadoopRDD$$anon$1.<init>(HadoopRDD.scala:237)
at org.apache.spark.rdd.HadoopRDD.compute(HadoopRDD.scala:208)
at org.apache.spark.rdd.HadoopRDD.compute(HadoopRDD.scala:101)
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.ShuffleMapTask.runTask(ShuffleMapTask.scala:73)
at
org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41)
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)
Caused by: java.lang.reflect.InvocationTargetException
at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
at
sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
at
sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
at
org.apache.hadoop.hive.shims.HadoopShimsSecure$CombineFileRecordReader.initNextRecordReader(HadoopShimsSecure.java:251)
... 18 more
Caused by: java.lang.OutOfMemoryError: Java heap space
at
parquet.hadoop.ParquetFileReader$ConsecutiveChunkList.readAll(ParquetFileReader.java:778)
at
parquet.hadoop.ParquetFileReader.readNextRowGroup(ParquetFileReader.java:511)
at
parquet.hadoop.InternalParquetRecordReader.checkRead(InternalParquetRecordReader.java:129)
at
parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:212)
at
parquet.hadoop.ParquetRecordReader.nextKeyValue(ParquetRecordReader.java:227)
at
org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper.<init>(ParquetRecordReaderWrapper.java:123)
at
org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper.<init>(ParquetRecordReaderWrapper.java:85)
at
org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat.getRecordReader(MapredParquetInputFormat.java:72)
at
org.apache.hadoop.hive.ql.io.CombineHiveRecordReader.<init>(CombineHiveRecordReader.java:67)
... 23 more
Driver stacktrace:
at
org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1431)
at
org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1419)
at
org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1418)
at
scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:47)
at
org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1418)
at
org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:799)
at
org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:799)
at scala.Option.foreach(Option.scala:236)
at
org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:799)
at
org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:1640)
at
org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1599)
at
org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1588)
at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:48)
Caused by: java.io.IOException: java.lang.reflect.InvocationTargetException
at
org.apache.hadoop.hive.io.HiveIOExceptionHandlerChain.handleRecordReaderCreationException(HiveIOExceptionHandlerChain.java:97)
at
org.apache.hadoop.hive.io.HiveIOExceptionHandlerUtil.handleRecordReaderCreationException(HiveIOExceptionHandlerUtil.java:57)
at
org.apache.hadoop.hive.shims.HadoopShimsSecure$CombineFileRecordReader.initNextRecordReader(HadoopShimsSecure.java:265)
at
org.apache.hadoop.hive.shims.HadoopShimsSecure$CombineFileRecordReader.<init>(HadoopShimsSecure.java:212)
at
org.apache.hadoop.hive.shims.HadoopShimsSecure$CombineFileInputFormatShim.getRecordReader(HadoopShimsSecure.java:332)
at
com.uber.hoodie.hadoop.hive.HoodieCombineHiveInputFormat.getRecordReader(HoodieCombineHiveInputFormat.java:741)
at org.apache.spark.rdd.HadoopRDD$$anon$1.<init>(HadoopRDD.scala:237)
at org.apache.spark.rdd.HadoopRDD.compute(HadoopRDD.scala:208)
at org.apache.spark.rdd.HadoopRDD.compute(HadoopRDD.scala:101)
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.ShuffleMapTask.runTask(ShuffleMapTask.scala:73)
at
org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41)
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)
Caused by: java.lang.reflect.InvocationTargetException
at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
at
sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
at
sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
at
org.apache.hadoop.hive.shims.HadoopShimsSecure$CombineFileRecordReader.initNextRecordReader(HadoopShimsSecure.java:251)
... 18 more
Caused by: java.lang.OutOfMemoryError: Java heap space
at
parquet.hadoop.ParquetFileReader$ConsecutiveChunkList.readAll(ParquetFileReader.java:778)
at
parquet.hadoop.ParquetFileReader.readNextRowGroup(ParquetFileReader.java:511)
at
parquet.hadoop.InternalParquetRecordReader.checkRead(InternalParquetRecordReader.java:129)
at
parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:212)
at
parquet.hadoop.ParquetRecordReader.nextKeyValue(ParquetRecordReader.java:227)
at
org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper.<init>(ParquetRecordReaderWrapper.java:123)
at
org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper.<init>(ParquetRecordReaderWrapper.java:85)
at
org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat.getRecordReader(MapredParquetInputFormat.java:72)
at
org.apache.hadoop.hive.ql.io.CombineHiveRecordReader.<init>(CombineHiveRecordReader.java:67)
... 23 more
hive> quit;
{code}
> Propagate Spark job failure to Hive
> -----------------------------------
>
> Key: HIVE-15237
> URL: https://issues.apache.org/jira/browse/HIVE-15237
> Project: Hive
> Issue Type: Bug
> Components: Spark
> Affects Versions: 2.1.0
> Reporter: Xuefu Zhang
> Assignee: Rui Li
> Attachments: HIVE-15237.patch
>
>
> If a Spark job failed for some reason, Hive doesn't get any additional error
> message, which makes it very hard for user to figure out why. Here is an
> example:
> {code}
> Status: Running (Hive on Spark job[0])
> Job Progress Format
> CurrentTime StageId_StageAttemptId:
> SucceededTasksCount(+RunningTasksCount-FailedTasksCount)/TotalTasksCount
> [StageCost]
> 2016-11-17 21:32:53,134 Stage-0_0: 0/23 Stage-1_0: 0/28
> 2016-11-17 21:32:55,156 Stage-0_0: 0(+1)/23 Stage-1_0: 0/28
> 2016-11-17 21:32:57,167 Stage-0_0: 0(+3)/23 Stage-1_0: 0/28
> 2016-11-17 21:33:00,216 Stage-0_0: 0(+3)/23 Stage-1_0: 0/28
> 2016-11-17 21:33:03,251 Stage-0_0: 0(+3)/23 Stage-1_0: 0/28
> 2016-11-17 21:33:06,286 Stage-0_0: 0(+4)/23 Stage-1_0: 0/28
> 2016-11-17 21:33:09,308 Stage-0_0: 0(+2,-3)/23 Stage-1_0: 0/28
> 2016-11-17 21:33:12,332 Stage-0_0: 0(+2,-3)/23 Stage-1_0: 0/28
> 2016-11-17 21:33:13,338 Stage-0_0: 0(+21,-3)/23 Stage-1_0: 0/28
> 2016-11-17 21:33:15,349 Stage-0_0: 0(+21,-5)/23 Stage-1_0: 0/28
> 2016-11-17 21:33:16,358 Stage-0_0: 0(+18,-8)/23 Stage-1_0: 0/28
> 2016-11-17 21:33:19,373 Stage-0_0: 0(+21,-8)/23 Stage-1_0: 0/28
> 2016-11-17 21:33:22,400 Stage-0_0: 0(+18,-14)/23 Stage-1_0: 0/28
> 2016-11-17 21:33:23,404 Stage-0_0: 0(+15,-20)/23 Stage-1_0: 0/28
> 2016-11-17 21:33:24,408 Stage-0_0: 0(+12,-23)/23 Stage-1_0: 0/28
> 2016-11-17 21:33:25,417 Stage-0_0: 0(+9,-26)/23 Stage-1_0: 0/28
> 2016-11-17 21:33:26,420 Stage-0_0: 0(+12,-26)/23 Stage-1_0: 0/28
> 2016-11-17 21:33:28,427 Stage-0_0: 0(+9,-29)/23 Stage-1_0: 0/28
> 2016-11-17 21:33:29,432 Stage-0_0: 0(+12,-29)/23 Stage-1_0: 0/28
> 2016-11-17 21:33:31,444 Stage-0_0: 0(+18,-29)/23 Stage-1_0: 0/28
> 2016-11-17 21:33:34,464 Stage-0_0: 0(+18,-29)/23 Stage-1_0: 0/28
> Status: Failed
> FAILED: Execution Error, return code 3 from
> org.apache.hadoop.hive.ql.exec.spark.SparkTask
> {code}
> It would be better if we can propagate Spark error to Hive.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)