diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py
index 78d992e415489d87d003af032fdedbd8c83b1df6..f7c354f51330caf288eeb56b17fe0b3eaa2e2bcc 100644
--- a/python/pyspark/sql/readwriter.py
+++ b/python/pyspark/sql/readwriter.py
@@ -633,7 +633,7 @@ class DataFrameWriter(OptionUtils):
 
     @since(2.0)
     def csv(self, path, mode=None, compression=None, sep=None, quote=None, escape=None,
-            header=None, nullValue=None, escapeQuotes=None):
+            header=None, nullValue=None, escapeQuotes=None, quoteAll=None):
         """Saves the content of the :class:`DataFrame` in CSV format at the specified path.
 
         :param path: the path in any Hadoop supported file system
@@ -658,6 +658,9 @@ class DataFrameWriter(OptionUtils):
         :param escapeQuotes: A flag indicating whether values containing quotes should always
                              be enclosed in quotes. If None is set, it uses the default value
                              ``true``, escaping all values containing a quote character.
+        :param quoteAll: A flag indicating whether all values should always be enclosed in
+                          quotes. If None is set, it uses the default value ``false``,
+                          only escaping values containing a quote character.
         :param header: writes the names of columns as the first line. If None is set, it uses
                        the default value, ``false``.
         :param nullValue: sets the string representation of a null value. If None is set, it uses
@@ -667,7 +670,7 @@ class DataFrameWriter(OptionUtils):
         """
         self.mode(mode)
         self._set_opts(compression=compression, sep=sep, quote=quote, escape=escape, header=header,
-                       nullValue=nullValue, escapeQuotes=escapeQuotes)
+                       nullValue=nullValue, escapeQuotes=escapeQuotes, quoteAll=quoteAll)
         self._jwrite.csv(path)
 
     @since(1.5)
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala
index f77af76d2bf3a4bcf66455c5ff42f8a85b41d613..12b304623d30b3fa1d659e240ae603b0b8820310 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala
@@ -537,6 +537,8 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) {
    * <li>`escapeQuotes` (default `true`): a flag indicating whether values containing
    * quotes should always be enclosed in quotes. Default is to escape all values containing
    * a quote character.</li>
+   * <li>`quoteAll` (default `false`): A flag indicating whether all values should always be
+   * enclosed in quotes. Default is to only escape values containing a quote character.</li>
    * <li>`header` (default `false`): writes the names of columns as the first line.</li>
    * <li>`nullValue` (default empty string): sets the string representation of a null value.</li>
    * <li>`compression` (default `null`): compression codec to use when saving to file. This can be
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala
index 581eda7e09a3e181a046bd88da83cb9dbf5c49da..22fb8163b1c0a671a71cb02342ae6a6978b05f2a 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala
@@ -115,6 +115,8 @@ private[sql] class CSVOptions(@transient private val parameters: Map[String, Str
 
   val maxMalformedLogPerPartition = getInt("maxMalformedLogPerPartition", 10)
 
+  val quoteAll = getBool("quoteAll", false)
+
   val inputBufferSize = 128
 
   val isCommentSet = this.comment != '\u0000'
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala
index bf62732dd40482d5e2dc45107c0ff846e1425ffb..13ae76d49893a4a1dfa9cca190e43ffa39326a33 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala
@@ -78,7 +78,7 @@ private[sql] class LineCsvWriter(params: CSVOptions, headers: Seq[String]) exten
   writerSettings.setNullValue(params.nullValue)
   writerSettings.setEmptyValue(params.nullValue)
   writerSettings.setSkipEmptyLines(true)
-  writerSettings.setQuoteAllFields(false)
+  writerSettings.setQuoteAllFields(params.quoteAll)
   writerSettings.setHeaders(headers: _*)
   writerSettings.setQuoteEscapingEnabled(params.escapeQuotes)
 
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala
index f170065132acda2d7fb91f9d68c415ff463a47af..311f1fa8d2aff7d7e5863060e58c5a1c759abf82 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala
@@ -366,6 +366,32 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils {
     }
   }
 
+  test("save csv with quoteAll enabled") {
+    withTempDir { dir =>
+      val csvDir = new File(dir, "csv").getCanonicalPath
+
+      val data = Seq(("test \"quote\"", 123, "it \"works\"!", "\"very\" well"))
+      val df = spark.createDataFrame(data)
+
+      // escapeQuotes should be true by default
+      df.coalesce(1).write
+        .format("csv")
+        .option("quote", "\"")
+        .option("escape", "\"")
+        .option("quoteAll", "true")
+        .save(csvDir)
+
+      val results = spark.read
+        .format("text")
+        .load(csvDir)
+        .collect()
+
+      val expected = "\"test \"\"quote\"\"\",\"123\",\"it \"\"works\"\"!\",\"\"\"very\"\" well\""
+
+      assert(results.toSeq.map(_.toSeq) === Seq(Seq(expected)))
+    }
+  }
+
   test("save csv with quote escaping enabled") {
     withTempDir { dir =>
       val csvDir = new File(dir, "csv").getCanonicalPath