From 3c9d277bd2e91e429c8f3363a5b6c50739468406 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Thu, 14 Dec 2017 12:52:22 -0800 Subject: [PATCH] [SPARK-22788][streaming] Use correct hadoop config for fs append support. Still look at the old one in case any Spark user is setting it explicitly, though. --- .../scala/org/apache/spark/streaming/util/HdfsUtils.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/HdfsUtils.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/HdfsUtils.scala index 6a3b3200dccd..a6997359d64d 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/HdfsUtils.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/HdfsUtils.scala @@ -29,7 +29,9 @@ private[streaming] object HdfsUtils { // If the file exists and we have append support, append instead of creating a new file val stream: FSDataOutputStream = { if (dfs.isFile(dfsPath)) { - if (conf.getBoolean("hdfs.append.support", false) || dfs.isInstanceOf[RawLocalFileSystem]) { + if (conf.getBoolean("dfs.support.append", true) || + conf.getBoolean("hdfs.append.support", false) || + dfs.isInstanceOf[RawLocalFileSystem]) { dfs.append(dfsPath) } else { throw new IllegalStateException("File exists and there is no append support!")