Skip to content
Snippets Groups Projects
Commit 905dc4b4 authored by Kousuke Saruta's avatar Kousuke Saruta Committed by Michael Armbrust
Browse files

[SPARK-2970] [SQL] spark-sql script ends with IOException when EventLogging is enabled

Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>

Closes #1891 from sarutak/SPARK-2970 and squashes the following commits:

4a2d2fe [Kousuke Saruta] Modified comment style
8bd833c [Kousuke Saruta] Modified style
6c0997c [Kousuke Saruta] Modified the timing of shutdown hook execution. It should be executed before shutdown hook of o.a.h.f.FileSystem
parent 9fde1ff5
No related branches found
No related tags found
No related merge requests found
...@@ -26,6 +26,8 @@ import jline.{ConsoleReader, History} ...@@ -26,6 +26,8 @@ import jline.{ConsoleReader, History}
import org.apache.commons.lang.StringUtils import org.apache.commons.lang.StringUtils
import org.apache.commons.logging.LogFactory import org.apache.commons.logging.LogFactory
import org.apache.hadoop.conf.Configuration import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.FileSystem
import org.apache.hadoop.util.ShutdownHookManager
import org.apache.hadoop.hive.cli.{CliDriver, CliSessionState, OptionsProcessor} import org.apache.hadoop.hive.cli.{CliDriver, CliSessionState, OptionsProcessor}
import org.apache.hadoop.hive.common.LogUtils.LogInitializationException import org.apache.hadoop.hive.common.LogUtils.LogInitializationException
import org.apache.hadoop.hive.common.{HiveInterruptCallback, HiveInterruptUtils, LogUtils} import org.apache.hadoop.hive.common.{HiveInterruptCallback, HiveInterruptUtils, LogUtils}
...@@ -116,13 +118,17 @@ private[hive] object SparkSQLCLIDriver { ...@@ -116,13 +118,17 @@ private[hive] object SparkSQLCLIDriver {
SessionState.start(sessionState) SessionState.start(sessionState)
// Clean up after we exit // Clean up after we exit
Runtime.getRuntime.addShutdownHook( /**
* This should be executed before shutdown hook of
* FileSystem to avoid race condition of FileSystem operation
*/
ShutdownHookManager.get.addShutdownHook(
new Thread() { new Thread() {
override def run() { override def run() {
SparkSQLEnv.stop() SparkSQLEnv.stop()
} }
} }
) , FileSystem.SHUTDOWN_HOOK_PRIORITY - 1)
// "-h" option has been passed, so connect to Hive thrift server. // "-h" option has been passed, so connect to Hive thrift server.
if (sessionState.getHost != null) { if (sessionState.getHost != null) {
......
0% Loading or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment