[SPARK-32896][SS][FOLLOW-UP] Rename the API to toTable
### What changes were proposed in this pull request? As the discussion in https://github.com/apache/spark/pull/30521#discussion_r531463427, rename the API to `toTable`. ### Why are the changes needed? Rename the API for further extension and accuracy. ### Does this PR introduce _any_ user-facing change? Yes, it's an API change but the new API is not released yet. ### How was this patch tested? Existing UT. Closes #30571 from xuanyuanking/SPARK-32896-follow. Authored-by: Yuanjian Li <yuanjian.li@databricks.com> Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
This commit is contained in:
parent
90d4d7d43f
commit
878cc0e6e9
|
@ -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)
|
||||
|
|
|
@ -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: _*)
|
||||
|
||||
|
|
Loading…
Reference in a new issue