Skip to content

Commit

Permalink
[SPARK-2970] [SQL] spark-sql script ends with IOException when EventL…
Browse files Browse the repository at this point in the history
…ogging 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

(cherry picked from commit 905dc4b)
Signed-off-by: Michael Armbrust <michael@databricks.com>
  • Loading branch information
sarutak authored and marmbrus committed Aug 14, 2014
1 parent e8e7f17 commit b5b632c
Showing 1 changed file with 8 additions and 2 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,8 @@ import jline.{ConsoleReader, History}
import org.apache.commons.lang.StringUtils
import org.apache.commons.logging.LogFactory
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.common.LogUtils.LogInitializationException
import org.apache.hadoop.hive.common.{HiveInterruptCallback, HiveInterruptUtils, LogUtils}
Expand Down Expand Up @@ -116,13 +118,17 @@ private[hive] object SparkSQLCLIDriver {
SessionState.start(sessionState)

// 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() {
override def run() {
SparkSQLEnv.stop()
}
}
)
, FileSystem.SHUTDOWN_HOOK_PRIORITY - 1)

// "-h" option has been passed, so connect to Hive thrift server.
if (sessionState.getHost != null) {
Expand Down

0 comments on commit b5b632c

Please sign in to comment.