diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala
index 2715fa93d0e98a2f727b3f262693792df1d41cb9..87a17cebdc10ce661510f1ad7689d3c3ea35a7ec 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala
@@ -42,9 +42,11 @@ object FileStreamSink extends Logging {
         try {
           val hdfsPath = new Path(singlePath)
           val fs = hdfsPath.getFileSystem(hadoopConf)
-          val metadataPath = new Path(hdfsPath, metadataDir)
-          val res = fs.exists(metadataPath)
-          res
+          if (fs.isDirectory(hdfsPath)) {
+            fs.exists(new Path(hdfsPath, metadataDir))
+          } else {
+            false
+          }
         } catch {
           case NonFatal(e) =>
             logWarning(s"Error while looking for metadata directory.")