diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala index d67e175c24dd..9e3599712fde 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala @@ -304,7 +304,7 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { * @since 3.1.0 */ @throws[TimeoutException] - def saveAsTable(tableName: String): StreamingQuery = { + def toTable(tableName: String): StreamingQuery = { this.source = SOURCE_NAME_TABLE this.tableName = tableName startInternal(None) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala index 062b1060bc60..bf850432d5c0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala @@ -291,7 +291,7 @@ class DataStreamTableAPISuite extends StreamTest with BeforeAndAfter { val query = inputDF .writeStream .option("checkpointLocation", checkpointDir.getAbsolutePath) - .saveAsTable(tableIdentifier) + .toTable(tableIdentifier) inputData.addData(newInputs: _*)