diff --git a/core/src/main/scala/org/apache/spark/util/FileLogger.scala b/core/src/main/scala/org/apache/spark/util/FileLogger.scala index 0e6d21b22023a610a94f480d37b7d422456c4545..6a95dc06e155dc8275f1a6a8ae0e9c7b97b7960e 100644 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -61,6 +61,14 @@ private[spark] class FileLogger( // Only defined if the file system scheme is not local private var hadoopDataStream: Option[FSDataOutputStream] = None + // The Hadoop APIs have changed over time, so we use reflection to figure out + // the correct method to use to flush a hadoop data stream. See SPARK-1518 + // for details. + private val hadoopFlushMethod = { + val cls = classOf[FSDataOutputStream] + scala.util.Try(cls.getMethod("hflush")).getOrElse(cls.getMethod("sync")) + } + private var writer: Option[PrintWriter] = None /** @@ -149,13 +157,13 @@ private[spark] class FileLogger( /** * Flush the writer to disk manually. * - * If the Hadoop FileSystem is used, the underlying FSDataOutputStream (r1.0.4) must be - * sync()'ed manually as it does not support flush(), which is invoked by when higher - * level streams are flushed. + * When using a Hadoop filesystem, we need to invoke the hflush or sync + * method. In HDFS, hflush guarantees that the data gets to all the + * DataNodes. */ def flush() { writer.foreach(_.flush()) - hadoopDataStream.foreach(_.sync()) + hadoopDataStream.foreach(hadoopFlushMethod.invoke(_)) } /**