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 78f30f413987651c76dbdc71ebaba0c84bf1689d..3349b8421b3e802fb4f5867b91b1e9b2fb4d28d8 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
@@ -206,7 +206,7 @@ final class DataFrameWriter private[sql](df: DataFrame) {
   }
 
   /**
-   * Specifies the name of the [[ContinuousQuery]] that can be started with `stream()`.
+   * Specifies the name of the [[ContinuousQuery]] that can be started with `startStream()`.
    * This name must be unique among all the currently active queries in the associated SQLContext.
    *
    * @since 2.0.0
@@ -223,8 +223,8 @@ final class DataFrameWriter private[sql](df: DataFrame) {
    *
    * @since 2.0.0
    */
-  def stream(path: String): ContinuousQuery = {
-    option("path", path).stream()
+  def startStream(path: String): ContinuousQuery = {
+    option("path", path).startStream()
   }
 
   /**
@@ -234,7 +234,7 @@ final class DataFrameWriter private[sql](df: DataFrame) {
    *
    * @since 2.0.0
    */
-  def stream(): ContinuousQuery = {
+  def startStream(): ContinuousQuery = {
     val dataSource =
       DataSource(
         df.sqlContext,
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/DataFrameReaderWriterSuite.scala
index f060c6f623b6e14e94c4e24da666d494b96ba793..0878277811e1283af3d060fb73d55c54d80f0ce1 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/DataFrameReaderWriterSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/DataFrameReaderWriterSuite.scala
@@ -72,7 +72,7 @@ class DataFrameReaderWriterSuite extends StreamTest with SharedSQLContext with B
       .stream()
       .write
       .format("org.apache.spark.sql.streaming.test")
-      .stream()
+      .startStream()
       .stop()
   }
 
@@ -82,7 +82,7 @@ class DataFrameReaderWriterSuite extends StreamTest with SharedSQLContext with B
       .stream()
       .write
       .format("org.apache.spark.sql.streaming.test")
-      .stream()
+      .startStream()
       .stop()
   }
 
@@ -108,7 +108,7 @@ class DataFrameReaderWriterSuite extends StreamTest with SharedSQLContext with B
       .option("opt1", "1")
       .options(Map("opt2" -> "2"))
       .options(map)
-      .stream()
+      .startStream()
       .stop()
 
     assert(LastOptions.parameters("opt1") == "1")
@@ -123,14 +123,14 @@ class DataFrameReaderWriterSuite extends StreamTest with SharedSQLContext with B
 
     df.write
       .format("org.apache.spark.sql.streaming.test")
-      .stream()
+      .startStream()
       .stop()
     assert(LastOptions.partitionColumns == Nil)
 
     df.write
       .format("org.apache.spark.sql.streaming.test")
       .partitionBy("a")
-      .stream()
+      .startStream()
       .stop()
     assert(LastOptions.partitionColumns == Seq("a"))
 
@@ -138,7 +138,7 @@ class DataFrameReaderWriterSuite extends StreamTest with SharedSQLContext with B
       df.write
         .format("org.apache.spark.sql.streaming.test")
         .partitionBy("A")
-        .stream()
+        .startStream()
         .stop()
       assert(LastOptions.partitionColumns == Seq("a"))
     }
@@ -147,7 +147,7 @@ class DataFrameReaderWriterSuite extends StreamTest with SharedSQLContext with B
       df.write
         .format("org.apache.spark.sql.streaming.test")
         .partitionBy("b")
-        .stream()
+        .startStream()
         .stop()
     }
   }
@@ -163,7 +163,7 @@ class DataFrameReaderWriterSuite extends StreamTest with SharedSQLContext with B
 
     df.write
       .format("org.apache.spark.sql.streaming.test")
-      .stream("/test")
+      .startStream("/test")
       .stop()
 
     assert(LastOptions.parameters("path") == "/test")
@@ -187,7 +187,7 @@ class DataFrameReaderWriterSuite extends StreamTest with SharedSQLContext with B
       .option("intOpt", 56)
       .option("boolOpt", false)
       .option("doubleOpt", 6.7)
-      .stream("/test")
+      .startStream("/test")
       .stop()
 
     assert(LastOptions.parameters("intOpt") == "56")
@@ -205,7 +205,7 @@ class DataFrameReaderWriterSuite extends StreamTest with SharedSQLContext with B
         .write
         .format("org.apache.spark.sql.streaming.test")
         .queryName(name)
-        .stream()
+        .startStream()
     }
 
     /** Start a query without specifying a name */
@@ -215,7 +215,7 @@ class DataFrameReaderWriterSuite extends StreamTest with SharedSQLContext with B
         .stream("/test")
         .write
         .format("org.apache.spark.sql.streaming.test")
-        .stream()
+        .startStream()
     }
 
     /** Get the names of active streams */