You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2022/11/25 13:46:07 UTC

[GitHub] [spark] grundprinzip opened a new pull request, #38801: [SPARK-40539] [CONNECT] [PYTHON] Add basic support for DataFrameWriter

grundprinzip opened a new pull request, #38801:
URL: https://github.com/apache/spark/pull/38801

   ### What changes were proposed in this pull request?
   Based on the server side implementation, this patch adds the client side implementation for the DataFrameWriter.
   
   ### Why are the changes needed?
   Compatibility
   
   ### Does this PR introduce _any_ user-facing change?
   No
   
   ### How was this patch tested?
   UT


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] bjornjorgensen commented on a diff in pull request #38801: [SPARK-40539] [CONNECT] [PYTHON] Add basic support for DataFrameWriter

Posted by GitBox <gi...@apache.org>.
bjornjorgensen commented on code in PR #38801:
URL: https://github.com/apache/spark/pull/38801#discussion_r1032666193


##########
python/pyspark/sql/connect/writer.py:
##########
@@ -0,0 +1,922 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+
+from typing import Optional, Union, List, overload, Tuple, cast, Any
+from typing import TYPE_CHECKING
+
+from pyspark.sql.connect.plan import WriteOperation, LogicalPlan
+from pyspark.sql.types import StructType
+from pyspark.sql.utils import to_str
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect._typing import OptionalPrimitiveType
+    from pyspark.sql.connect.client import RemoteSparkSession
+
+
+PathOrPaths = Union[str, List[str]]
+TupleOrListOfString = Union[List[str], Tuple[str, ...]]
+
+
+class OptionUtils:
+    def _set_opts(
+        self,
+        schema: Optional[Union[StructType, str]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """
+        Set named options (filter out those the value is None)
+        """
+        if schema is not None:
+            self.schema(schema)  # type: ignore[attr-defined]
+        for k, v in options.items():
+            if v is not None:
+                self.option(k, v)  # type: ignore[attr-defined]
+
+
+class DataFrameWriter(OptionUtils):
+    """
+    Interface used to write a :class:`DataFrame` to external storage systems
+    (e.g. file systems, key-value stores, etc). Use :attr:`DataFrame.write`
+    to access this.
+
+    .. versionadded:: 3.4.0
+    """
+
+    def __init__(self, plan: "LogicalPlan", session: "RemoteSparkSession"):
+        self._df: "LogicalPlan" = plan
+        self._spark: "RemoteSparkSession" = session
+        self._write: "WriteOperation" = WriteOperation(self._df)
+
+    def mode(self, saveMode: Optional[str]) -> "DataFrameWriter":
+        """Specifies the behavior when data or table already exists.
+
+        Options include:
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Examples
+        --------
+        Raise an error when writing to an existing path.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     spark.createDataFrame(
+        ...         [{"age": 80, "name": "Xinrong Meng"}]
+        ...     ).write.mode("error").format("parquet").save(d)
+        Traceback (most recent call last):
+            ...
+        pyspark.sql.utils.AnalysisException: ...
+
+        Write a Parquet file back with various options, and read it back.
+
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Overwrite the path with a new Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 120, "name": "Takuya Ueshin"}]
+        ...     ).write.mode("append").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 140, "name": "Haejoon Lee"}]
+        ...     ).write.mode("ignore").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |120|Takuya Ueshin|
+        |100| Hyukjin Kwon|
+        +---+-------------+
+        """
+        # At the JVM side, the default value of mode is already set to "error".
+        # So, if the given saveMode is None, we will not call JVM-side's mode method.
+        if saveMode is not None:
+            self._write.mode = saveMode
+        return self
+
+    def format(self, source: str) -> "DataFrameWriter":
+        """Specifies the underlying output data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        source : str
+            string, name of the data source, e.g. 'json', 'parquet'.
+
+        Examples
+        --------
+        >>> spark.range(1).write.format('parquet')
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Write a DataFrame into a Parquet file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.format('parquet').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.source = source
+        return self
+
+    def option(self, key: str, value: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds a output option for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        key : str
+            The key for the option to set.
+        value
+            The value for the option to set.
+
+        Examples
+        --------
+        >>> spark.range(1).write.option("key", "value")
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Specify the option 'nullValue' with writing a CSV file.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file with 'nullValue' option set to 'Hyukjin Kwon'.
+        ...     df = spark.createDataFrame([(100, None)], "age INT, name STRING")
+        ...     df.write.option("nullValue", "Hyukjin Kwon").mode("overwrite").format("csv").save(d)
+        ...
+        ...     # Read the CSV file as a DataFrame.
+        ...     spark.read.schema(df.schema).format('csv').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.options[key] = to_str(value)
+        return self
+
+    def options(self, **options: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds output options for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        **options : dict
+            The dictionary of string keys and prmitive-type values.
+
+        Examples
+        --------
+        >>> spark.range(1).write.option("key", "value")
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Specify the option 'nullValue' and 'header' with writing a CSV file.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file with 'nullValue' option set to 'Hyukjin Kwon',
+        ...     # and 'header' option set to `True`.
+        ...     df = spark.createDataFrame([(100, "Hyukjin Kwon")], ["age", "name"])
+        ...     df.write.options(nullValue="Hyukjin Kwon", header=True).mode(
+        ...         "overwrite").format("csv").save(d)
+        ...
+        ...     # Read the CSV file as a DataFrame.
+        ...     spark.read.option("header", True).format('csv').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        for k in options:
+            self._write.options[k] = to_str(options[k])
+        return self
+
+    @overload
+    def partitionBy(self, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def partitionBy(self, *cols: List[str]) -> "DataFrameWriter":
+        ...
+
+    def partitionBy(self, *cols: Union[str, List[str]]) -> "DataFrameWriter":
+        """Partitions the output by the given columns on the file system.
+
+        If specified, the output is laid out on the file system similar
+        to Hive's partitioning scheme.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        cols : str or list
+            name of columns
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a partitioned manner, and read it back.
+
+        >>> import tempfile
+        >>> import os
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file in a partitioned manner.
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}, {"age": 120, "name": "Ruifeng Zheng"}]
+        ...     ).write.partitionBy("name").mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).sort("age").show()
+        ...
+        ...     # Read one partition as a DataFrame.
+        ...     spark.read.parquet(f"{d}{os.path.sep}name=Hyukjin Kwon").show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |100| Hyukjin Kwon|
+        |120|Ruifeng Zheng|
+        +---+-------------+
+        +---+
+        |age|
+        +---+
+        |100|
+        +---+
+        """
+        if len(cols) == 1 and isinstance(cols[0], (list, tuple)):
+            cols = cols[0]  # type: ignore[assignment]
+
+        self._write.partitioning_cols = cast(List[str], cols)
+        return self
+
+    @overload
+    def bucketBy(self, numBuckets: int, col: str, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def bucketBy(self, numBuckets: int, col: TupleOrListOfString) -> "DataFrameWriter":
+        ...
+
+    def bucketBy(
+        self, numBuckets: int, col: Union[str, TupleOrListOfString], *cols: Optional[str]
+    ) -> "DataFrameWriter":
+        """Buckets the output by the given columns. If specified,
+        the output is laid out on the file system similar to Hive's bucketing scheme,
+        but with a different bucket hash function and is not compatible with Hive's bucketing.
+
+        .. versionadded:: 2.3.0
+
+        Parameters
+        ----------
+        numBuckets : int
+            the number of buckets to save
+        col : str, list or tuple
+            a name of a column, or a list of names.
+        cols : str
+            additional names (optional). If `col` is a list it should be empty.
+
+        Notes
+        -----
+        Applicable for file-based data sources in combination with
+        :py:meth:`DataFrameWriter.saveAsTable`.
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a buckted manner, and read it back.
+
+        >>> from pyspark.sql.functions import input_file_name
+        >>> # Write a DataFrame into a Parquet file in a bucketed manner.
+        ... _ = spark.sql("DROP TABLE IF EXISTS bucketed_table")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.bucketBy(2, "name").mode("overwrite").saveAsTable("bucketed_table")
+        >>> # Read the Parquet file as a DataFrame.
+        ... spark.read.table("bucketed_table").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE bucketed_table")
+        """
+        if not isinstance(numBuckets, int):
+            raise TypeError("numBuckets should be an int, got {0}.".format(type(numBuckets)))
+
+        if isinstance(col, (list, tuple)):
+            if cols:
+                raise ValueError("col is a {0} but cols are not empty".format(type(col)))
+
+            col, cols = col[0], col[1:]  # type: ignore[assignment]
+
+        if not all(isinstance(c, str) for c in cols) or not (isinstance(col, str)):
+            raise TypeError("all names should be `str`")
+
+        self._write.num_buckets = numBuckets
+        self._write.bucket_cols = cast(List[str], cols)
+        return self
+
+    @overload
+    def sortBy(self, col: str, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def sortBy(self, col: TupleOrListOfString) -> "DataFrameWriter":
+        ...
+
+    def sortBy(
+        self, col: Union[str, TupleOrListOfString], *cols: Optional[str]
+    ) -> "DataFrameWriter":
+        """Sorts the output in each bucket by the given columns on the file system.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        col : str, tuple or list
+            a name of a column, or a list of names.
+        cols : str
+            additional names (optional). If `col` is a list it should be empty.
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a sorted-buckted manner, and read it back.
+
+        >>> from pyspark.sql.functions import input_file_name
+        >>> # Write a DataFrame into a Parquet file in a sorted-bucketed manner.
+        ... _ = spark.sql("DROP TABLE IF EXISTS sorted_bucketed_table")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.bucketBy(1, "name").sortBy("age").mode(
+        ...     "overwrite").saveAsTable("sorted_bucketed_table")
+        >>> # Read the Parquet file as a DataFrame.
+        ... spark.read.table("sorted_bucketed_table").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE sorted_bucketed_table")
+        """
+        if isinstance(col, (list, tuple)):
+            if cols:
+                raise ValueError("col is a {0} but cols are not empty".format(type(col)))
+
+            col, cols = col[0], col[1:]  # type: ignore[assignment]
+
+        if not all(isinstance(c, str) for c in cols) or not (isinstance(col, str)):
+            raise TypeError("all names should be `str`")
+
+        self._write.sort_cols = cast(List[str], cols)
+        return self
+
+    def save(
+        self,
+        path: Optional[str] = None,
+        format: Optional[str] = None,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """Saves the contents of the :class:`DataFrame` to a data source.
+
+        The data source is specified by the ``format`` and a set of ``options``.
+        If ``format`` is not specified, the default data source configured by
+        ``spark.sql.sources.default`` will be used.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str, optional
+            the path in a Hadoop supported file system
+        format : str, optional
+            the format used to save
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+        partitionBy : list, optional
+            names of partitioning columns
+        **options : dict
+            all other string options
+
+        Examples
+        --------
+        Write a DataFrame into a JSON file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a JSON file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("json").save(d)
+        ...
+        ...     # Read the JSON file as a DataFrame.
+        ...     spark.read.format('json').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode).options(**options)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        if format is not None:
+            self.format(format)
+        self._write.path = path
+        self._spark.execute_command(self._write.command(self._spark))
+
+    def insertInto(self, tableName: str, overwrite: Optional[bool] = None) -> None:
+        """Inserts the content of the :class:`DataFrame` to the specified table.
+
+        It requires that the schema of the :class:`DataFrame` is the same as the
+        schema of the table.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        overwrite : bool, optional
+            If true, overwrites existing data. Disabled by default
+
+        Notes
+        -----
+        Unlike :meth:`DataFrameWriter.saveAsTable`, :meth:`DataFrameWriter.insertInto` ignores
+        the column names and just uses position-based resolution.
+
+        Examples
+        --------
+        >>> _ = spark.sql("DROP TABLE IF EXISTS tblA")
+        >>> df = spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... )
+        >>> df.write.saveAsTable("tblA")
+
+        Insert the data into 'tblA' table but with different column names.
+
+        >>> df.selectExpr("age AS col1", "name AS col2").write.insertInto("tblA")
+        >>> spark.read.table("tblA").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE tblA")
+        """
+        if overwrite is not None:
+            self.mode("overwrite" if overwrite else "append")
+        self.saveAsTable(tableName)
+
+    def saveAsTable(
+        self,
+        name: str,
+        format: Optional[str] = None,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` as the specified table.
+
+        In the case the table already exists, behavior of this function depends on the
+        save mode, specified by the `mode` function (default to throwing an exception).
+        When `mode` is `Overwrite`, the schema of the :class:`DataFrame` does not need to be
+        the same as that of the existing table.
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Notes
+        -----
+        When `mode` is `Append`, if there is an existing table, we will use the format and
+        options of the existing table. The column order in the schema of the :class:`DataFrame`
+        doesn't need to be same as that of the existing table. Unlike

Review Comment:
   doesn't need to be the same as  
   Add the before same



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] bjornjorgensen commented on a diff in pull request #38801: [SPARK-40539] [CONNECT] [PYTHON] Add basic support for DataFrameWriter

Posted by GitBox <gi...@apache.org>.
bjornjorgensen commented on code in PR #38801:
URL: https://github.com/apache/spark/pull/38801#discussion_r1032674002


##########
python/pyspark/sql/connect/writer.py:
##########
@@ -0,0 +1,922 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+
+from typing import Optional, Union, List, overload, Tuple, cast, Any
+from typing import TYPE_CHECKING
+
+from pyspark.sql.connect.plan import WriteOperation, LogicalPlan
+from pyspark.sql.types import StructType
+from pyspark.sql.utils import to_str
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect._typing import OptionalPrimitiveType
+    from pyspark.sql.connect.client import RemoteSparkSession
+
+
+PathOrPaths = Union[str, List[str]]
+TupleOrListOfString = Union[List[str], Tuple[str, ...]]
+
+
+class OptionUtils:
+    def _set_opts(
+        self,
+        schema: Optional[Union[StructType, str]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """
+        Set named options (filter out those the value is None)
+        """
+        if schema is not None:
+            self.schema(schema)  # type: ignore[attr-defined]
+        for k, v in options.items():
+            if v is not None:
+                self.option(k, v)  # type: ignore[attr-defined]
+
+
+class DataFrameWriter(OptionUtils):
+    """
+    Interface used to write a :class:`DataFrame` to external storage systems
+    (e.g. file systems, key-value stores, etc). Use :attr:`DataFrame.write`
+    to access this.
+
+    .. versionadded:: 3.4.0
+    """
+
+    def __init__(self, plan: "LogicalPlan", session: "RemoteSparkSession"):
+        self._df: "LogicalPlan" = plan
+        self._spark: "RemoteSparkSession" = session
+        self._write: "WriteOperation" = WriteOperation(self._df)
+
+    def mode(self, saveMode: Optional[str]) -> "DataFrameWriter":
+        """Specifies the behavior when data or table already exists.
+
+        Options include:
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Examples
+        --------
+        Raise an error when writing to an existing path.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     spark.createDataFrame(
+        ...         [{"age": 80, "name": "Xinrong Meng"}]
+        ...     ).write.mode("error").format("parquet").save(d)
+        Traceback (most recent call last):
+            ...
+        pyspark.sql.utils.AnalysisException: ...
+
+        Write a Parquet file back with various options, and read it back.
+
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Overwrite the path with a new Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 120, "name": "Takuya Ueshin"}]
+        ...     ).write.mode("append").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 140, "name": "Haejoon Lee"}]
+        ...     ).write.mode("ignore").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |120|Takuya Ueshin|
+        |100| Hyukjin Kwon|
+        +---+-------------+
+        """
+        # At the JVM side, the default value of mode is already set to "error".
+        # So, if the given saveMode is None, we will not call JVM-side's mode method.
+        if saveMode is not None:
+            self._write.mode = saveMode
+        return self
+
+    def format(self, source: str) -> "DataFrameWriter":
+        """Specifies the underlying output data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        source : str
+            string, name of the data source, e.g. 'json', 'parquet'.
+
+        Examples
+        --------
+        >>> spark.range(1).write.format('parquet')
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Write a DataFrame into a Parquet file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.format('parquet').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.source = source
+        return self
+
+    def option(self, key: str, value: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds a output option for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        key : str
+            The key for the option to set.
+        value
+            The value for the option to set.
+
+        Examples
+        --------
+        >>> spark.range(1).write.option("key", "value")
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Specify the option 'nullValue' with writing a CSV file.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file with 'nullValue' option set to 'Hyukjin Kwon'.
+        ...     df = spark.createDataFrame([(100, None)], "age INT, name STRING")
+        ...     df.write.option("nullValue", "Hyukjin Kwon").mode("overwrite").format("csv").save(d)
+        ...
+        ...     # Read the CSV file as a DataFrame.
+        ...     spark.read.schema(df.schema).format('csv').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.options[key] = to_str(value)
+        return self
+
+    def options(self, **options: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds output options for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        **options : dict
+            The dictionary of string keys and prmitive-type values.
+
+        Examples
+        --------
+        >>> spark.range(1).write.option("key", "value")
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Specify the option 'nullValue' and 'header' with writing a CSV file.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file with 'nullValue' option set to 'Hyukjin Kwon',
+        ...     # and 'header' option set to `True`.
+        ...     df = spark.createDataFrame([(100, "Hyukjin Kwon")], ["age", "name"])

Review Comment:
   "Hyukjin Kwon" should be None here?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] hvanhovell commented on a diff in pull request #38801: [SPARK-40539] [CONNECT] [PYTHON] Add basic support for DataFrameWriter

Posted by GitBox <gi...@apache.org>.
hvanhovell commented on code in PR #38801:
URL: https://github.com/apache/spark/pull/38801#discussion_r1032699394


##########
python/pyspark/sql/connect/writer.py:
##########
@@ -0,0 +1,922 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+
+from typing import Optional, Union, List, overload, Tuple, cast, Any
+from typing import TYPE_CHECKING
+
+from pyspark.sql.connect.plan import WriteOperation, LogicalPlan
+from pyspark.sql.types import StructType
+from pyspark.sql.utils import to_str
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect._typing import OptionalPrimitiveType
+    from pyspark.sql.connect.client import RemoteSparkSession
+
+
+PathOrPaths = Union[str, List[str]]
+TupleOrListOfString = Union[List[str], Tuple[str, ...]]
+
+
+class OptionUtils:
+    def _set_opts(
+        self,
+        schema: Optional[Union[StructType, str]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """
+        Set named options (filter out those the value is None)
+        """
+        if schema is not None:
+            self.schema(schema)  # type: ignore[attr-defined]
+        for k, v in options.items():
+            if v is not None:
+                self.option(k, v)  # type: ignore[attr-defined]
+
+
+class DataFrameWriter(OptionUtils):
+    """
+    Interface used to write a :class:`DataFrame` to external storage systems
+    (e.g. file systems, key-value stores, etc). Use :attr:`DataFrame.write`
+    to access this.
+
+    .. versionadded:: 3.4.0
+    """
+
+    def __init__(self, plan: "LogicalPlan", session: "RemoteSparkSession"):
+        self._df: "LogicalPlan" = plan
+        self._spark: "RemoteSparkSession" = session
+        self._write: "WriteOperation" = WriteOperation(self._df)
+
+    def mode(self, saveMode: Optional[str]) -> "DataFrameWriter":
+        """Specifies the behavior when data or table already exists.
+
+        Options include:
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Examples
+        --------
+        Raise an error when writing to an existing path.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     spark.createDataFrame(
+        ...         [{"age": 80, "name": "Xinrong Meng"}]
+        ...     ).write.mode("error").format("parquet").save(d)
+        Traceback (most recent call last):
+            ...
+        pyspark.sql.utils.AnalysisException: ...
+
+        Write a Parquet file back with various options, and read it back.
+
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Overwrite the path with a new Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 120, "name": "Takuya Ueshin"}]
+        ...     ).write.mode("append").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 140, "name": "Haejoon Lee"}]
+        ...     ).write.mode("ignore").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |120|Takuya Ueshin|
+        |100| Hyukjin Kwon|
+        +---+-------------+
+        """
+        # At the JVM side, the default value of mode is already set to "error".
+        # So, if the given saveMode is None, we will not call JVM-side's mode method.
+        if saveMode is not None:
+            self._write.mode = saveMode
+        return self
+
+    def format(self, source: str) -> "DataFrameWriter":
+        """Specifies the underlying output data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        source : str
+            string, name of the data source, e.g. 'json', 'parquet'.
+
+        Examples
+        --------
+        >>> spark.range(1).write.format('parquet')
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Write a DataFrame into a Parquet file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.format('parquet').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.source = source
+        return self
+
+    def option(self, key: str, value: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds a output option for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        key : str
+            The key for the option to set.
+        value
+            The value for the option to set.
+
+        Examples
+        --------
+        >>> spark.range(1).write.option("key", "value")
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Specify the option 'nullValue' with writing a CSV file.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file with 'nullValue' option set to 'Hyukjin Kwon'.
+        ...     df = spark.createDataFrame([(100, None)], "age INT, name STRING")
+        ...     df.write.option("nullValue", "Hyukjin Kwon").mode("overwrite").format("csv").save(d)
+        ...
+        ...     # Read the CSV file as a DataFrame.
+        ...     spark.read.schema(df.schema).format('csv').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.options[key] = to_str(value)
+        return self
+
+    def options(self, **options: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds output options for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        **options : dict
+            The dictionary of string keys and prmitive-type values.
+
+        Examples
+        --------
+        >>> spark.range(1).write.option("key", "value")
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Specify the option 'nullValue' and 'header' with writing a CSV file.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file with 'nullValue' option set to 'Hyukjin Kwon',
+        ...     # and 'header' option set to `True`.
+        ...     df = spark.createDataFrame([(100, "Hyukjin Kwon")], ["age", "name"])
+        ...     df.write.options(nullValue="Hyukjin Kwon", header=True).mode(
+        ...         "overwrite").format("csv").save(d)
+        ...
+        ...     # Read the CSV file as a DataFrame.
+        ...     spark.read.option("header", True).format('csv').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        for k in options:
+            self._write.options[k] = to_str(options[k])
+        return self
+
+    @overload
+    def partitionBy(self, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def partitionBy(self, *cols: List[str]) -> "DataFrameWriter":
+        ...
+
+    def partitionBy(self, *cols: Union[str, List[str]]) -> "DataFrameWriter":
+        """Partitions the output by the given columns on the file system.
+
+        If specified, the output is laid out on the file system similar
+        to Hive's partitioning scheme.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        cols : str or list
+            name of columns
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a partitioned manner, and read it back.
+
+        >>> import tempfile
+        >>> import os
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file in a partitioned manner.
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}, {"age": 120, "name": "Ruifeng Zheng"}]
+        ...     ).write.partitionBy("name").mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).sort("age").show()
+        ...
+        ...     # Read one partition as a DataFrame.
+        ...     spark.read.parquet(f"{d}{os.path.sep}name=Hyukjin Kwon").show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |100| Hyukjin Kwon|
+        |120|Ruifeng Zheng|
+        +---+-------------+
+        +---+
+        |age|
+        +---+
+        |100|
+        +---+
+        """
+        if len(cols) == 1 and isinstance(cols[0], (list, tuple)):
+            cols = cols[0]  # type: ignore[assignment]
+
+        self._write.partitioning_cols = cast(List[str], cols)
+        return self
+
+    @overload
+    def bucketBy(self, numBuckets: int, col: str, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def bucketBy(self, numBuckets: int, col: TupleOrListOfString) -> "DataFrameWriter":
+        ...
+
+    def bucketBy(
+        self, numBuckets: int, col: Union[str, TupleOrListOfString], *cols: Optional[str]
+    ) -> "DataFrameWriter":
+        """Buckets the output by the given columns. If specified,
+        the output is laid out on the file system similar to Hive's bucketing scheme,
+        but with a different bucket hash function and is not compatible with Hive's bucketing.
+
+        .. versionadded:: 2.3.0
+
+        Parameters
+        ----------
+        numBuckets : int
+            the number of buckets to save
+        col : str, list or tuple
+            a name of a column, or a list of names.
+        cols : str
+            additional names (optional). If `col` is a list it should be empty.
+
+        Notes
+        -----
+        Applicable for file-based data sources in combination with
+        :py:meth:`DataFrameWriter.saveAsTable`.
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a buckted manner, and read it back.
+
+        >>> from pyspark.sql.functions import input_file_name
+        >>> # Write a DataFrame into a Parquet file in a bucketed manner.
+        ... _ = spark.sql("DROP TABLE IF EXISTS bucketed_table")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.bucketBy(2, "name").mode("overwrite").saveAsTable("bucketed_table")
+        >>> # Read the Parquet file as a DataFrame.
+        ... spark.read.table("bucketed_table").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE bucketed_table")
+        """
+        if not isinstance(numBuckets, int):
+            raise TypeError("numBuckets should be an int, got {0}.".format(type(numBuckets)))
+
+        if isinstance(col, (list, tuple)):
+            if cols:
+                raise ValueError("col is a {0} but cols are not empty".format(type(col)))
+
+            col, cols = col[0], col[1:]  # type: ignore[assignment]
+
+        if not all(isinstance(c, str) for c in cols) or not (isinstance(col, str)):
+            raise TypeError("all names should be `str`")
+
+        self._write.num_buckets = numBuckets
+        self._write.bucket_cols = cast(List[str], cols)
+        return self
+
+    @overload
+    def sortBy(self, col: str, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def sortBy(self, col: TupleOrListOfString) -> "DataFrameWriter":
+        ...
+
+    def sortBy(
+        self, col: Union[str, TupleOrListOfString], *cols: Optional[str]
+    ) -> "DataFrameWriter":
+        """Sorts the output in each bucket by the given columns on the file system.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        col : str, tuple or list
+            a name of a column, or a list of names.
+        cols : str
+            additional names (optional). If `col` is a list it should be empty.
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a sorted-buckted manner, and read it back.
+
+        >>> from pyspark.sql.functions import input_file_name
+        >>> # Write a DataFrame into a Parquet file in a sorted-bucketed manner.
+        ... _ = spark.sql("DROP TABLE IF EXISTS sorted_bucketed_table")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.bucketBy(1, "name").sortBy("age").mode(
+        ...     "overwrite").saveAsTable("sorted_bucketed_table")
+        >>> # Read the Parquet file as a DataFrame.
+        ... spark.read.table("sorted_bucketed_table").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE sorted_bucketed_table")
+        """
+        if isinstance(col, (list, tuple)):
+            if cols:
+                raise ValueError("col is a {0} but cols are not empty".format(type(col)))
+
+            col, cols = col[0], col[1:]  # type: ignore[assignment]
+
+        if not all(isinstance(c, str) for c in cols) or not (isinstance(col, str)):
+            raise TypeError("all names should be `str`")
+
+        self._write.sort_cols = cast(List[str], cols)
+        return self
+
+    def save(
+        self,
+        path: Optional[str] = None,
+        format: Optional[str] = None,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """Saves the contents of the :class:`DataFrame` to a data source.
+
+        The data source is specified by the ``format`` and a set of ``options``.
+        If ``format`` is not specified, the default data source configured by
+        ``spark.sql.sources.default`` will be used.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str, optional
+            the path in a Hadoop supported file system
+        format : str, optional
+            the format used to save
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+        partitionBy : list, optional
+            names of partitioning columns
+        **options : dict
+            all other string options
+
+        Examples
+        --------
+        Write a DataFrame into a JSON file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a JSON file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("json").save(d)
+        ...
+        ...     # Read the JSON file as a DataFrame.
+        ...     spark.read.format('json').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode).options(**options)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        if format is not None:
+            self.format(format)
+        self._write.path = path
+        self._spark.execute_command(self._write.command(self._spark))
+
+    def insertInto(self, tableName: str, overwrite: Optional[bool] = None) -> None:
+        """Inserts the content of the :class:`DataFrame` to the specified table.
+
+        It requires that the schema of the :class:`DataFrame` is the same as the
+        schema of the table.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        overwrite : bool, optional
+            If true, overwrites existing data. Disabled by default
+
+        Notes
+        -----
+        Unlike :meth:`DataFrameWriter.saveAsTable`, :meth:`DataFrameWriter.insertInto` ignores
+        the column names and just uses position-based resolution.
+
+        Examples
+        --------
+        >>> _ = spark.sql("DROP TABLE IF EXISTS tblA")
+        >>> df = spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... )
+        >>> df.write.saveAsTable("tblA")
+
+        Insert the data into 'tblA' table but with different column names.
+
+        >>> df.selectExpr("age AS col1", "name AS col2").write.insertInto("tblA")
+        >>> spark.read.table("tblA").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE tblA")
+        """
+        if overwrite is not None:
+            self.mode("overwrite" if overwrite else "append")
+        self.saveAsTable(tableName)
+
+    def saveAsTable(
+        self,
+        name: str,
+        format: Optional[str] = None,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` as the specified table.
+
+        In the case the table already exists, behavior of this function depends on the
+        save mode, specified by the `mode` function (default to throwing an exception).
+        When `mode` is `Overwrite`, the schema of the :class:`DataFrame` does not need to be
+        the same as that of the existing table.
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Notes
+        -----
+        When `mode` is `Append`, if there is an existing table, we will use the format and
+        options of the existing table. The column order in the schema of the :class:`DataFrame`
+        doesn't need to be same as that of the existing table. Unlike
+        :meth:`DataFrameWriter.insertInto`, :meth:`DataFrameWriter.saveAsTable` will use the
+        column names to find the correct column positions.
+
+        Parameters
+        ----------
+        name : str
+            the table name
+        format : str, optional
+            the format used to save
+        mode : str, optional
+            one of `append`, `overwrite`, `error`, `errorifexists`, `ignore` \
+            (default: error)
+        partitionBy : str or list
+            names of partitioning columns
+        **options : dict
+            all other string options
+
+        Examples
+        --------
+        Creates a table from a DataFrame, and read it back.
+
+        >>> _ = spark.sql("DROP TABLE IF EXISTS tblA")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.saveAsTable("tblA")
+        >>> spark.read.table("tblA").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE tblA")
+        """
+        self.mode(mode).options(**options)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        if format is not None:
+            self.format(format)
+        self._write.table_name = name
+        self._spark.execute_command(self._write.command(self._spark))
+
+    def json(
+        self,
+        path: str,
+        mode: Optional[str] = None,
+        compression: Optional[str] = None,
+        dateFormat: Optional[str] = None,
+        timestampFormat: Optional[str] = None,
+        lineSep: Optional[str] = None,
+        encoding: Optional[str] = None,
+        ignoreNullFields: Optional[Union[bool, str]] = None,
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` in JSON format
+        (`JSON Lines text format or newline-delimited JSON <http://jsonlines.org/>`_) at the
+        specified path.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-json.html#data-source-option>`_
+            in the version you use.
+
+            .. # noqa
+
+        Examples
+        --------
+        Write a DataFrame into a JSON file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a JSON file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.json(d, mode="overwrite")
+        ...
+        ...     # Read the JSON file as a DataFrame.
+        ...     spark.read.format("json").load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode)
+        self._set_opts(
+            compression=compression,
+            dateFormat=dateFormat,
+            timestampFormat=timestampFormat,
+            lineSep=lineSep,
+            encoding=encoding,
+            ignoreNullFields=ignoreNullFields,
+        )
+        self.format("json").save(path)
+
+    def parquet(
+        self,
+        path: str,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        compression: Optional[str] = None,
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` in Parquet format at the specified path.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+        partitionBy : str or list, optional
+            names of partitioning columns
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#data-source-option>`_
+            in the version you use.
+
+            .. # noqa
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.parquet(d, mode="overwrite")
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.format("parquet").load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        self.option("compression", compression)
+        self.format("parquet").save(path)
+
+    def text(
+        self, path: str, compression: Optional[str] = None, lineSep: Optional[str] = None
+    ) -> None:
+        """Saves the content of the DataFrame in a text file at the specified path.
+        The text files will be encoded as UTF-8.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-text.html#data-source-option>`_
+            in the version you use.
+
+            .. # noqa
+
+        Notes
+        -----
+        The DataFrame must have only one column that is of string type.
+        Each row becomes a new line in the output file.
+
+        Examples
+        --------
+        Write a DataFrame into a text file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a text file
+        ...     df = spark.createDataFrame([("a",), ("b",), ("c",)], schema=["alphabets"])
+        ...     df.write.mode("overwrite").text(d)
+        ...
+        ...     # Read the text file as a DataFrame.
+        ...     spark.read.schema(df.schema).format("text").load(d).sort("alphabets").show()
+        +---------+
+        |alphabets|
+        +---------+
+        |        a|
+        |        b|
+        |        c|
+        +---------+
+        """
+        self._set_opts(compression=compression, lineSep=lineSep)
+        self.format("text").save(path)
+
+    def csv(
+        self,
+        path: str,
+        mode: Optional[str] = None,
+        compression: Optional[str] = None,
+        sep: Optional[str] = None,
+        quote: Optional[str] = None,
+        escape: Optional[str] = None,
+        header: Optional[Union[bool, str]] = None,
+        nullValue: Optional[str] = None,
+        escapeQuotes: Optional[Union[bool, str]] = None,
+        quoteAll: Optional[Union[bool, str]] = None,
+        dateFormat: Optional[str] = None,
+        timestampFormat: Optional[str] = None,
+        ignoreLeadingWhiteSpace: Optional[Union[bool, str]] = None,
+        ignoreTrailingWhiteSpace: Optional[Union[bool, str]] = None,
+        charToEscapeQuoteEscaping: Optional[str] = None,
+        encoding: Optional[str] = None,
+        emptyValue: Optional[str] = None,
+        lineSep: Optional[str] = None,
+    ) -> None:
+        r"""Saves the content of the :class:`DataFrame` in CSV format at the specified path.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-csv.html#data-source-option>`_
+            in the version you use.
+
+            .. # noqa
+
+        Examples
+        --------
+        Write a DataFrame into a CSV file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file
+        ...     df = spark.createDataFrame([{"age": 100, "name": "Hyukjin Kwon"}])

Review Comment:
   None? Not sure I follow.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] amaliujia commented on pull request #38801: [SPARK-41317][CONNECT][PYTHON] Add basic support for DataFrameWriter

Posted by GitBox <gi...@apache.org>.
amaliujia commented on PR #38801:
URL: https://github.com/apache/spark/pull/38801#issuecomment-1331118141

   Yeah this PR is not blocked which is built on existing API. 
   
   Also given that we need to match PySpark API, future changes will only change underlying protocols and the public should be just this in this PR.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] grundprinzip commented on a diff in pull request #38801: [SPARK-40539] [CONNECT] [PYTHON] Add basic support for DataFrameWriter

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #38801:
URL: https://github.com/apache/spark/pull/38801#discussion_r1032566396


##########
python/pyspark/sql/connect/writer.py:
##########
@@ -0,0 +1,922 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+
+from typing import Optional, Union, List, overload, Tuple, cast, Any
+from typing import TYPE_CHECKING
+
+from pyspark.sql.connect.plan import WriteOperation, LogicalPlan
+from pyspark.sql.types import StructType
+from pyspark.sql.utils import to_str
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect._typing import OptionalPrimitiveType
+    from pyspark.sql.connect.client import RemoteSparkSession
+
+
+PathOrPaths = Union[str, List[str]]
+TupleOrListOfString = Union[List[str], Tuple[str, ...]]
+
+
+class OptionUtils:
+    def _set_opts(
+        self,
+        schema: Optional[Union[StructType, str]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """
+        Set named options (filter out those the value is None)
+        """
+        if schema is not None:
+            self.schema(schema)  # type: ignore[attr-defined]
+        for k, v in options.items():
+            if v is not None:
+                self.option(k, v)  # type: ignore[attr-defined]
+
+
+class DataFrameWriter(OptionUtils):
+    """
+    Interface used to write a :class:`DataFrame` to external storage systems
+    (e.g. file systems, key-value stores, etc). Use :attr:`DataFrame.write`
+    to access this.
+
+    .. versionadded:: 3.4.0
+    """
+
+    def __init__(self, plan: "LogicalPlan", session: "RemoteSparkSession"):
+        self._df: "LogicalPlan" = plan
+        self._spark: "RemoteSparkSession" = session
+        self._write: "WriteOperation" = WriteOperation(self._df)
+
+    def mode(self, saveMode: Optional[str]) -> "DataFrameWriter":
+        """Specifies the behavior when data or table already exists.
+
+        Options include:
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Examples
+        --------
+        Raise an error when writing to an existing path.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     spark.createDataFrame(
+        ...         [{"age": 80, "name": "Xinrong Meng"}]
+        ...     ).write.mode("error").format("parquet").save(d)
+        Traceback (most recent call last):
+            ...
+        pyspark.sql.utils.AnalysisException: ...
+
+        Write a Parquet file back with various options, and read it back.
+
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Overwrite the path with a new Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 120, "name": "Takuya Ueshin"}]
+        ...     ).write.mode("append").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 140, "name": "Haejoon Lee"}]
+        ...     ).write.mode("ignore").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |120|Takuya Ueshin|
+        |100| Hyukjin Kwon|
+        +---+-------------+
+        """
+        # At the JVM side, the default value of mode is already set to "error".
+        # So, if the given saveMode is None, we will not call JVM-side's mode method.
+        if saveMode is not None:
+            self._write.mode = saveMode
+        return self
+
+    def format(self, source: str) -> "DataFrameWriter":
+        """Specifies the underlying output data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        source : str
+            string, name of the data source, e.g. 'json', 'parquet'.
+
+        Examples
+        --------
+        >>> spark.range(1).write.format('parquet')
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Write a DataFrame into a Parquet file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.format('parquet').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.source = source
+        return self
+
+    def option(self, key: str, value: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds a output option for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        key : str
+            The key for the option to set.
+        value
+            The value for the option to set.
+
+        Examples
+        --------
+        >>> spark.range(1).write.option("key", "value")
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Specify the option 'nullValue' with writing a CSV file.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file with 'nullValue' option set to 'Hyukjin Kwon'.
+        ...     df = spark.createDataFrame([(100, None)], "age INT, name STRING")
+        ...     df.write.option("nullValue", "Hyukjin Kwon").mode("overwrite").format("csv").save(d)
+        ...
+        ...     # Read the CSV file as a DataFrame.
+        ...     spark.read.schema(df.schema).format('csv').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.options[key] = to_str(value)
+        return self
+
+    def options(self, **options: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds output options for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        **options : dict
+            The dictionary of string keys and prmitive-type values.
+
+        Examples
+        --------
+        >>> spark.range(1).write.option("key", "value")
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Specify the option 'nullValue' and 'header' with writing a CSV file.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file with 'nullValue' option set to 'Hyukjin Kwon',
+        ...     # and 'header' option set to `True`.
+        ...     df = spark.createDataFrame([(100, "Hyukjin Kwon")], ["age", "name"])
+        ...     df.write.options(nullValue="Hyukjin Kwon", header=True).mode(
+        ...         "overwrite").format("csv").save(d)
+        ...
+        ...     # Read the CSV file as a DataFrame.
+        ...     spark.read.option("header", True).format('csv').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        for k in options:
+            self._write.options[k] = to_str(options[k])
+        return self
+
+    @overload
+    def partitionBy(self, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def partitionBy(self, *cols: List[str]) -> "DataFrameWriter":
+        ...
+
+    def partitionBy(self, *cols: Union[str, List[str]]) -> "DataFrameWriter":
+        """Partitions the output by the given columns on the file system.
+
+        If specified, the output is laid out on the file system similar
+        to Hive's partitioning scheme.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        cols : str or list
+            name of columns
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a partitioned manner, and read it back.
+
+        >>> import tempfile
+        >>> import os
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file in a partitioned manner.
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}, {"age": 120, "name": "Ruifeng Zheng"}]
+        ...     ).write.partitionBy("name").mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).sort("age").show()
+        ...
+        ...     # Read one partition as a DataFrame.
+        ...     spark.read.parquet(f"{d}{os.path.sep}name=Hyukjin Kwon").show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |100| Hyukjin Kwon|
+        |120|Ruifeng Zheng|
+        +---+-------------+
+        +---+
+        |age|
+        +---+
+        |100|
+        +---+
+        """
+        if len(cols) == 1 and isinstance(cols[0], (list, tuple)):
+            cols = cols[0]  # type: ignore[assignment]
+
+        self._write.partitioning_cols = cast(List[str], cols)
+        return self
+
+    @overload
+    def bucketBy(self, numBuckets: int, col: str, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def bucketBy(self, numBuckets: int, col: TupleOrListOfString) -> "DataFrameWriter":
+        ...
+
+    def bucketBy(
+        self, numBuckets: int, col: Union[str, TupleOrListOfString], *cols: Optional[str]
+    ) -> "DataFrameWriter":
+        """Buckets the output by the given columns. If specified,
+        the output is laid out on the file system similar to Hive's bucketing scheme,
+        but with a different bucket hash function and is not compatible with Hive's bucketing.
+
+        .. versionadded:: 2.3.0
+
+        Parameters
+        ----------
+        numBuckets : int
+            the number of buckets to save
+        col : str, list or tuple
+            a name of a column, or a list of names.
+        cols : str
+            additional names (optional). If `col` is a list it should be empty.
+
+        Notes
+        -----
+        Applicable for file-based data sources in combination with
+        :py:meth:`DataFrameWriter.saveAsTable`.
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a buckted manner, and read it back.
+
+        >>> from pyspark.sql.functions import input_file_name
+        >>> # Write a DataFrame into a Parquet file in a bucketed manner.
+        ... _ = spark.sql("DROP TABLE IF EXISTS bucketed_table")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.bucketBy(2, "name").mode("overwrite").saveAsTable("bucketed_table")
+        >>> # Read the Parquet file as a DataFrame.
+        ... spark.read.table("bucketed_table").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE bucketed_table")
+        """
+        if not isinstance(numBuckets, int):
+            raise TypeError("numBuckets should be an int, got {0}.".format(type(numBuckets)))
+
+        if isinstance(col, (list, tuple)):
+            if cols:
+                raise ValueError("col is a {0} but cols are not empty".format(type(col)))
+
+            col, cols = col[0], col[1:]  # type: ignore[assignment]
+
+        if not all(isinstance(c, str) for c in cols) or not (isinstance(col, str)):
+            raise TypeError("all names should be `str`")
+
+        self._write.num_buckets = numBuckets
+        self._write.bucket_cols = cast(List[str], cols)
+        return self
+
+    @overload
+    def sortBy(self, col: str, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def sortBy(self, col: TupleOrListOfString) -> "DataFrameWriter":
+        ...
+
+    def sortBy(
+        self, col: Union[str, TupleOrListOfString], *cols: Optional[str]
+    ) -> "DataFrameWriter":
+        """Sorts the output in each bucket by the given columns on the file system.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        col : str, tuple or list
+            a name of a column, or a list of names.
+        cols : str
+            additional names (optional). If `col` is a list it should be empty.
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a sorted-buckted manner, and read it back.
+
+        >>> from pyspark.sql.functions import input_file_name
+        >>> # Write a DataFrame into a Parquet file in a sorted-bucketed manner.
+        ... _ = spark.sql("DROP TABLE IF EXISTS sorted_bucketed_table")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.bucketBy(1, "name").sortBy("age").mode(
+        ...     "overwrite").saveAsTable("sorted_bucketed_table")
+        >>> # Read the Parquet file as a DataFrame.
+        ... spark.read.table("sorted_bucketed_table").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE sorted_bucketed_table")
+        """
+        if isinstance(col, (list, tuple)):
+            if cols:
+                raise ValueError("col is a {0} but cols are not empty".format(type(col)))
+
+            col, cols = col[0], col[1:]  # type: ignore[assignment]
+
+        if not all(isinstance(c, str) for c in cols) or not (isinstance(col, str)):
+            raise TypeError("all names should be `str`")
+
+        self._write.sort_cols = cast(List[str], cols)
+        return self
+
+    def save(
+        self,
+        path: Optional[str] = None,
+        format: Optional[str] = None,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """Saves the contents of the :class:`DataFrame` to a data source.
+
+        The data source is specified by the ``format`` and a set of ``options``.
+        If ``format`` is not specified, the default data source configured by
+        ``spark.sql.sources.default`` will be used.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str, optional
+            the path in a Hadoop supported file system
+        format : str, optional
+            the format used to save
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+        partitionBy : list, optional
+            names of partitioning columns
+        **options : dict
+            all other string options
+
+        Examples
+        --------
+        Write a DataFrame into a JSON file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a JSON file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("json").save(d)
+        ...
+        ...     # Read the JSON file as a DataFrame.
+        ...     spark.read.format('json').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode).options(**options)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        if format is not None:
+            self.format(format)
+        self._write.path = path
+        self._spark.execute_command(self._write.command(self._spark))
+
+    def insertInto(self, tableName: str, overwrite: Optional[bool] = None) -> None:
+        """Inserts the content of the :class:`DataFrame` to the specified table.
+
+        It requires that the schema of the :class:`DataFrame` is the same as the
+        schema of the table.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        overwrite : bool, optional
+            If true, overwrites existing data. Disabled by default
+
+        Notes
+        -----
+        Unlike :meth:`DataFrameWriter.saveAsTable`, :meth:`DataFrameWriter.insertInto` ignores
+        the column names and just uses position-based resolution.
+
+        Examples
+        --------
+        >>> _ = spark.sql("DROP TABLE IF EXISTS tblA")
+        >>> df = spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... )
+        >>> df.write.saveAsTable("tblA")
+
+        Insert the data into 'tblA' table but with different column names.
+
+        >>> df.selectExpr("age AS col1", "name AS col2").write.insertInto("tblA")
+        >>> spark.read.table("tblA").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE tblA")
+        """
+        if overwrite is not None:
+            self.mode("overwrite" if overwrite else "append")
+        self.saveAsTable(tableName)
+
+    def saveAsTable(
+        self,
+        name: str,
+        format: Optional[str] = None,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` as the specified table.
+
+        In the case the table already exists, behavior of this function depends on the
+        save mode, specified by the `mode` function (default to throwing an exception).
+        When `mode` is `Overwrite`, the schema of the :class:`DataFrame` does not need to be
+        the same as that of the existing table.
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Notes
+        -----
+        When `mode` is `Append`, if there is an existing table, we will use the format and
+        options of the existing table. The column order in the schema of the :class:`DataFrame`
+        doesn't need to be same as that of the existing table. Unlike
+        :meth:`DataFrameWriter.insertInto`, :meth:`DataFrameWriter.saveAsTable` will use the
+        column names to find the correct column positions.
+
+        Parameters
+        ----------
+        name : str
+            the table name
+        format : str, optional
+            the format used to save
+        mode : str, optional
+            one of `append`, `overwrite`, `error`, `errorifexists`, `ignore` \
+            (default: error)
+        partitionBy : str or list
+            names of partitioning columns
+        **options : dict
+            all other string options
+
+        Examples
+        --------
+        Creates a table from a DataFrame, and read it back.
+
+        >>> _ = spark.sql("DROP TABLE IF EXISTS tblA")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.saveAsTable("tblA")
+        >>> spark.read.table("tblA").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE tblA")
+        """
+        self.mode(mode).options(**options)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        if format is not None:
+            self.format(format)
+        self._write.table_name = name
+        self._spark.execute_command(self._write.command(self._spark))
+
+    def json(
+        self,
+        path: str,
+        mode: Optional[str] = None,
+        compression: Optional[str] = None,
+        dateFormat: Optional[str] = None,
+        timestampFormat: Optional[str] = None,
+        lineSep: Optional[str] = None,
+        encoding: Optional[str] = None,
+        ignoreNullFields: Optional[Union[bool, str]] = None,
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` in JSON format
+        (`JSON Lines text format or newline-delimited JSON <http://jsonlines.org/>`_) at the
+        specified path.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-json.html#data-source-option>`_
+            in the version you use.
+
+            .. # noqa
+
+        Examples
+        --------
+        Write a DataFrame into a JSON file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a JSON file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.json(d, mode="overwrite")
+        ...
+        ...     # Read the JSON file as a DataFrame.
+        ...     spark.read.format("json").load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode)
+        self._set_opts(
+            compression=compression,
+            dateFormat=dateFormat,
+            timestampFormat=timestampFormat,
+            lineSep=lineSep,
+            encoding=encoding,
+            ignoreNullFields=ignoreNullFields,
+        )
+        self.format("json").save(path)
+
+    def parquet(
+        self,
+        path: str,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        compression: Optional[str] = None,
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` in Parquet format at the specified path.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+        partitionBy : str or list, optional
+            names of partitioning columns
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#data-source-option>`_
+            in the version you use.
+
+            .. # noqa
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.parquet(d, mode="overwrite")
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.format("parquet").load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        self.option("compression", compression)
+        self.format("parquet").save(path)
+
+    def text(
+        self, path: str, compression: Optional[str] = None, lineSep: Optional[str] = None
+    ) -> None:
+        """Saves the content of the DataFrame in a text file at the specified path.
+        The text files will be encoded as UTF-8.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-text.html#data-source-option>`_
+            in the version you use.
+
+            .. # noqa
+
+        Notes
+        -----
+        The DataFrame must have only one column that is of string type.
+        Each row becomes a new line in the output file.
+
+        Examples
+        --------
+        Write a DataFrame into a text file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a text file
+        ...     df = spark.createDataFrame([("a",), ("b",), ("c",)], schema=["alphabets"])
+        ...     df.write.mode("overwrite").text(d)
+        ...
+        ...     # Read the text file as a DataFrame.
+        ...     spark.read.schema(df.schema).format("text").load(d).sort("alphabets").show()
+        +---------+
+        |alphabets|
+        +---------+
+        |        a|
+        |        b|
+        |        c|
+        +---------+
+        """
+        self._set_opts(compression=compression, lineSep=lineSep)
+        self.format("text").save(path)
+
+    def csv(
+        self,
+        path: str,
+        mode: Optional[str] = None,
+        compression: Optional[str] = None,
+        sep: Optional[str] = None,
+        quote: Optional[str] = None,
+        escape: Optional[str] = None,
+        header: Optional[Union[bool, str]] = None,
+        nullValue: Optional[str] = None,
+        escapeQuotes: Optional[Union[bool, str]] = None,
+        quoteAll: Optional[Union[bool, str]] = None,
+        dateFormat: Optional[str] = None,
+        timestampFormat: Optional[str] = None,
+        ignoreLeadingWhiteSpace: Optional[Union[bool, str]] = None,
+        ignoreTrailingWhiteSpace: Optional[Union[bool, str]] = None,
+        charToEscapeQuoteEscaping: Optional[str] = None,
+        encoding: Optional[str] = None,
+        emptyValue: Optional[str] = None,
+        lineSep: Optional[str] = None,
+    ) -> None:
+        r"""Saves the content of the :class:`DataFrame` in CSV format at the specified path.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-csv.html#data-source-option>`_
+            in the version you use.
+
+            .. # noqa
+
+        Examples
+        --------
+        Write a DataFrame into a CSV file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file
+        ...     df = spark.createDataFrame([{"age": 100, "name": "Hyukjin Kwon"}])
+        ...     df.write.csv(d, mode="overwrite")
+        ...
+        ...     # Read the CSV file as a DataFrame with 'nullValue' option set to 'Hyukjin Kwon'.
+        ...     spark.read.schema(df.schema).format("csv").option(
+        ...         "nullValue", "Hyukjin Kwon").load(d).show()
+        +---+----+
+        |age|name|
+        +---+----+
+        |100|null|
+        +---+----+
+        """
+        self.mode(mode)
+        self._set_opts(
+            compression=compression,
+            sep=sep,
+            quote=quote,
+            escape=escape,
+            header=header,
+            nullValue=nullValue,
+            escapeQuotes=escapeQuotes,
+            quoteAll=quoteAll,
+            dateFormat=dateFormat,
+            timestampFormat=timestampFormat,
+            ignoreLeadingWhiteSpace=ignoreLeadingWhiteSpace,
+            ignoreTrailingWhiteSpace=ignoreTrailingWhiteSpace,
+            charToEscapeQuoteEscaping=charToEscapeQuoteEscaping,
+            encoding=encoding,
+            emptyValue=emptyValue,
+            lineSep=lineSep,
+        )
+        self.format("csv").save(path)
+
+    def orc(
+        self,
+        path: str,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        compression: Optional[str] = None,
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` in ORC format at the specified path.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+        partitionBy : str or list, optional
+            names of partitioning columns
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-orc.html#data-source-option>`_
+            in the version you use.
+
+            .. # noqa
+
+        Examples
+        --------
+        Write a DataFrame into a ORC file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a ORC file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.orc(d, mode="overwrite")
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.format("orc").load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        self._set_opts(compression=compression)
+        self.format("orc").save(path)
+
+    def jdbc(self, *args: Any, **kwargs: Any) -> None:

Review Comment:
   We've been discussing this and the conclusion was that it's better to give a clear error message than failing in a weird way. For Scala it's a bit different because you will get this error at compile time, but in Python you will get a method not found at runtime error that might not be as clear.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HyukjinKwon commented on a diff in pull request #38801: [SPARK-41317][CONNECT][PYTHON] Add basic support for DataFrameWriter

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on code in PR #38801:
URL: https://github.com/apache/spark/pull/38801#discussion_r1034418111


##########
python/pyspark/sql/connect/readwriter.py:
##########
@@ -140,9 +161,891 @@ def load(
             self.option("path", path)
 
         plan = DataSource(format=self._format, schema=self._schema, options=self._options)
-        df = DataFrame.withPlan(plan, self._client)
-        return df
+        return self._df(plan)
+
+    def _df(self, plan: LogicalPlan) -> "DataFrame":
+        # The import is needed here to avoid circular import issues.
+        from pyspark.sql.connect.dataframe import DataFrame
+
+        return DataFrame.withPlan(plan, self._client)
 
     def table(self, tableName: str) -> "DataFrame":
-        df = DataFrame.withPlan(Read(tableName), self._client)
-        return df
+        return self._df(Read(tableName))
+
+
+class DataFrameWriter(OptionUtils):
+    """
+    Interface used to write a :class:`DataFrame` to external storage systems
+    (e.g. file systems, key-value stores, etc). Use :attr:`DataFrame.write`
+    to access this.
+
+    .. versionadded:: 3.4.0
+    """
+
+    def __init__(self, plan: "LogicalPlan", session: "SparkSession"):
+        self._df: "LogicalPlan" = plan
+        self._spark: "SparkSession" = session
+        self._write: "WriteOperation" = WriteOperation(self._df)
+
+    def mode(self, saveMode: Optional[str]) -> "DataFrameWriter":
+        """Specifies the behavior when data or table already exists.
+
+        Options include:
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Examples
+        --------
+        Raise an error when writing to an existing path.
+
+        >>> import tempfile

Review Comment:
   Quick comment.. We should 1. add set up the globals e.g., https://github.com/apache/spark/blob/master/python/pyspark/sql/readwriter.py#L1969-L1994 and 2. add it into https://github.com/apache/spark/blob/master/dev/sparktestsupport/modules.py#L506-L508 to run them as doctests ..
   
   Would have to be done separately I guess.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] bjornjorgensen commented on a diff in pull request #38801: [SPARK-40539] [CONNECT] [PYTHON] Add basic support for DataFrameWriter

Posted by GitBox <gi...@apache.org>.
bjornjorgensen commented on code in PR #38801:
URL: https://github.com/apache/spark/pull/38801#discussion_r1032792236


##########
python/pyspark/sql/connect/writer.py:
##########
@@ -0,0 +1,922 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+
+from typing import Optional, Union, List, overload, Tuple, cast, Any
+from typing import TYPE_CHECKING
+
+from pyspark.sql.connect.plan import WriteOperation, LogicalPlan
+from pyspark.sql.types import StructType
+from pyspark.sql.utils import to_str
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect._typing import OptionalPrimitiveType
+    from pyspark.sql.connect.client import RemoteSparkSession
+
+
+PathOrPaths = Union[str, List[str]]
+TupleOrListOfString = Union[List[str], Tuple[str, ...]]
+
+
+class OptionUtils:
+    def _set_opts(
+        self,
+        schema: Optional[Union[StructType, str]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """
+        Set named options (filter out those the value is None)
+        """
+        if schema is not None:
+            self.schema(schema)  # type: ignore[attr-defined]
+        for k, v in options.items():
+            if v is not None:
+                self.option(k, v)  # type: ignore[attr-defined]
+
+
+class DataFrameWriter(OptionUtils):
+    """
+    Interface used to write a :class:`DataFrame` to external storage systems
+    (e.g. file systems, key-value stores, etc). Use :attr:`DataFrame.write`
+    to access this.
+
+    .. versionadded:: 3.4.0
+    """
+
+    def __init__(self, plan: "LogicalPlan", session: "RemoteSparkSession"):
+        self._df: "LogicalPlan" = plan
+        self._spark: "RemoteSparkSession" = session
+        self._write: "WriteOperation" = WriteOperation(self._df)
+
+    def mode(self, saveMode: Optional[str]) -> "DataFrameWriter":
+        """Specifies the behavior when data or table already exists.
+
+        Options include:
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Examples
+        --------
+        Raise an error when writing to an existing path.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     spark.createDataFrame(
+        ...         [{"age": 80, "name": "Xinrong Meng"}]
+        ...     ).write.mode("error").format("parquet").save(d)
+        Traceback (most recent call last):
+            ...
+        pyspark.sql.utils.AnalysisException: ...
+
+        Write a Parquet file back with various options, and read it back.
+
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Overwrite the path with a new Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 120, "name": "Takuya Ueshin"}]
+        ...     ).write.mode("append").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 140, "name": "Haejoon Lee"}]
+        ...     ).write.mode("ignore").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |120|Takuya Ueshin|
+        |100| Hyukjin Kwon|
+        +---+-------------+
+        """
+        # At the JVM side, the default value of mode is already set to "error".
+        # So, if the given saveMode is None, we will not call JVM-side's mode method.
+        if saveMode is not None:
+            self._write.mode = saveMode
+        return self
+
+    def format(self, source: str) -> "DataFrameWriter":
+        """Specifies the underlying output data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        source : str
+            string, name of the data source, e.g. 'json', 'parquet'.
+
+        Examples
+        --------
+        >>> spark.range(1).write.format('parquet')
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Write a DataFrame into a Parquet file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.format('parquet').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.source = source
+        return self
+
+    def option(self, key: str, value: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds a output option for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        key : str
+            The key for the option to set.
+        value
+            The value for the option to set.
+
+        Examples
+        --------
+        >>> spark.range(1).write.option("key", "value")
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Specify the option 'nullValue' with writing a CSV file.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file with 'nullValue' option set to 'Hyukjin Kwon'.
+        ...     df = spark.createDataFrame([(100, None)], "age INT, name STRING")
+        ...     df.write.option("nullValue", "Hyukjin Kwon").mode("overwrite").format("csv").save(d)
+        ...
+        ...     # Read the CSV file as a DataFrame.
+        ...     spark.read.schema(df.schema).format('csv').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.options[key] = to_str(value)
+        return self
+
+    def options(self, **options: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds output options for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        **options : dict
+            The dictionary of string keys and prmitive-type values.
+
+        Examples
+        --------
+        >>> spark.range(1).write.option("key", "value")
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Specify the option 'nullValue' and 'header' with writing a CSV file.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file with 'nullValue' option set to 'Hyukjin Kwon',
+        ...     # and 'header' option set to `True`.
+        ...     df = spark.createDataFrame([(100, "Hyukjin Kwon")], ["age", "name"])

Review Comment:
   Change this to df = spark.createDataFrame([(100, None)], ["age", "name"])



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] grundprinzip commented on a diff in pull request #38801: [SPARK-40539] [CONNECT] [PYTHON] Add basic support for DataFrameWriter

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #38801:
URL: https://github.com/apache/spark/pull/38801#discussion_r1033220681


##########
python/pyspark/sql/tests/connect/test_connect_basic.py:
##########
@@ -566,6 +569,27 @@ def test_agg_with_two_agg_exprs(self):
             self.spark.read.table(self.tbl_name).agg({"name": "min", "id": "max"}).toPandas(),
         )
 
+    def test_write_operations(self):
+        with tempfile.TemporaryDirectory() as d:
+            df = self.connect.range(1, 100)
+            df.write.mode("overwrite").format("csv").save(d)
+
+            ndf = self.connect.read.load(d, format="csv")
+            df.toPandas().equals(ndf.toPandas())

Review Comment:
   it's actually `self.assert_eq` which is a custom function provided in the test suite



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] grundprinzip commented on a diff in pull request #38801: [SPARK-40539] [CONNECT] [PYTHON] Add basic support for DataFrameWriter

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #38801:
URL: https://github.com/apache/spark/pull/38801#discussion_r1033194065


##########
python/pyspark/sql/tests/connect/test_connect_basic.py:
##########
@@ -566,6 +569,27 @@ def test_agg_with_two_agg_exprs(self):
             self.spark.read.table(self.tbl_name).agg({"name": "min", "id": "max"}).toPandas(),
         )
 
+    def test_write_operations(self):
+        with tempfile.TemporaryDirectory() as d:
+            df = self.connect.range(1, 100)
+            df.write.mode("overwrite").format("csv").save(d)
+
+            ndf = self.connect.read.load(d, format="csv")
+            df.toPandas().equals(ndf.toPandas())

Review Comment:
   Done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HyukjinKwon commented on pull request #38801: [SPARK-41317][CONNECT][PYTHON] Add basic support for DataFrameWriter

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on PR #38801:
URL: https://github.com/apache/spark/pull/38801#issuecomment-1330374602

   is this maybe flaky (?) Would be good to try to retrigger https://github.com/grundprinzip/spark/actions/runs/3568989326/jobs/5998522486


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] bjornjorgensen commented on a diff in pull request #38801: [SPARK-40539] [CONNECT] [PYTHON] Add basic support for DataFrameWriter

Posted by GitBox <gi...@apache.org>.
bjornjorgensen commented on code in PR #38801:
URL: https://github.com/apache/spark/pull/38801#discussion_r1032662076


##########
python/pyspark/sql/connect/writer.py:
##########
@@ -0,0 +1,922 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+
+from typing import Optional, Union, List, overload, Tuple, cast, Any
+from typing import TYPE_CHECKING
+
+from pyspark.sql.connect.plan import WriteOperation, LogicalPlan
+from pyspark.sql.types import StructType
+from pyspark.sql.utils import to_str
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect._typing import OptionalPrimitiveType
+    from pyspark.sql.connect.client import RemoteSparkSession
+
+
+PathOrPaths = Union[str, List[str]]
+TupleOrListOfString = Union[List[str], Tuple[str, ...]]
+
+
+class OptionUtils:
+    def _set_opts(
+        self,
+        schema: Optional[Union[StructType, str]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """
+        Set named options (filter out those the value is None)
+        """
+        if schema is not None:
+            self.schema(schema)  # type: ignore[attr-defined]
+        for k, v in options.items():
+            if v is not None:
+                self.option(k, v)  # type: ignore[attr-defined]
+
+
+class DataFrameWriter(OptionUtils):
+    """
+    Interface used to write a :class:`DataFrame` to external storage systems
+    (e.g. file systems, key-value stores, etc). Use :attr:`DataFrame.write`
+    to access this.
+
+    .. versionadded:: 3.4.0
+    """
+
+    def __init__(self, plan: "LogicalPlan", session: "RemoteSparkSession"):
+        self._df: "LogicalPlan" = plan
+        self._spark: "RemoteSparkSession" = session
+        self._write: "WriteOperation" = WriteOperation(self._df)
+
+    def mode(self, saveMode: Optional[str]) -> "DataFrameWriter":
+        """Specifies the behavior when data or table already exists.
+
+        Options include:
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Examples
+        --------
+        Raise an error when writing to an existing path.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     spark.createDataFrame(
+        ...         [{"age": 80, "name": "Xinrong Meng"}]
+        ...     ).write.mode("error").format("parquet").save(d)
+        Traceback (most recent call last):
+            ...
+        pyspark.sql.utils.AnalysisException: ...
+
+        Write a Parquet file back with various options, and read it back.
+
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Overwrite the path with a new Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 120, "name": "Takuya Ueshin"}]
+        ...     ).write.mode("append").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 140, "name": "Haejoon Lee"}]
+        ...     ).write.mode("ignore").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |120|Takuya Ueshin|
+        |100| Hyukjin Kwon|
+        +---+-------------+
+        """
+        # At the JVM side, the default value of mode is already set to "error".
+        # So, if the given saveMode is None, we will not call JVM-side's mode method.
+        if saveMode is not None:
+            self._write.mode = saveMode
+        return self
+
+    def format(self, source: str) -> "DataFrameWriter":
+        """Specifies the underlying output data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        source : str
+            string, name of the data source, e.g. 'json', 'parquet'.
+
+        Examples
+        --------
+        >>> spark.range(1).write.format('parquet')
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Write a DataFrame into a Parquet file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.format('parquet').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.source = source
+        return self
+
+    def option(self, key: str, value: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds a output option for the underlying data source.

Review Comment:
   Adds an output



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] grundprinzip commented on a diff in pull request #38801: [SPARK-40539] [CONNECT] [PYTHON] Add basic support for DataFrameWriter

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #38801:
URL: https://github.com/apache/spark/pull/38801#discussion_r1033228943


##########
python/pyspark/sql/tests/connect/test_connect_basic.py:
##########
@@ -566,6 +569,27 @@ def test_agg_with_two_agg_exprs(self):
             self.spark.read.table(self.tbl_name).agg({"name": "min", "id": "max"}).toPandas(),
         )
 
+    def test_write_operations(self):
+        with tempfile.TemporaryDirectory() as d:
+            df = self.connect.range(1, 100)
+            df.write.mode("overwrite").format("csv").save(d)
+
+            ndf = self.connect.read.load(d, format="csv")
+            df.toPandas().equals(ndf.toPandas())

Review Comment:
   I'm using collect now to avoid some data type issues.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] bjornjorgensen commented on pull request #38801: [SPARK-40539] [CONNECT] [PYTHON] Add basic support for DataFrameWriter

Posted by GitBox <gi...@apache.org>.
bjornjorgensen commented on PR #38801:
URL: https://github.com/apache/spark/pull/38801#issuecomment-1327867830

   @grundprinzip Some language changes also I don't get all the examples with None values to match completely.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] grundprinzip commented on a diff in pull request #38801: [SPARK-40539] [CONNECT] [PYTHON] Add basic support for DataFrameWriter

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #38801:
URL: https://github.com/apache/spark/pull/38801#discussion_r1033007152


##########
python/pyspark/sql/connect/writer.py:
##########
@@ -0,0 +1,926 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+
+from typing import Optional, Union, List, overload, Tuple, cast, Any
+from typing import TYPE_CHECKING
+
+from pyspark.sql.connect.plan import WriteOperation, LogicalPlan
+from pyspark.sql.types import StructType
+from pyspark.sql.utils import to_str
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect._typing import OptionalPrimitiveType
+    from pyspark.sql.connect.session import SparkSession
+
+
+PathOrPaths = Union[str, List[str]]
+TupleOrListOfString = Union[List[str], Tuple[str, ...]]
+
+
+class OptionUtils:
+    def _set_opts(
+        self,
+        schema: Optional[Union[StructType, str]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """
+        Set named options (filter out those the value is None)
+        """
+        if schema is not None:
+            self.schema(schema)  # type: ignore[attr-defined]
+        for k, v in options.items():
+            if v is not None:
+                self.option(k, v)  # type: ignore[attr-defined]
+
+
+class DataFrameWriter(OptionUtils):
+    """
+    Interface used to write a :class:`DataFrame` to external storage systems
+    (e.g. file systems, key-value stores, etc). Use :attr:`DataFrame.write`
+    to access this.
+
+    .. versionadded:: 3.4.0
+    """
+
+    def __init__(self, plan: "LogicalPlan", session: "SparkSession"):
+        self._df: "LogicalPlan" = plan
+        self._spark: "SparkSession" = session
+        self._write: "WriteOperation" = WriteOperation(self._df)
+
+    def mode(self, saveMode: Optional[str]) -> "DataFrameWriter":
+        """Specifies the behavior when data or table already exists.
+
+        Options include:
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Examples
+        --------
+        Raise an error when writing to an existing path.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     spark.createDataFrame(
+        ...         [{"age": 80, "name": "Xinrong Meng"}]
+        ...     ).write.mode("error").format("parquet").save(d)
+        Traceback (most recent call last):
+            ...
+        pyspark.sql.utils.AnalysisException: ...
+
+        Write a Parquet file back with various options, and read it back.
+
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Overwrite the path with a new Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 120, "name": "Takuya Ueshin"}]
+        ...     ).write.mode("append").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 140, "name": "Haejoon Lee"}]
+        ...     ).write.mode("ignore").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |120|Takuya Ueshin|
+        |100| Hyukjin Kwon|
+        +---+-------------+
+        """
+        # At the JVM side, the default value of mode is already set to "error".
+        # So, if the given saveMode is None, we will not call JVM-side's mode method.
+        if saveMode is not None:
+            self._write.mode = saveMode
+        return self
+
+    def format(self, source: str) -> "DataFrameWriter":
+        """Specifies the underlying output data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        source : str
+            string, name of the data source, e.g. 'json', 'parquet'.
+
+        Examples
+        --------
+        >>> spark.range(1).write.format('parquet')
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Write a DataFrame into a Parquet file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.format('parquet').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.source = source
+        return self
+
+    def option(self, key: str, value: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds an output option for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        key : str
+            The key for the option to set.
+        value
+            The value for the option to set.
+
+        Examples
+        --------
+        >>> spark.range(1).write.option("key", "value")
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Specify the option 'nullValue' with writing a CSV file.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file with 'nullValue' option set to 'Hyukjin Kwon'.
+        ...     df = spark.createDataFrame([(100, None)], "age INT, name STRING")
+        ...     df.write.option("nullValue", "Hyukjin Kwon").mode("overwrite").format("csv").save(d)
+        ...
+        ...     # Read the CSV file as a DataFrame.
+        ...     spark.read.schema(df.schema).format('csv').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.options[key] = to_str(value)
+        return self
+
+    def options(self, **options: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds output options for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        **options : dict
+            The dictionary of string keys and primitive-type values.
+
+        Examples
+        --------
+        >>> spark.range(1).write.option("key", "value")
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Specify the option 'nullValue' and 'header' with writing a CSV file.
+        >>> from pyspark.sql.types import StructType,StructField, StringType, IntegerType
+        >>> schema = StructType([
+        ...     StructField("age",IntegerType(),True),
+        ...     StructField("name",StringType(),True),
+        ...])
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file with 'nullValue' option set to 'Hyukjin Kwon',
+        ...     # and 'header' option set to `True`.
+        ...     df = spark.createDataFrame([(100, None], schema=schema)
+        ...     df.write.options(nullValue="Hyukjin Kwon", header=True).mode(
+        ...         "overwrite").format("csv").save(d)
+        ...
+        ...     # Read the CSV file as a DataFrame.
+        ...     spark.read.option("header", True).format('csv').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        for k in options:
+            self._write.options[k] = to_str(options[k])
+        return self
+
+    @overload
+    def partitionBy(self, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def partitionBy(self, *cols: List[str]) -> "DataFrameWriter":
+        ...
+
+    def partitionBy(self, *cols: Union[str, List[str]]) -> "DataFrameWriter":
+        """Partitions the output by the given columns on the file system.
+
+        If specified, the output is laid out on the file system similar
+        to Hive's partitioning scheme.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        cols : str or list
+            name of columns
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a partitioned manner, and read it back.
+
+        >>> import tempfile
+        >>> import os
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file in a partitioned manner.
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}, {"age": 120, "name": "Ruifeng Zheng"}]
+        ...     ).write.partitionBy("name").mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).sort("age").show()
+        ...
+        ...     # Read one partition as a DataFrame.
+        ...     spark.read.parquet(f"{d}{os.path.sep}name=Hyukjin Kwon").show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |100| Hyukjin Kwon|
+        |120|Ruifeng Zheng|
+        +---+-------------+
+        +---+
+        |age|
+        +---+
+        |100|
+        +---+
+        """
+        if len(cols) == 1 and isinstance(cols[0], (list, tuple)):
+            cols = cols[0]  # type: ignore[assignment]
+
+        self._write.partitioning_cols = cast(List[str], cols)
+        return self
+
+    @overload
+    def bucketBy(self, numBuckets: int, col: str, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def bucketBy(self, numBuckets: int, col: TupleOrListOfString) -> "DataFrameWriter":
+        ...
+
+    def bucketBy(
+        self, numBuckets: int, col: Union[str, TupleOrListOfString], *cols: Optional[str]
+    ) -> "DataFrameWriter":
+        """Buckets the output by the given columns. If specified,
+        the output is laid out on the file system similar to Hive's bucketing scheme,
+        but with a different bucket hash function and is not compatible with Hive's bucketing.
+
+        .. versionadded:: 2.3.0
+
+        Parameters
+        ----------
+        numBuckets : int
+            the number of buckets to save
+        col : str, list or tuple
+            a name of a column, or a list of names.
+        cols : str
+            additional names (optional). If `col` is a list it should be empty.
+
+        Notes
+        -----
+        Applicable for file-based data sources in combination with
+        :py:meth:`DataFrameWriter.saveAsTable`.
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a buckted manner, and read it back.
+
+        >>> from pyspark.sql.functions import input_file_name
+        >>> # Write a DataFrame into a Parquet file in a bucketed manner.
+        ... _ = spark.sql("DROP TABLE IF EXISTS bucketed_table")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.bucketBy(2, "name").mode("overwrite").saveAsTable("bucketed_table")
+        >>> # Read the Parquet file as a DataFrame.
+        ... spark.read.table("bucketed_table").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE bucketed_table")
+        """
+        if not isinstance(numBuckets, int):
+            raise TypeError("numBuckets should be an int, got {0}.".format(type(numBuckets)))
+
+        if isinstance(col, (list, tuple)):
+            if cols:
+                raise ValueError("col is a {0} but cols are not empty".format(type(col)))
+
+            col, cols = col[0], col[1:]  # type: ignore[assignment]
+
+        if not all(isinstance(c, str) for c in cols) or not (isinstance(col, str)):
+            raise TypeError("all names should be `str`")
+
+        self._write.num_buckets = numBuckets
+        self._write.bucket_cols = cast(List[str], cols)
+        return self
+
+    @overload
+    def sortBy(self, col: str, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def sortBy(self, col: TupleOrListOfString) -> "DataFrameWriter":
+        ...
+
+    def sortBy(
+        self, col: Union[str, TupleOrListOfString], *cols: Optional[str]
+    ) -> "DataFrameWriter":
+        """Sorts the output in each bucket by the given columns on the file system.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        col : str, tuple or list
+            a name of a column, or a list of names.
+        cols : str
+            additional names (optional). If `col` is a list it should be empty.
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a sorted-buckted manner, and read it back.
+
+        >>> from pyspark.sql.functions import input_file_name
+        >>> # Write a DataFrame into a Parquet file in a sorted-bucketed manner.
+        ... _ = spark.sql("DROP TABLE IF EXISTS sorted_bucketed_table")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.bucketBy(1, "name").sortBy("age").mode(
+        ...     "overwrite").saveAsTable("sorted_bucketed_table")
+        >>> # Read the Parquet file as a DataFrame.
+        ... spark.read.table("sorted_bucketed_table").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE sorted_bucketed_table")
+        """
+        if isinstance(col, (list, tuple)):
+            if cols:
+                raise ValueError("col is a {0} but cols are not empty".format(type(col)))
+
+            col, cols = col[0], col[1:]  # type: ignore[assignment]
+
+        if not all(isinstance(c, str) for c in cols) or not (isinstance(col, str)):
+            raise TypeError("all names should be `str`")
+
+        self._write.sort_cols = cast(List[str], cols)
+        return self
+
+    def save(
+        self,
+        path: Optional[str] = None,
+        format: Optional[str] = None,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """Saves the contents of the :class:`DataFrame` to a data source.
+
+        The data source is specified by the ``format`` and a set of ``options``.
+        If ``format`` is not specified, the default data source configured by
+        ``spark.sql.sources.default`` will be used.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str, optional
+            the path in a Hadoop supported file system
+        format : str, optional
+            the format used to save
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+        partitionBy : list, optional
+            names of partitioning columns
+        **options : dict
+            all other string options
+
+        Examples
+        --------
+        Write a DataFrame into a JSON file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a JSON file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("json").save(d)
+        ...
+        ...     # Read the JSON file as a DataFrame.
+        ...     spark.read.format('json').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode).options(**options)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        if format is not None:
+            self.format(format)
+        self._write.path = path
+        self._spark.client.execute_command(self._write.command(self._spark.client))
+
+    def insertInto(self, tableName: str, overwrite: Optional[bool] = None) -> None:
+        """Inserts the content of the :class:`DataFrame` to the specified table.
+
+        It requires that the schema of the :class:`DataFrame` is the same as the
+        schema of the table.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        overwrite : bool, optional
+            If true, overwrites existing data. Disabled by default
+
+        Notes
+        -----
+        Unlike :meth:`DataFrameWriter.saveAsTable`, :meth:`DataFrameWriter.insertInto` ignores
+        the column names and just uses position-based resolution.
+
+        Examples
+        --------
+        >>> _ = spark.sql("DROP TABLE IF EXISTS tblA")
+        >>> df = spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... )
+        >>> df.write.saveAsTable("tblA")
+
+        Insert the data into 'tblA' table but with different column names.
+
+        >>> df.selectExpr("age AS col1", "name AS col2").write.insertInto("tblA")
+        >>> spark.read.table("tblA").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE tblA")
+        """
+        if overwrite is not None:
+            self.mode("overwrite" if overwrite else "append")
+        self.saveAsTable(tableName)
+
+    def saveAsTable(
+        self,
+        name: str,
+        format: Optional[str] = None,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` as the specified table.
+
+        In the case the table already exists, behavior of this function depends on the
+        save mode, specified by the `mode` function (default to throwing an exception).
+        When `mode` is `Overwrite`, the schema of the :class:`DataFrame` does not need to be
+        the same as that of the existing table.
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Notes
+        -----
+        When `mode` is `Append`, if there is an existing table, we will use the format and
+        options of the existing table. The column order in the schema of the :class:`DataFrame`
+        doesn't need to be the same as that of the existing table. Unlike
+        :meth:`DataFrameWriter.insertInto`, :meth:`DataFrameWriter.saveAsTable` will use the
+        column names to find the correct column positions.
+
+        Parameters
+        ----------
+        name : str
+            the table name
+        format : str, optional
+            the format used to save
+        mode : str, optional
+            one of `append`, `overwrite`, `error`, `errorifexists`, `ignore` \
+            (default: error)
+        partitionBy : str or list
+            names of partitioning columns
+        **options : dict
+            all other string options
+
+        Examples
+        --------
+        Creates a table from a DataFrame, and read it back.
+
+        >>> _ = spark.sql("DROP TABLE IF EXISTS tblA")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.saveAsTable("tblA")
+        >>> spark.read.table("tblA").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE tblA")
+        """
+        self.mode(mode).options(**options)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        if format is not None:
+            self.format(format)
+        self._write.table_name = name
+        self._spark.client.execute_command(self._write.command(self._spark.client))
+
+    def json(
+        self,
+        path: str,
+        mode: Optional[str] = None,
+        compression: Optional[str] = None,
+        dateFormat: Optional[str] = None,
+        timestampFormat: Optional[str] = None,
+        lineSep: Optional[str] = None,
+        encoding: Optional[str] = None,
+        ignoreNullFields: Optional[Union[bool, str]] = None,
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` in JSON format
+        (`JSON Lines text format or newline-delimited JSON <http://jsonlines.org/>`_) at the
+        specified path.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-json.html#data-source-option>`_
+            for the version you use.
+
+            .. # noqa
+
+        Examples
+        --------
+        Write a DataFrame into a JSON file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a JSON file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.json(d, mode="overwrite")
+        ...
+        ...     # Read the JSON file as a DataFrame.
+        ...     spark.read.format("json").load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode)
+        self._set_opts(
+            compression=compression,
+            dateFormat=dateFormat,
+            timestampFormat=timestampFormat,
+            lineSep=lineSep,
+            encoding=encoding,
+            ignoreNullFields=ignoreNullFields,
+        )
+        self.format("json").save(path)
+
+    def parquet(
+        self,
+        path: str,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        compression: Optional[str] = None,
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` in Parquet format at the specified path.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+        partitionBy : str or list, optional
+            names of partitioning columns
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#data-source-option>`_
+            in the version you use.
+
+            .. # noqa
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.parquet(d, mode="overwrite")
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.format("parquet").load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        self.option("compression", compression)
+        self.format("parquet").save(path)
+
+    def text(
+        self, path: str, compression: Optional[str] = None, lineSep: Optional[str] = None
+    ) -> None:
+        """Saves the content of the DataFrame in a text file at the specified path.
+        The text files will be encoded as UTF-8.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-text.html#data-source-option>`_
+            in the version you use.

Review Comment:
   done.



##########
python/pyspark/sql/connect/writer.py:
##########
@@ -0,0 +1,926 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+
+from typing import Optional, Union, List, overload, Tuple, cast, Any
+from typing import TYPE_CHECKING
+
+from pyspark.sql.connect.plan import WriteOperation, LogicalPlan
+from pyspark.sql.types import StructType
+from pyspark.sql.utils import to_str
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect._typing import OptionalPrimitiveType
+    from pyspark.sql.connect.session import SparkSession
+
+
+PathOrPaths = Union[str, List[str]]
+TupleOrListOfString = Union[List[str], Tuple[str, ...]]
+
+
+class OptionUtils:
+    def _set_opts(
+        self,
+        schema: Optional[Union[StructType, str]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """
+        Set named options (filter out those the value is None)
+        """
+        if schema is not None:
+            self.schema(schema)  # type: ignore[attr-defined]
+        for k, v in options.items():
+            if v is not None:
+                self.option(k, v)  # type: ignore[attr-defined]
+
+
+class DataFrameWriter(OptionUtils):
+    """
+    Interface used to write a :class:`DataFrame` to external storage systems
+    (e.g. file systems, key-value stores, etc). Use :attr:`DataFrame.write`
+    to access this.
+
+    .. versionadded:: 3.4.0
+    """
+
+    def __init__(self, plan: "LogicalPlan", session: "SparkSession"):
+        self._df: "LogicalPlan" = plan
+        self._spark: "SparkSession" = session
+        self._write: "WriteOperation" = WriteOperation(self._df)
+
+    def mode(self, saveMode: Optional[str]) -> "DataFrameWriter":
+        """Specifies the behavior when data or table already exists.
+
+        Options include:
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Examples
+        --------
+        Raise an error when writing to an existing path.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     spark.createDataFrame(
+        ...         [{"age": 80, "name": "Xinrong Meng"}]
+        ...     ).write.mode("error").format("parquet").save(d)
+        Traceback (most recent call last):
+            ...
+        pyspark.sql.utils.AnalysisException: ...
+
+        Write a Parquet file back with various options, and read it back.
+
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Overwrite the path with a new Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 120, "name": "Takuya Ueshin"}]
+        ...     ).write.mode("append").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 140, "name": "Haejoon Lee"}]
+        ...     ).write.mode("ignore").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |120|Takuya Ueshin|
+        |100| Hyukjin Kwon|
+        +---+-------------+
+        """
+        # At the JVM side, the default value of mode is already set to "error".
+        # So, if the given saveMode is None, we will not call JVM-side's mode method.
+        if saveMode is not None:
+            self._write.mode = saveMode
+        return self
+
+    def format(self, source: str) -> "DataFrameWriter":
+        """Specifies the underlying output data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        source : str
+            string, name of the data source, e.g. 'json', 'parquet'.
+
+        Examples
+        --------
+        >>> spark.range(1).write.format('parquet')
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Write a DataFrame into a Parquet file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.format('parquet').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.source = source
+        return self
+
+    def option(self, key: str, value: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds an output option for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        key : str
+            The key for the option to set.
+        value
+            The value for the option to set.
+
+        Examples
+        --------
+        >>> spark.range(1).write.option("key", "value")
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Specify the option 'nullValue' with writing a CSV file.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file with 'nullValue' option set to 'Hyukjin Kwon'.
+        ...     df = spark.createDataFrame([(100, None)], "age INT, name STRING")
+        ...     df.write.option("nullValue", "Hyukjin Kwon").mode("overwrite").format("csv").save(d)
+        ...
+        ...     # Read the CSV file as a DataFrame.
+        ...     spark.read.schema(df.schema).format('csv').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.options[key] = to_str(value)
+        return self
+
+    def options(self, **options: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds output options for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        **options : dict
+            The dictionary of string keys and primitive-type values.
+
+        Examples
+        --------
+        >>> spark.range(1).write.option("key", "value")
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Specify the option 'nullValue' and 'header' with writing a CSV file.
+        >>> from pyspark.sql.types import StructType,StructField, StringType, IntegerType
+        >>> schema = StructType([
+        ...     StructField("age",IntegerType(),True),
+        ...     StructField("name",StringType(),True),
+        ...])
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file with 'nullValue' option set to 'Hyukjin Kwon',
+        ...     # and 'header' option set to `True`.
+        ...     df = spark.createDataFrame([(100, None], schema=schema)
+        ...     df.write.options(nullValue="Hyukjin Kwon", header=True).mode(
+        ...         "overwrite").format("csv").save(d)
+        ...
+        ...     # Read the CSV file as a DataFrame.
+        ...     spark.read.option("header", True).format('csv').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        for k in options:
+            self._write.options[k] = to_str(options[k])
+        return self
+
+    @overload
+    def partitionBy(self, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def partitionBy(self, *cols: List[str]) -> "DataFrameWriter":
+        ...
+
+    def partitionBy(self, *cols: Union[str, List[str]]) -> "DataFrameWriter":
+        """Partitions the output by the given columns on the file system.
+
+        If specified, the output is laid out on the file system similar
+        to Hive's partitioning scheme.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        cols : str or list
+            name of columns
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a partitioned manner, and read it back.
+
+        >>> import tempfile
+        >>> import os
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file in a partitioned manner.
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}, {"age": 120, "name": "Ruifeng Zheng"}]
+        ...     ).write.partitionBy("name").mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).sort("age").show()
+        ...
+        ...     # Read one partition as a DataFrame.
+        ...     spark.read.parquet(f"{d}{os.path.sep}name=Hyukjin Kwon").show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |100| Hyukjin Kwon|
+        |120|Ruifeng Zheng|
+        +---+-------------+
+        +---+
+        |age|
+        +---+
+        |100|
+        +---+
+        """
+        if len(cols) == 1 and isinstance(cols[0], (list, tuple)):
+            cols = cols[0]  # type: ignore[assignment]
+
+        self._write.partitioning_cols = cast(List[str], cols)
+        return self
+
+    @overload
+    def bucketBy(self, numBuckets: int, col: str, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def bucketBy(self, numBuckets: int, col: TupleOrListOfString) -> "DataFrameWriter":
+        ...
+
+    def bucketBy(
+        self, numBuckets: int, col: Union[str, TupleOrListOfString], *cols: Optional[str]
+    ) -> "DataFrameWriter":
+        """Buckets the output by the given columns. If specified,
+        the output is laid out on the file system similar to Hive's bucketing scheme,
+        but with a different bucket hash function and is not compatible with Hive's bucketing.
+
+        .. versionadded:: 2.3.0
+
+        Parameters
+        ----------
+        numBuckets : int
+            the number of buckets to save
+        col : str, list or tuple
+            a name of a column, or a list of names.
+        cols : str
+            additional names (optional). If `col` is a list it should be empty.
+
+        Notes
+        -----
+        Applicable for file-based data sources in combination with
+        :py:meth:`DataFrameWriter.saveAsTable`.
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a buckted manner, and read it back.
+
+        >>> from pyspark.sql.functions import input_file_name
+        >>> # Write a DataFrame into a Parquet file in a bucketed manner.
+        ... _ = spark.sql("DROP TABLE IF EXISTS bucketed_table")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.bucketBy(2, "name").mode("overwrite").saveAsTable("bucketed_table")
+        >>> # Read the Parquet file as a DataFrame.
+        ... spark.read.table("bucketed_table").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE bucketed_table")
+        """
+        if not isinstance(numBuckets, int):
+            raise TypeError("numBuckets should be an int, got {0}.".format(type(numBuckets)))
+
+        if isinstance(col, (list, tuple)):
+            if cols:
+                raise ValueError("col is a {0} but cols are not empty".format(type(col)))
+
+            col, cols = col[0], col[1:]  # type: ignore[assignment]
+
+        if not all(isinstance(c, str) for c in cols) or not (isinstance(col, str)):
+            raise TypeError("all names should be `str`")
+
+        self._write.num_buckets = numBuckets
+        self._write.bucket_cols = cast(List[str], cols)
+        return self
+
+    @overload
+    def sortBy(self, col: str, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def sortBy(self, col: TupleOrListOfString) -> "DataFrameWriter":
+        ...
+
+    def sortBy(
+        self, col: Union[str, TupleOrListOfString], *cols: Optional[str]
+    ) -> "DataFrameWriter":
+        """Sorts the output in each bucket by the given columns on the file system.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        col : str, tuple or list
+            a name of a column, or a list of names.
+        cols : str
+            additional names (optional). If `col` is a list it should be empty.
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a sorted-buckted manner, and read it back.
+
+        >>> from pyspark.sql.functions import input_file_name
+        >>> # Write a DataFrame into a Parquet file in a sorted-bucketed manner.
+        ... _ = spark.sql("DROP TABLE IF EXISTS sorted_bucketed_table")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.bucketBy(1, "name").sortBy("age").mode(
+        ...     "overwrite").saveAsTable("sorted_bucketed_table")
+        >>> # Read the Parquet file as a DataFrame.
+        ... spark.read.table("sorted_bucketed_table").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE sorted_bucketed_table")
+        """
+        if isinstance(col, (list, tuple)):
+            if cols:
+                raise ValueError("col is a {0} but cols are not empty".format(type(col)))
+
+            col, cols = col[0], col[1:]  # type: ignore[assignment]
+
+        if not all(isinstance(c, str) for c in cols) or not (isinstance(col, str)):
+            raise TypeError("all names should be `str`")
+
+        self._write.sort_cols = cast(List[str], cols)
+        return self
+
+    def save(
+        self,
+        path: Optional[str] = None,
+        format: Optional[str] = None,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """Saves the contents of the :class:`DataFrame` to a data source.
+
+        The data source is specified by the ``format`` and a set of ``options``.
+        If ``format`` is not specified, the default data source configured by
+        ``spark.sql.sources.default`` will be used.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str, optional
+            the path in a Hadoop supported file system
+        format : str, optional
+            the format used to save
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+        partitionBy : list, optional
+            names of partitioning columns
+        **options : dict
+            all other string options
+
+        Examples
+        --------
+        Write a DataFrame into a JSON file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a JSON file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("json").save(d)
+        ...
+        ...     # Read the JSON file as a DataFrame.
+        ...     spark.read.format('json').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode).options(**options)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        if format is not None:
+            self.format(format)
+        self._write.path = path
+        self._spark.client.execute_command(self._write.command(self._spark.client))
+
+    def insertInto(self, tableName: str, overwrite: Optional[bool] = None) -> None:
+        """Inserts the content of the :class:`DataFrame` to the specified table.
+
+        It requires that the schema of the :class:`DataFrame` is the same as the
+        schema of the table.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        overwrite : bool, optional
+            If true, overwrites existing data. Disabled by default
+
+        Notes
+        -----
+        Unlike :meth:`DataFrameWriter.saveAsTable`, :meth:`DataFrameWriter.insertInto` ignores
+        the column names and just uses position-based resolution.
+
+        Examples
+        --------
+        >>> _ = spark.sql("DROP TABLE IF EXISTS tblA")
+        >>> df = spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... )
+        >>> df.write.saveAsTable("tblA")
+
+        Insert the data into 'tblA' table but with different column names.
+
+        >>> df.selectExpr("age AS col1", "name AS col2").write.insertInto("tblA")
+        >>> spark.read.table("tblA").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE tblA")
+        """
+        if overwrite is not None:
+            self.mode("overwrite" if overwrite else "append")
+        self.saveAsTable(tableName)
+
+    def saveAsTable(
+        self,
+        name: str,
+        format: Optional[str] = None,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` as the specified table.
+
+        In the case the table already exists, behavior of this function depends on the
+        save mode, specified by the `mode` function (default to throwing an exception).
+        When `mode` is `Overwrite`, the schema of the :class:`DataFrame` does not need to be
+        the same as that of the existing table.
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Notes
+        -----
+        When `mode` is `Append`, if there is an existing table, we will use the format and
+        options of the existing table. The column order in the schema of the :class:`DataFrame`
+        doesn't need to be the same as that of the existing table. Unlike
+        :meth:`DataFrameWriter.insertInto`, :meth:`DataFrameWriter.saveAsTable` will use the
+        column names to find the correct column positions.
+
+        Parameters
+        ----------
+        name : str
+            the table name
+        format : str, optional
+            the format used to save
+        mode : str, optional
+            one of `append`, `overwrite`, `error`, `errorifexists`, `ignore` \
+            (default: error)
+        partitionBy : str or list
+            names of partitioning columns
+        **options : dict
+            all other string options
+
+        Examples
+        --------
+        Creates a table from a DataFrame, and read it back.
+
+        >>> _ = spark.sql("DROP TABLE IF EXISTS tblA")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.saveAsTable("tblA")
+        >>> spark.read.table("tblA").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE tblA")
+        """
+        self.mode(mode).options(**options)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        if format is not None:
+            self.format(format)
+        self._write.table_name = name
+        self._spark.client.execute_command(self._write.command(self._spark.client))
+
+    def json(
+        self,
+        path: str,
+        mode: Optional[str] = None,
+        compression: Optional[str] = None,
+        dateFormat: Optional[str] = None,
+        timestampFormat: Optional[str] = None,
+        lineSep: Optional[str] = None,
+        encoding: Optional[str] = None,
+        ignoreNullFields: Optional[Union[bool, str]] = None,
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` in JSON format
+        (`JSON Lines text format or newline-delimited JSON <http://jsonlines.org/>`_) at the
+        specified path.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-json.html#data-source-option>`_
+            for the version you use.
+
+            .. # noqa
+
+        Examples
+        --------
+        Write a DataFrame into a JSON file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a JSON file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.json(d, mode="overwrite")
+        ...
+        ...     # Read the JSON file as a DataFrame.
+        ...     spark.read.format("json").load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode)
+        self._set_opts(
+            compression=compression,
+            dateFormat=dateFormat,
+            timestampFormat=timestampFormat,
+            lineSep=lineSep,
+            encoding=encoding,
+            ignoreNullFields=ignoreNullFields,
+        )
+        self.format("json").save(path)
+
+    def parquet(
+        self,
+        path: str,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        compression: Optional[str] = None,
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` in Parquet format at the specified path.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+        partitionBy : str or list, optional
+            names of partitioning columns
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#data-source-option>`_
+            in the version you use.
+
+            .. # noqa
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.parquet(d, mode="overwrite")
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.format("parquet").load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        self.option("compression", compression)
+        self.format("parquet").save(path)
+
+    def text(
+        self, path: str, compression: Optional[str] = None, lineSep: Optional[str] = None
+    ) -> None:
+        """Saves the content of the DataFrame in a text file at the specified path.
+        The text files will be encoded as UTF-8.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-text.html#data-source-option>`_
+            in the version you use.
+
+            .. # noqa
+
+        Notes
+        -----
+        The DataFrame must have only one column that is of string type.
+        Each row becomes a new line in the output file.
+
+        Examples
+        --------
+        Write a DataFrame into a text file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a text file
+        ...     df = spark.createDataFrame([("a",), ("b",), ("c",)], schema=["alphabets"])
+        ...     df.write.mode("overwrite").text(d)
+        ...
+        ...     # Read the text file as a DataFrame.
+        ...     spark.read.schema(df.schema).format("text").load(d).sort("alphabets").show()
+        +---------+
+        |alphabets|
+        +---------+
+        |        a|
+        |        b|
+        |        c|
+        +---------+
+        """
+        self._set_opts(compression=compression, lineSep=lineSep)
+        self.format("text").save(path)
+
+    def csv(
+        self,
+        path: str,
+        mode: Optional[str] = None,
+        compression: Optional[str] = None,
+        sep: Optional[str] = None,
+        quote: Optional[str] = None,
+        escape: Optional[str] = None,
+        header: Optional[Union[bool, str]] = None,
+        nullValue: Optional[str] = None,
+        escapeQuotes: Optional[Union[bool, str]] = None,
+        quoteAll: Optional[Union[bool, str]] = None,
+        dateFormat: Optional[str] = None,
+        timestampFormat: Optional[str] = None,
+        ignoreLeadingWhiteSpace: Optional[Union[bool, str]] = None,
+        ignoreTrailingWhiteSpace: Optional[Union[bool, str]] = None,
+        charToEscapeQuoteEscaping: Optional[str] = None,
+        encoding: Optional[str] = None,
+        emptyValue: Optional[str] = None,
+        lineSep: Optional[str] = None,
+    ) -> None:
+        r"""Saves the content of the :class:`DataFrame` in CSV format at the specified path.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-csv.html#data-source-option>`_
+            in the version you use.
+
+            .. # noqa
+
+        Examples
+        --------
+        Write a DataFrame into a CSV file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file
+        ...     df = spark.createDataFrame([{"age": 100, "name": "Hyukjin Kwon"}])
+        ...     df.write.csv(d, mode="overwrite")
+        ...
+        ...     # Read the CSV file as a DataFrame with 'nullValue' option set to 'Hyukjin Kwon'.
+        ...     spark.read.schema(df.schema).format("csv").option(
+        ...         "nullValue", "Hyukjin Kwon").load(d).show()
+        +---+----+
+        |age|name|
+        +---+----+
+        |100|null|
+        +---+----+
+        """
+        self.mode(mode)
+        self._set_opts(
+            compression=compression,
+            sep=sep,
+            quote=quote,
+            escape=escape,
+            header=header,
+            nullValue=nullValue,
+            escapeQuotes=escapeQuotes,
+            quoteAll=quoteAll,
+            dateFormat=dateFormat,
+            timestampFormat=timestampFormat,
+            ignoreLeadingWhiteSpace=ignoreLeadingWhiteSpace,
+            ignoreTrailingWhiteSpace=ignoreTrailingWhiteSpace,
+            charToEscapeQuoteEscaping=charToEscapeQuoteEscaping,
+            encoding=encoding,
+            emptyValue=emptyValue,
+            lineSep=lineSep,
+        )
+        self.format("csv").save(path)
+
+    def orc(
+        self,
+        path: str,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        compression: Optional[str] = None,
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` in ORC format at the specified path.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+        partitionBy : str or list, optional
+            names of partitioning columns
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-orc.html#data-source-option>`_
+            in the version you use.

Review Comment:
   done



##########
python/pyspark/sql/connect/writer.py:
##########
@@ -0,0 +1,926 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+
+from typing import Optional, Union, List, overload, Tuple, cast, Any
+from typing import TYPE_CHECKING
+
+from pyspark.sql.connect.plan import WriteOperation, LogicalPlan
+from pyspark.sql.types import StructType
+from pyspark.sql.utils import to_str
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect._typing import OptionalPrimitiveType
+    from pyspark.sql.connect.session import SparkSession
+
+
+PathOrPaths = Union[str, List[str]]
+TupleOrListOfString = Union[List[str], Tuple[str, ...]]
+
+
+class OptionUtils:
+    def _set_opts(
+        self,
+        schema: Optional[Union[StructType, str]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """
+        Set named options (filter out those the value is None)
+        """
+        if schema is not None:
+            self.schema(schema)  # type: ignore[attr-defined]
+        for k, v in options.items():
+            if v is not None:
+                self.option(k, v)  # type: ignore[attr-defined]
+
+
+class DataFrameWriter(OptionUtils):
+    """
+    Interface used to write a :class:`DataFrame` to external storage systems
+    (e.g. file systems, key-value stores, etc). Use :attr:`DataFrame.write`
+    to access this.
+
+    .. versionadded:: 3.4.0
+    """
+
+    def __init__(self, plan: "LogicalPlan", session: "SparkSession"):
+        self._df: "LogicalPlan" = plan
+        self._spark: "SparkSession" = session
+        self._write: "WriteOperation" = WriteOperation(self._df)
+
+    def mode(self, saveMode: Optional[str]) -> "DataFrameWriter":
+        """Specifies the behavior when data or table already exists.
+
+        Options include:
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Examples
+        --------
+        Raise an error when writing to an existing path.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     spark.createDataFrame(
+        ...         [{"age": 80, "name": "Xinrong Meng"}]
+        ...     ).write.mode("error").format("parquet").save(d)
+        Traceback (most recent call last):
+            ...
+        pyspark.sql.utils.AnalysisException: ...
+
+        Write a Parquet file back with various options, and read it back.
+
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Overwrite the path with a new Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 120, "name": "Takuya Ueshin"}]
+        ...     ).write.mode("append").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 140, "name": "Haejoon Lee"}]
+        ...     ).write.mode("ignore").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |120|Takuya Ueshin|
+        |100| Hyukjin Kwon|
+        +---+-------------+
+        """
+        # At the JVM side, the default value of mode is already set to "error".
+        # So, if the given saveMode is None, we will not call JVM-side's mode method.
+        if saveMode is not None:
+            self._write.mode = saveMode
+        return self
+
+    def format(self, source: str) -> "DataFrameWriter":
+        """Specifies the underlying output data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        source : str
+            string, name of the data source, e.g. 'json', 'parquet'.
+
+        Examples
+        --------
+        >>> spark.range(1).write.format('parquet')
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Write a DataFrame into a Parquet file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.format('parquet').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.source = source
+        return self
+
+    def option(self, key: str, value: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds an output option for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        key : str
+            The key for the option to set.
+        value
+            The value for the option to set.
+
+        Examples
+        --------
+        >>> spark.range(1).write.option("key", "value")
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Specify the option 'nullValue' with writing a CSV file.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file with 'nullValue' option set to 'Hyukjin Kwon'.
+        ...     df = spark.createDataFrame([(100, None)], "age INT, name STRING")
+        ...     df.write.option("nullValue", "Hyukjin Kwon").mode("overwrite").format("csv").save(d)
+        ...
+        ...     # Read the CSV file as a DataFrame.
+        ...     spark.read.schema(df.schema).format('csv').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.options[key] = to_str(value)
+        return self
+
+    def options(self, **options: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds output options for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        **options : dict
+            The dictionary of string keys and primitive-type values.
+
+        Examples
+        --------
+        >>> spark.range(1).write.option("key", "value")
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Specify the option 'nullValue' and 'header' with writing a CSV file.
+        >>> from pyspark.sql.types import StructType,StructField, StringType, IntegerType
+        >>> schema = StructType([
+        ...     StructField("age",IntegerType(),True),
+        ...     StructField("name",StringType(),True),
+        ...])
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file with 'nullValue' option set to 'Hyukjin Kwon',
+        ...     # and 'header' option set to `True`.
+        ...     df = spark.createDataFrame([(100, None], schema=schema)
+        ...     df.write.options(nullValue="Hyukjin Kwon", header=True).mode(
+        ...         "overwrite").format("csv").save(d)
+        ...
+        ...     # Read the CSV file as a DataFrame.
+        ...     spark.read.option("header", True).format('csv').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        for k in options:
+            self._write.options[k] = to_str(options[k])
+        return self
+
+    @overload
+    def partitionBy(self, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def partitionBy(self, *cols: List[str]) -> "DataFrameWriter":
+        ...
+
+    def partitionBy(self, *cols: Union[str, List[str]]) -> "DataFrameWriter":
+        """Partitions the output by the given columns on the file system.
+
+        If specified, the output is laid out on the file system similar
+        to Hive's partitioning scheme.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        cols : str or list
+            name of columns
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a partitioned manner, and read it back.
+
+        >>> import tempfile
+        >>> import os
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file in a partitioned manner.
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}, {"age": 120, "name": "Ruifeng Zheng"}]
+        ...     ).write.partitionBy("name").mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).sort("age").show()
+        ...
+        ...     # Read one partition as a DataFrame.
+        ...     spark.read.parquet(f"{d}{os.path.sep}name=Hyukjin Kwon").show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |100| Hyukjin Kwon|
+        |120|Ruifeng Zheng|
+        +---+-------------+
+        +---+
+        |age|
+        +---+
+        |100|
+        +---+
+        """
+        if len(cols) == 1 and isinstance(cols[0], (list, tuple)):
+            cols = cols[0]  # type: ignore[assignment]
+
+        self._write.partitioning_cols = cast(List[str], cols)
+        return self
+
+    @overload
+    def bucketBy(self, numBuckets: int, col: str, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def bucketBy(self, numBuckets: int, col: TupleOrListOfString) -> "DataFrameWriter":
+        ...
+
+    def bucketBy(
+        self, numBuckets: int, col: Union[str, TupleOrListOfString], *cols: Optional[str]
+    ) -> "DataFrameWriter":
+        """Buckets the output by the given columns. If specified,
+        the output is laid out on the file system similar to Hive's bucketing scheme,
+        but with a different bucket hash function and is not compatible with Hive's bucketing.
+
+        .. versionadded:: 2.3.0
+
+        Parameters
+        ----------
+        numBuckets : int
+            the number of buckets to save
+        col : str, list or tuple
+            a name of a column, or a list of names.
+        cols : str
+            additional names (optional). If `col` is a list it should be empty.
+
+        Notes
+        -----
+        Applicable for file-based data sources in combination with
+        :py:meth:`DataFrameWriter.saveAsTable`.
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a buckted manner, and read it back.
+
+        >>> from pyspark.sql.functions import input_file_name
+        >>> # Write a DataFrame into a Parquet file in a bucketed manner.
+        ... _ = spark.sql("DROP TABLE IF EXISTS bucketed_table")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.bucketBy(2, "name").mode("overwrite").saveAsTable("bucketed_table")
+        >>> # Read the Parquet file as a DataFrame.
+        ... spark.read.table("bucketed_table").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE bucketed_table")
+        """
+        if not isinstance(numBuckets, int):
+            raise TypeError("numBuckets should be an int, got {0}.".format(type(numBuckets)))
+
+        if isinstance(col, (list, tuple)):
+            if cols:
+                raise ValueError("col is a {0} but cols are not empty".format(type(col)))
+
+            col, cols = col[0], col[1:]  # type: ignore[assignment]
+
+        if not all(isinstance(c, str) for c in cols) or not (isinstance(col, str)):
+            raise TypeError("all names should be `str`")
+
+        self._write.num_buckets = numBuckets
+        self._write.bucket_cols = cast(List[str], cols)
+        return self
+
+    @overload
+    def sortBy(self, col: str, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def sortBy(self, col: TupleOrListOfString) -> "DataFrameWriter":
+        ...
+
+    def sortBy(
+        self, col: Union[str, TupleOrListOfString], *cols: Optional[str]
+    ) -> "DataFrameWriter":
+        """Sorts the output in each bucket by the given columns on the file system.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        col : str, tuple or list
+            a name of a column, or a list of names.
+        cols : str
+            additional names (optional). If `col` is a list it should be empty.
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a sorted-buckted manner, and read it back.
+
+        >>> from pyspark.sql.functions import input_file_name
+        >>> # Write a DataFrame into a Parquet file in a sorted-bucketed manner.
+        ... _ = spark.sql("DROP TABLE IF EXISTS sorted_bucketed_table")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.bucketBy(1, "name").sortBy("age").mode(
+        ...     "overwrite").saveAsTable("sorted_bucketed_table")
+        >>> # Read the Parquet file as a DataFrame.
+        ... spark.read.table("sorted_bucketed_table").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE sorted_bucketed_table")
+        """
+        if isinstance(col, (list, tuple)):
+            if cols:
+                raise ValueError("col is a {0} but cols are not empty".format(type(col)))
+
+            col, cols = col[0], col[1:]  # type: ignore[assignment]
+
+        if not all(isinstance(c, str) for c in cols) or not (isinstance(col, str)):
+            raise TypeError("all names should be `str`")
+
+        self._write.sort_cols = cast(List[str], cols)
+        return self
+
+    def save(
+        self,
+        path: Optional[str] = None,
+        format: Optional[str] = None,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """Saves the contents of the :class:`DataFrame` to a data source.
+
+        The data source is specified by the ``format`` and a set of ``options``.
+        If ``format`` is not specified, the default data source configured by
+        ``spark.sql.sources.default`` will be used.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str, optional
+            the path in a Hadoop supported file system
+        format : str, optional
+            the format used to save
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+        partitionBy : list, optional
+            names of partitioning columns
+        **options : dict
+            all other string options
+
+        Examples
+        --------
+        Write a DataFrame into a JSON file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a JSON file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("json").save(d)
+        ...
+        ...     # Read the JSON file as a DataFrame.
+        ...     spark.read.format('json').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode).options(**options)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        if format is not None:
+            self.format(format)
+        self._write.path = path
+        self._spark.client.execute_command(self._write.command(self._spark.client))
+
+    def insertInto(self, tableName: str, overwrite: Optional[bool] = None) -> None:
+        """Inserts the content of the :class:`DataFrame` to the specified table.
+
+        It requires that the schema of the :class:`DataFrame` is the same as the
+        schema of the table.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        overwrite : bool, optional
+            If true, overwrites existing data. Disabled by default
+
+        Notes
+        -----
+        Unlike :meth:`DataFrameWriter.saveAsTable`, :meth:`DataFrameWriter.insertInto` ignores
+        the column names and just uses position-based resolution.
+
+        Examples
+        --------
+        >>> _ = spark.sql("DROP TABLE IF EXISTS tblA")
+        >>> df = spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... )
+        >>> df.write.saveAsTable("tblA")
+
+        Insert the data into 'tblA' table but with different column names.
+
+        >>> df.selectExpr("age AS col1", "name AS col2").write.insertInto("tblA")
+        >>> spark.read.table("tblA").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE tblA")
+        """
+        if overwrite is not None:
+            self.mode("overwrite" if overwrite else "append")
+        self.saveAsTable(tableName)
+
+    def saveAsTable(
+        self,
+        name: str,
+        format: Optional[str] = None,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` as the specified table.
+
+        In the case the table already exists, behavior of this function depends on the
+        save mode, specified by the `mode` function (default to throwing an exception).
+        When `mode` is `Overwrite`, the schema of the :class:`DataFrame` does not need to be
+        the same as that of the existing table.
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Notes
+        -----
+        When `mode` is `Append`, if there is an existing table, we will use the format and
+        options of the existing table. The column order in the schema of the :class:`DataFrame`
+        doesn't need to be the same as that of the existing table. Unlike
+        :meth:`DataFrameWriter.insertInto`, :meth:`DataFrameWriter.saveAsTable` will use the
+        column names to find the correct column positions.
+
+        Parameters
+        ----------
+        name : str
+            the table name
+        format : str, optional
+            the format used to save
+        mode : str, optional
+            one of `append`, `overwrite`, `error`, `errorifexists`, `ignore` \
+            (default: error)
+        partitionBy : str or list
+            names of partitioning columns
+        **options : dict
+            all other string options
+
+        Examples
+        --------
+        Creates a table from a DataFrame, and read it back.
+
+        >>> _ = spark.sql("DROP TABLE IF EXISTS tblA")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.saveAsTable("tblA")
+        >>> spark.read.table("tblA").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE tblA")
+        """
+        self.mode(mode).options(**options)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        if format is not None:
+            self.format(format)
+        self._write.table_name = name
+        self._spark.client.execute_command(self._write.command(self._spark.client))
+
+    def json(
+        self,
+        path: str,
+        mode: Optional[str] = None,
+        compression: Optional[str] = None,
+        dateFormat: Optional[str] = None,
+        timestampFormat: Optional[str] = None,
+        lineSep: Optional[str] = None,
+        encoding: Optional[str] = None,
+        ignoreNullFields: Optional[Union[bool, str]] = None,
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` in JSON format
+        (`JSON Lines text format or newline-delimited JSON <http://jsonlines.org/>`_) at the
+        specified path.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-json.html#data-source-option>`_
+            for the version you use.
+
+            .. # noqa
+
+        Examples
+        --------
+        Write a DataFrame into a JSON file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a JSON file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.json(d, mode="overwrite")
+        ...
+        ...     # Read the JSON file as a DataFrame.
+        ...     spark.read.format("json").load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode)
+        self._set_opts(
+            compression=compression,
+            dateFormat=dateFormat,
+            timestampFormat=timestampFormat,
+            lineSep=lineSep,
+            encoding=encoding,
+            ignoreNullFields=ignoreNullFields,
+        )
+        self.format("json").save(path)
+
+    def parquet(
+        self,
+        path: str,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        compression: Optional[str] = None,
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` in Parquet format at the specified path.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+        partitionBy : str or list, optional
+            names of partitioning columns
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#data-source-option>`_
+            in the version you use.
+
+            .. # noqa
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.parquet(d, mode="overwrite")
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.format("parquet").load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        self.option("compression", compression)
+        self.format("parquet").save(path)
+
+    def text(
+        self, path: str, compression: Optional[str] = None, lineSep: Optional[str] = None
+    ) -> None:
+        """Saves the content of the DataFrame in a text file at the specified path.
+        The text files will be encoded as UTF-8.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-text.html#data-source-option>`_
+            in the version you use.
+
+            .. # noqa
+
+        Notes
+        -----
+        The DataFrame must have only one column that is of string type.
+        Each row becomes a new line in the output file.
+
+        Examples
+        --------
+        Write a DataFrame into a text file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a text file
+        ...     df = spark.createDataFrame([("a",), ("b",), ("c",)], schema=["alphabets"])
+        ...     df.write.mode("overwrite").text(d)
+        ...
+        ...     # Read the text file as a DataFrame.
+        ...     spark.read.schema(df.schema).format("text").load(d).sort("alphabets").show()
+        +---------+
+        |alphabets|
+        +---------+
+        |        a|
+        |        b|
+        |        c|
+        +---------+
+        """
+        self._set_opts(compression=compression, lineSep=lineSep)
+        self.format("text").save(path)
+
+    def csv(
+        self,
+        path: str,
+        mode: Optional[str] = None,
+        compression: Optional[str] = None,
+        sep: Optional[str] = None,
+        quote: Optional[str] = None,
+        escape: Optional[str] = None,
+        header: Optional[Union[bool, str]] = None,
+        nullValue: Optional[str] = None,
+        escapeQuotes: Optional[Union[bool, str]] = None,
+        quoteAll: Optional[Union[bool, str]] = None,
+        dateFormat: Optional[str] = None,
+        timestampFormat: Optional[str] = None,
+        ignoreLeadingWhiteSpace: Optional[Union[bool, str]] = None,
+        ignoreTrailingWhiteSpace: Optional[Union[bool, str]] = None,
+        charToEscapeQuoteEscaping: Optional[str] = None,
+        encoding: Optional[str] = None,
+        emptyValue: Optional[str] = None,
+        lineSep: Optional[str] = None,
+    ) -> None:
+        r"""Saves the content of the :class:`DataFrame` in CSV format at the specified path.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-csv.html#data-source-option>`_
+            in the version you use.

Review Comment:
   DOne



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] hvanhovell commented on a diff in pull request #38801: [SPARK-40539] [CONNECT] [PYTHON] Add basic support for DataFrameWriter

Posted by GitBox <gi...@apache.org>.
hvanhovell commented on code in PR #38801:
URL: https://github.com/apache/spark/pull/38801#discussion_r1032556096


##########
python/pyspark/sql/connect/writer.py:
##########
@@ -0,0 +1,922 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+
+from typing import Optional, Union, List, overload, Tuple, cast, Any
+from typing import TYPE_CHECKING
+
+from pyspark.sql.connect.plan import WriteOperation, LogicalPlan
+from pyspark.sql.types import StructType
+from pyspark.sql.utils import to_str
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect._typing import OptionalPrimitiveType
+    from pyspark.sql.connect.client import RemoteSparkSession
+
+
+PathOrPaths = Union[str, List[str]]
+TupleOrListOfString = Union[List[str], Tuple[str, ...]]
+
+
+class OptionUtils:
+    def _set_opts(
+        self,
+        schema: Optional[Union[StructType, str]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """
+        Set named options (filter out those the value is None)
+        """
+        if schema is not None:
+            self.schema(schema)  # type: ignore[attr-defined]
+        for k, v in options.items():
+            if v is not None:
+                self.option(k, v)  # type: ignore[attr-defined]
+
+
+class DataFrameWriter(OptionUtils):
+    """
+    Interface used to write a :class:`DataFrame` to external storage systems
+    (e.g. file systems, key-value stores, etc). Use :attr:`DataFrame.write`
+    to access this.
+
+    .. versionadded:: 3.4.0
+    """
+
+    def __init__(self, plan: "LogicalPlan", session: "RemoteSparkSession"):
+        self._df: "LogicalPlan" = plan
+        self._spark: "RemoteSparkSession" = session
+        self._write: "WriteOperation" = WriteOperation(self._df)
+
+    def mode(self, saveMode: Optional[str]) -> "DataFrameWriter":
+        """Specifies the behavior when data or table already exists.
+
+        Options include:
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Examples
+        --------
+        Raise an error when writing to an existing path.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     spark.createDataFrame(
+        ...         [{"age": 80, "name": "Xinrong Meng"}]
+        ...     ).write.mode("error").format("parquet").save(d)
+        Traceback (most recent call last):
+            ...
+        pyspark.sql.utils.AnalysisException: ...
+
+        Write a Parquet file back with various options, and read it back.
+
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Overwrite the path with a new Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 120, "name": "Takuya Ueshin"}]
+        ...     ).write.mode("append").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 140, "name": "Haejoon Lee"}]
+        ...     ).write.mode("ignore").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |120|Takuya Ueshin|
+        |100| Hyukjin Kwon|
+        +---+-------------+
+        """
+        # At the JVM side, the default value of mode is already set to "error".
+        # So, if the given saveMode is None, we will not call JVM-side's mode method.
+        if saveMode is not None:
+            self._write.mode = saveMode
+        return self
+
+    def format(self, source: str) -> "DataFrameWriter":
+        """Specifies the underlying output data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        source : str
+            string, name of the data source, e.g. 'json', 'parquet'.
+
+        Examples
+        --------
+        >>> spark.range(1).write.format('parquet')
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Write a DataFrame into a Parquet file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.format('parquet').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.source = source
+        return self
+
+    def option(self, key: str, value: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds a output option for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        key : str
+            The key for the option to set.
+        value
+            The value for the option to set.
+
+        Examples
+        --------
+        >>> spark.range(1).write.option("key", "value")
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Specify the option 'nullValue' with writing a CSV file.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file with 'nullValue' option set to 'Hyukjin Kwon'.
+        ...     df = spark.createDataFrame([(100, None)], "age INT, name STRING")
+        ...     df.write.option("nullValue", "Hyukjin Kwon").mode("overwrite").format("csv").save(d)
+        ...
+        ...     # Read the CSV file as a DataFrame.
+        ...     spark.read.schema(df.schema).format('csv').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.options[key] = to_str(value)
+        return self
+
+    def options(self, **options: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds output options for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        **options : dict
+            The dictionary of string keys and prmitive-type values.
+
+        Examples
+        --------
+        >>> spark.range(1).write.option("key", "value")
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Specify the option 'nullValue' and 'header' with writing a CSV file.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file with 'nullValue' option set to 'Hyukjin Kwon',
+        ...     # and 'header' option set to `True`.
+        ...     df = spark.createDataFrame([(100, "Hyukjin Kwon")], ["age", "name"])
+        ...     df.write.options(nullValue="Hyukjin Kwon", header=True).mode(
+        ...         "overwrite").format("csv").save(d)
+        ...
+        ...     # Read the CSV file as a DataFrame.
+        ...     spark.read.option("header", True).format('csv').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        for k in options:
+            self._write.options[k] = to_str(options[k])
+        return self
+
+    @overload
+    def partitionBy(self, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def partitionBy(self, *cols: List[str]) -> "DataFrameWriter":
+        ...
+
+    def partitionBy(self, *cols: Union[str, List[str]]) -> "DataFrameWriter":
+        """Partitions the output by the given columns on the file system.
+
+        If specified, the output is laid out on the file system similar
+        to Hive's partitioning scheme.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        cols : str or list
+            name of columns
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a partitioned manner, and read it back.
+
+        >>> import tempfile
+        >>> import os
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file in a partitioned manner.
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}, {"age": 120, "name": "Ruifeng Zheng"}]
+        ...     ).write.partitionBy("name").mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).sort("age").show()
+        ...
+        ...     # Read one partition as a DataFrame.
+        ...     spark.read.parquet(f"{d}{os.path.sep}name=Hyukjin Kwon").show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |100| Hyukjin Kwon|
+        |120|Ruifeng Zheng|
+        +---+-------------+
+        +---+
+        |age|
+        +---+
+        |100|
+        +---+
+        """
+        if len(cols) == 1 and isinstance(cols[0], (list, tuple)):
+            cols = cols[0]  # type: ignore[assignment]
+
+        self._write.partitioning_cols = cast(List[str], cols)
+        return self
+
+    @overload
+    def bucketBy(self, numBuckets: int, col: str, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def bucketBy(self, numBuckets: int, col: TupleOrListOfString) -> "DataFrameWriter":
+        ...
+
+    def bucketBy(
+        self, numBuckets: int, col: Union[str, TupleOrListOfString], *cols: Optional[str]
+    ) -> "DataFrameWriter":
+        """Buckets the output by the given columns. If specified,
+        the output is laid out on the file system similar to Hive's bucketing scheme,
+        but with a different bucket hash function and is not compatible with Hive's bucketing.
+
+        .. versionadded:: 2.3.0
+
+        Parameters
+        ----------
+        numBuckets : int
+            the number of buckets to save
+        col : str, list or tuple
+            a name of a column, or a list of names.
+        cols : str
+            additional names (optional). If `col` is a list it should be empty.
+
+        Notes
+        -----
+        Applicable for file-based data sources in combination with
+        :py:meth:`DataFrameWriter.saveAsTable`.
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a buckted manner, and read it back.
+
+        >>> from pyspark.sql.functions import input_file_name
+        >>> # Write a DataFrame into a Parquet file in a bucketed manner.
+        ... _ = spark.sql("DROP TABLE IF EXISTS bucketed_table")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.bucketBy(2, "name").mode("overwrite").saveAsTable("bucketed_table")
+        >>> # Read the Parquet file as a DataFrame.
+        ... spark.read.table("bucketed_table").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE bucketed_table")
+        """
+        if not isinstance(numBuckets, int):
+            raise TypeError("numBuckets should be an int, got {0}.".format(type(numBuckets)))
+
+        if isinstance(col, (list, tuple)):
+            if cols:
+                raise ValueError("col is a {0} but cols are not empty".format(type(col)))
+
+            col, cols = col[0], col[1:]  # type: ignore[assignment]
+
+        if not all(isinstance(c, str) for c in cols) or not (isinstance(col, str)):
+            raise TypeError("all names should be `str`")
+
+        self._write.num_buckets = numBuckets
+        self._write.bucket_cols = cast(List[str], cols)
+        return self
+
+    @overload
+    def sortBy(self, col: str, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def sortBy(self, col: TupleOrListOfString) -> "DataFrameWriter":
+        ...
+
+    def sortBy(
+        self, col: Union[str, TupleOrListOfString], *cols: Optional[str]
+    ) -> "DataFrameWriter":
+        """Sorts the output in each bucket by the given columns on the file system.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        col : str, tuple or list
+            a name of a column, or a list of names.
+        cols : str
+            additional names (optional). If `col` is a list it should be empty.
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a sorted-buckted manner, and read it back.
+
+        >>> from pyspark.sql.functions import input_file_name
+        >>> # Write a DataFrame into a Parquet file in a sorted-bucketed manner.
+        ... _ = spark.sql("DROP TABLE IF EXISTS sorted_bucketed_table")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.bucketBy(1, "name").sortBy("age").mode(
+        ...     "overwrite").saveAsTable("sorted_bucketed_table")
+        >>> # Read the Parquet file as a DataFrame.
+        ... spark.read.table("sorted_bucketed_table").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE sorted_bucketed_table")
+        """
+        if isinstance(col, (list, tuple)):
+            if cols:
+                raise ValueError("col is a {0} but cols are not empty".format(type(col)))
+
+            col, cols = col[0], col[1:]  # type: ignore[assignment]
+
+        if not all(isinstance(c, str) for c in cols) or not (isinstance(col, str)):
+            raise TypeError("all names should be `str`")
+
+        self._write.sort_cols = cast(List[str], cols)
+        return self
+
+    def save(
+        self,
+        path: Optional[str] = None,
+        format: Optional[str] = None,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """Saves the contents of the :class:`DataFrame` to a data source.
+
+        The data source is specified by the ``format`` and a set of ``options``.
+        If ``format`` is not specified, the default data source configured by
+        ``spark.sql.sources.default`` will be used.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str, optional
+            the path in a Hadoop supported file system
+        format : str, optional
+            the format used to save
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+        partitionBy : list, optional
+            names of partitioning columns
+        **options : dict
+            all other string options
+
+        Examples
+        --------
+        Write a DataFrame into a JSON file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a JSON file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("json").save(d)
+        ...
+        ...     # Read the JSON file as a DataFrame.
+        ...     spark.read.format('json').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode).options(**options)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        if format is not None:
+            self.format(format)
+        self._write.path = path
+        self._spark.execute_command(self._write.command(self._spark))
+
+    def insertInto(self, tableName: str, overwrite: Optional[bool] = None) -> None:
+        """Inserts the content of the :class:`DataFrame` to the specified table.
+
+        It requires that the schema of the :class:`DataFrame` is the same as the
+        schema of the table.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        overwrite : bool, optional
+            If true, overwrites existing data. Disabled by default
+
+        Notes
+        -----
+        Unlike :meth:`DataFrameWriter.saveAsTable`, :meth:`DataFrameWriter.insertInto` ignores
+        the column names and just uses position-based resolution.
+
+        Examples
+        --------
+        >>> _ = spark.sql("DROP TABLE IF EXISTS tblA")
+        >>> df = spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... )
+        >>> df.write.saveAsTable("tblA")
+
+        Insert the data into 'tblA' table but with different column names.
+
+        >>> df.selectExpr("age AS col1", "name AS col2").write.insertInto("tblA")
+        >>> spark.read.table("tblA").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE tblA")
+        """
+        if overwrite is not None:
+            self.mode("overwrite" if overwrite else "append")
+        self.saveAsTable(tableName)
+
+    def saveAsTable(
+        self,
+        name: str,
+        format: Optional[str] = None,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` as the specified table.
+
+        In the case the table already exists, behavior of this function depends on the
+        save mode, specified by the `mode` function (default to throwing an exception).
+        When `mode` is `Overwrite`, the schema of the :class:`DataFrame` does not need to be
+        the same as that of the existing table.
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Notes
+        -----
+        When `mode` is `Append`, if there is an existing table, we will use the format and
+        options of the existing table. The column order in the schema of the :class:`DataFrame`
+        doesn't need to be same as that of the existing table. Unlike
+        :meth:`DataFrameWriter.insertInto`, :meth:`DataFrameWriter.saveAsTable` will use the
+        column names to find the correct column positions.
+
+        Parameters
+        ----------
+        name : str
+            the table name
+        format : str, optional
+            the format used to save
+        mode : str, optional
+            one of `append`, `overwrite`, `error`, `errorifexists`, `ignore` \
+            (default: error)
+        partitionBy : str or list
+            names of partitioning columns
+        **options : dict
+            all other string options
+
+        Examples
+        --------
+        Creates a table from a DataFrame, and read it back.
+
+        >>> _ = spark.sql("DROP TABLE IF EXISTS tblA")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.saveAsTable("tblA")
+        >>> spark.read.table("tblA").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE tblA")
+        """
+        self.mode(mode).options(**options)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        if format is not None:
+            self.format(format)
+        self._write.table_name = name
+        self._spark.execute_command(self._write.command(self._spark))
+
+    def json(
+        self,
+        path: str,
+        mode: Optional[str] = None,
+        compression: Optional[str] = None,
+        dateFormat: Optional[str] = None,
+        timestampFormat: Optional[str] = None,
+        lineSep: Optional[str] = None,
+        encoding: Optional[str] = None,
+        ignoreNullFields: Optional[Union[bool, str]] = None,
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` in JSON format
+        (`JSON Lines text format or newline-delimited JSON <http://jsonlines.org/>`_) at the
+        specified path.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-json.html#data-source-option>`_
+            in the version you use.
+
+            .. # noqa
+
+        Examples
+        --------
+        Write a DataFrame into a JSON file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a JSON file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.json(d, mode="overwrite")
+        ...
+        ...     # Read the JSON file as a DataFrame.
+        ...     spark.read.format("json").load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode)
+        self._set_opts(
+            compression=compression,
+            dateFormat=dateFormat,
+            timestampFormat=timestampFormat,
+            lineSep=lineSep,
+            encoding=encoding,
+            ignoreNullFields=ignoreNullFields,
+        )
+        self.format("json").save(path)
+
+    def parquet(
+        self,
+        path: str,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        compression: Optional[str] = None,
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` in Parquet format at the specified path.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+        partitionBy : str or list, optional
+            names of partitioning columns
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#data-source-option>`_
+            in the version you use.
+
+            .. # noqa
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.parquet(d, mode="overwrite")
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.format("parquet").load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        self.option("compression", compression)
+        self.format("parquet").save(path)
+
+    def text(
+        self, path: str, compression: Optional[str] = None, lineSep: Optional[str] = None
+    ) -> None:
+        """Saves the content of the DataFrame in a text file at the specified path.
+        The text files will be encoded as UTF-8.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-text.html#data-source-option>`_
+            in the version you use.
+
+            .. # noqa
+
+        Notes
+        -----
+        The DataFrame must have only one column that is of string type.
+        Each row becomes a new line in the output file.
+
+        Examples
+        --------
+        Write a DataFrame into a text file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a text file
+        ...     df = spark.createDataFrame([("a",), ("b",), ("c",)], schema=["alphabets"])
+        ...     df.write.mode("overwrite").text(d)
+        ...
+        ...     # Read the text file as a DataFrame.
+        ...     spark.read.schema(df.schema).format("text").load(d).sort("alphabets").show()
+        +---------+
+        |alphabets|
+        +---------+
+        |        a|
+        |        b|
+        |        c|
+        +---------+
+        """
+        self._set_opts(compression=compression, lineSep=lineSep)
+        self.format("text").save(path)
+
+    def csv(
+        self,
+        path: str,
+        mode: Optional[str] = None,
+        compression: Optional[str] = None,
+        sep: Optional[str] = None,
+        quote: Optional[str] = None,
+        escape: Optional[str] = None,
+        header: Optional[Union[bool, str]] = None,
+        nullValue: Optional[str] = None,
+        escapeQuotes: Optional[Union[bool, str]] = None,
+        quoteAll: Optional[Union[bool, str]] = None,
+        dateFormat: Optional[str] = None,
+        timestampFormat: Optional[str] = None,
+        ignoreLeadingWhiteSpace: Optional[Union[bool, str]] = None,
+        ignoreTrailingWhiteSpace: Optional[Union[bool, str]] = None,
+        charToEscapeQuoteEscaping: Optional[str] = None,
+        encoding: Optional[str] = None,
+        emptyValue: Optional[str] = None,
+        lineSep: Optional[str] = None,
+    ) -> None:
+        r"""Saves the content of the :class:`DataFrame` in CSV format at the specified path.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-csv.html#data-source-option>`_
+            in the version you use.
+
+            .. # noqa
+
+        Examples
+        --------
+        Write a DataFrame into a CSV file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file
+        ...     df = spark.createDataFrame([{"age": 100, "name": "Hyukjin Kwon"}])
+        ...     df.write.csv(d, mode="overwrite")
+        ...
+        ...     # Read the CSV file as a DataFrame with 'nullValue' option set to 'Hyukjin Kwon'.
+        ...     spark.read.schema(df.schema).format("csv").option(
+        ...         "nullValue", "Hyukjin Kwon").load(d).show()
+        +---+----+
+        |age|name|
+        +---+----+
+        |100|null|
+        +---+----+
+        """
+        self.mode(mode)
+        self._set_opts(
+            compression=compression,
+            sep=sep,
+            quote=quote,
+            escape=escape,
+            header=header,
+            nullValue=nullValue,
+            escapeQuotes=escapeQuotes,
+            quoteAll=quoteAll,
+            dateFormat=dateFormat,
+            timestampFormat=timestampFormat,
+            ignoreLeadingWhiteSpace=ignoreLeadingWhiteSpace,
+            ignoreTrailingWhiteSpace=ignoreTrailingWhiteSpace,
+            charToEscapeQuoteEscaping=charToEscapeQuoteEscaping,
+            encoding=encoding,
+            emptyValue=emptyValue,
+            lineSep=lineSep,
+        )
+        self.format("csv").save(path)
+
+    def orc(
+        self,
+        path: str,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        compression: Optional[str] = None,
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` in ORC format at the specified path.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+        partitionBy : str or list, optional
+            names of partitioning columns
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-orc.html#data-source-option>`_
+            in the version you use.
+
+            .. # noqa
+
+        Examples
+        --------
+        Write a DataFrame into a ORC file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a ORC file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.orc(d, mode="overwrite")
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.format("orc").load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        self._set_opts(compression=compression)
+        self.format("orc").save(path)
+
+    def jdbc(self, *args: Any, **kwargs: Any) -> None:

Review Comment:
   Why add it if we are not supporting it now?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] amaliujia commented on pull request #38801: [SPARK-41317][CONNECT][PYTHON] Add basic support for DataFrameWriter

Posted by GitBox <gi...@apache.org>.
amaliujia commented on PR #38801:
URL: https://github.com/apache/spark/pull/38801#issuecomment-1331078267

   Overall look good to me.
   
   This is based on the past scala and proto side writer API work so no big concerns. 
   
   IIRC there were still some proto API level design comments that worth to take a look. I probably could re-visit that later after expression work.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on a diff in pull request #38801: [SPARK-41317][CONNECT][PYTHON] Add basic support for DataFrameWriter

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on code in PR #38801:
URL: https://github.com/apache/spark/pull/38801#discussion_r1040708581


##########
python/pyspark/sql/tests/connect/test_connect_plan_only.py:
##########
@@ -17,6 +17,7 @@
 from typing import cast
 import unittest
 
+from pyspark.sql.connect.plan import WriteOperation

Review Comment:
   This is the last piece we need to fix to recover `pyspark-connect` from `pandas` issue for now. Here is the last PR for `pyspark-connect`.
   - https://github.com/apache/spark/pull/38934



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] grundprinzip commented on pull request #38801: [SPARK-40539] [CONNECT] [PYTHON] Add basic support for DataFrameWriter

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on PR #38801:
URL: https://github.com/apache/spark/pull/38801#issuecomment-1327986740

   The base code is copy and paste from the original source files. I'll update the language in both places and take a look at the examples. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] grundprinzip commented on a diff in pull request #38801: [SPARK-40539] [CONNECT] [PYTHON] Add basic support for DataFrameWriter

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #38801:
URL: https://github.com/apache/spark/pull/38801#discussion_r1033202925


##########
python/pyspark/sql/connect/writer.py:
##########
@@ -0,0 +1,926 @@
+#

Review Comment:
   The reason was circular imports, I was able to do a small trick to avoid this moved it back into `readwriter.py`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #38801: [SPARK-40539] [CONNECT] [PYTHON] Add basic support for DataFrameWriter

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on PR #38801:
URL: https://github.com/apache/spark/pull/38801#issuecomment-1328169165

   Can one of the admins verify this patch?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] bjornjorgensen commented on a diff in pull request #38801: [SPARK-40539] [CONNECT] [PYTHON] Add basic support for DataFrameWriter

Posted by GitBox <gi...@apache.org>.
bjornjorgensen commented on code in PR #38801:
URL: https://github.com/apache/spark/pull/38801#discussion_r1032667339


##########
python/pyspark/sql/connect/writer.py:
##########
@@ -0,0 +1,922 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+
+from typing import Optional, Union, List, overload, Tuple, cast, Any
+from typing import TYPE_CHECKING
+
+from pyspark.sql.connect.plan import WriteOperation, LogicalPlan
+from pyspark.sql.types import StructType
+from pyspark.sql.utils import to_str
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect._typing import OptionalPrimitiveType
+    from pyspark.sql.connect.client import RemoteSparkSession
+
+
+PathOrPaths = Union[str, List[str]]
+TupleOrListOfString = Union[List[str], Tuple[str, ...]]
+
+
+class OptionUtils:
+    def _set_opts(
+        self,
+        schema: Optional[Union[StructType, str]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """
+        Set named options (filter out those the value is None)
+        """
+        if schema is not None:
+            self.schema(schema)  # type: ignore[attr-defined]
+        for k, v in options.items():
+            if v is not None:
+                self.option(k, v)  # type: ignore[attr-defined]
+
+
+class DataFrameWriter(OptionUtils):
+    """
+    Interface used to write a :class:`DataFrame` to external storage systems
+    (e.g. file systems, key-value stores, etc). Use :attr:`DataFrame.write`
+    to access this.
+
+    .. versionadded:: 3.4.0
+    """
+
+    def __init__(self, plan: "LogicalPlan", session: "RemoteSparkSession"):
+        self._df: "LogicalPlan" = plan
+        self._spark: "RemoteSparkSession" = session
+        self._write: "WriteOperation" = WriteOperation(self._df)
+
+    def mode(self, saveMode: Optional[str]) -> "DataFrameWriter":
+        """Specifies the behavior when data or table already exists.
+
+        Options include:
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Examples
+        --------
+        Raise an error when writing to an existing path.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     spark.createDataFrame(
+        ...         [{"age": 80, "name": "Xinrong Meng"}]
+        ...     ).write.mode("error").format("parquet").save(d)
+        Traceback (most recent call last):
+            ...
+        pyspark.sql.utils.AnalysisException: ...
+
+        Write a Parquet file back with various options, and read it back.
+
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Overwrite the path with a new Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 120, "name": "Takuya Ueshin"}]
+        ...     ).write.mode("append").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 140, "name": "Haejoon Lee"}]
+        ...     ).write.mode("ignore").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |120|Takuya Ueshin|
+        |100| Hyukjin Kwon|
+        +---+-------------+
+        """
+        # At the JVM side, the default value of mode is already set to "error".
+        # So, if the given saveMode is None, we will not call JVM-side's mode method.
+        if saveMode is not None:
+            self._write.mode = saveMode
+        return self
+
+    def format(self, source: str) -> "DataFrameWriter":
+        """Specifies the underlying output data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        source : str
+            string, name of the data source, e.g. 'json', 'parquet'.
+
+        Examples
+        --------
+        >>> spark.range(1).write.format('parquet')
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Write a DataFrame into a Parquet file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.format('parquet').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.source = source
+        return self
+
+    def option(self, key: str, value: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds a output option for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        key : str
+            The key for the option to set.
+        value
+            The value for the option to set.
+
+        Examples
+        --------
+        >>> spark.range(1).write.option("key", "value")
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Specify the option 'nullValue' with writing a CSV file.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file with 'nullValue' option set to 'Hyukjin Kwon'.
+        ...     df = spark.createDataFrame([(100, None)], "age INT, name STRING")
+        ...     df.write.option("nullValue", "Hyukjin Kwon").mode("overwrite").format("csv").save(d)
+        ...
+        ...     # Read the CSV file as a DataFrame.
+        ...     spark.read.schema(df.schema).format('csv').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.options[key] = to_str(value)
+        return self
+
+    def options(self, **options: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds output options for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        **options : dict
+            The dictionary of string keys and prmitive-type values.
+
+        Examples
+        --------
+        >>> spark.range(1).write.option("key", "value")
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Specify the option 'nullValue' and 'header' with writing a CSV file.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file with 'nullValue' option set to 'Hyukjin Kwon',
+        ...     # and 'header' option set to `True`.
+        ...     df = spark.createDataFrame([(100, "Hyukjin Kwon")], ["age", "name"])
+        ...     df.write.options(nullValue="Hyukjin Kwon", header=True).mode(
+        ...         "overwrite").format("csv").save(d)
+        ...
+        ...     # Read the CSV file as a DataFrame.
+        ...     spark.read.option("header", True).format('csv').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        for k in options:
+            self._write.options[k] = to_str(options[k])
+        return self
+
+    @overload
+    def partitionBy(self, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def partitionBy(self, *cols: List[str]) -> "DataFrameWriter":
+        ...
+
+    def partitionBy(self, *cols: Union[str, List[str]]) -> "DataFrameWriter":
+        """Partitions the output by the given columns on the file system.
+
+        If specified, the output is laid out on the file system similar
+        to Hive's partitioning scheme.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        cols : str or list
+            name of columns
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a partitioned manner, and read it back.
+
+        >>> import tempfile
+        >>> import os
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file in a partitioned manner.
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}, {"age": 120, "name": "Ruifeng Zheng"}]
+        ...     ).write.partitionBy("name").mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).sort("age").show()
+        ...
+        ...     # Read one partition as a DataFrame.
+        ...     spark.read.parquet(f"{d}{os.path.sep}name=Hyukjin Kwon").show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |100| Hyukjin Kwon|
+        |120|Ruifeng Zheng|
+        +---+-------------+
+        +---+
+        |age|
+        +---+
+        |100|
+        +---+
+        """
+        if len(cols) == 1 and isinstance(cols[0], (list, tuple)):
+            cols = cols[0]  # type: ignore[assignment]
+
+        self._write.partitioning_cols = cast(List[str], cols)
+        return self
+
+    @overload
+    def bucketBy(self, numBuckets: int, col: str, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def bucketBy(self, numBuckets: int, col: TupleOrListOfString) -> "DataFrameWriter":
+        ...
+
+    def bucketBy(
+        self, numBuckets: int, col: Union[str, TupleOrListOfString], *cols: Optional[str]
+    ) -> "DataFrameWriter":
+        """Buckets the output by the given columns. If specified,
+        the output is laid out on the file system similar to Hive's bucketing scheme,
+        but with a different bucket hash function and is not compatible with Hive's bucketing.
+
+        .. versionadded:: 2.3.0
+
+        Parameters
+        ----------
+        numBuckets : int
+            the number of buckets to save
+        col : str, list or tuple
+            a name of a column, or a list of names.
+        cols : str
+            additional names (optional). If `col` is a list it should be empty.
+
+        Notes
+        -----
+        Applicable for file-based data sources in combination with
+        :py:meth:`DataFrameWriter.saveAsTable`.
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a buckted manner, and read it back.
+
+        >>> from pyspark.sql.functions import input_file_name
+        >>> # Write a DataFrame into a Parquet file in a bucketed manner.
+        ... _ = spark.sql("DROP TABLE IF EXISTS bucketed_table")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.bucketBy(2, "name").mode("overwrite").saveAsTable("bucketed_table")
+        >>> # Read the Parquet file as a DataFrame.
+        ... spark.read.table("bucketed_table").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE bucketed_table")
+        """
+        if not isinstance(numBuckets, int):
+            raise TypeError("numBuckets should be an int, got {0}.".format(type(numBuckets)))
+
+        if isinstance(col, (list, tuple)):
+            if cols:
+                raise ValueError("col is a {0} but cols are not empty".format(type(col)))
+
+            col, cols = col[0], col[1:]  # type: ignore[assignment]
+
+        if not all(isinstance(c, str) for c in cols) or not (isinstance(col, str)):
+            raise TypeError("all names should be `str`")
+
+        self._write.num_buckets = numBuckets
+        self._write.bucket_cols = cast(List[str], cols)
+        return self
+
+    @overload
+    def sortBy(self, col: str, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def sortBy(self, col: TupleOrListOfString) -> "DataFrameWriter":
+        ...
+
+    def sortBy(
+        self, col: Union[str, TupleOrListOfString], *cols: Optional[str]
+    ) -> "DataFrameWriter":
+        """Sorts the output in each bucket by the given columns on the file system.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        col : str, tuple or list
+            a name of a column, or a list of names.
+        cols : str
+            additional names (optional). If `col` is a list it should be empty.
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a sorted-buckted manner, and read it back.
+
+        >>> from pyspark.sql.functions import input_file_name
+        >>> # Write a DataFrame into a Parquet file in a sorted-bucketed manner.
+        ... _ = spark.sql("DROP TABLE IF EXISTS sorted_bucketed_table")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.bucketBy(1, "name").sortBy("age").mode(
+        ...     "overwrite").saveAsTable("sorted_bucketed_table")
+        >>> # Read the Parquet file as a DataFrame.
+        ... spark.read.table("sorted_bucketed_table").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE sorted_bucketed_table")
+        """
+        if isinstance(col, (list, tuple)):
+            if cols:
+                raise ValueError("col is a {0} but cols are not empty".format(type(col)))
+
+            col, cols = col[0], col[1:]  # type: ignore[assignment]
+
+        if not all(isinstance(c, str) for c in cols) or not (isinstance(col, str)):
+            raise TypeError("all names should be `str`")
+
+        self._write.sort_cols = cast(List[str], cols)
+        return self
+
+    def save(
+        self,
+        path: Optional[str] = None,
+        format: Optional[str] = None,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """Saves the contents of the :class:`DataFrame` to a data source.
+
+        The data source is specified by the ``format`` and a set of ``options``.
+        If ``format`` is not specified, the default data source configured by
+        ``spark.sql.sources.default`` will be used.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str, optional
+            the path in a Hadoop supported file system
+        format : str, optional
+            the format used to save
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+        partitionBy : list, optional
+            names of partitioning columns
+        **options : dict
+            all other string options
+
+        Examples
+        --------
+        Write a DataFrame into a JSON file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a JSON file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("json").save(d)
+        ...
+        ...     # Read the JSON file as a DataFrame.
+        ...     spark.read.format('json').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode).options(**options)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        if format is not None:
+            self.format(format)
+        self._write.path = path
+        self._spark.execute_command(self._write.command(self._spark))
+
+    def insertInto(self, tableName: str, overwrite: Optional[bool] = None) -> None:
+        """Inserts the content of the :class:`DataFrame` to the specified table.
+
+        It requires that the schema of the :class:`DataFrame` is the same as the
+        schema of the table.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        overwrite : bool, optional
+            If true, overwrites existing data. Disabled by default
+
+        Notes
+        -----
+        Unlike :meth:`DataFrameWriter.saveAsTable`, :meth:`DataFrameWriter.insertInto` ignores
+        the column names and just uses position-based resolution.
+
+        Examples
+        --------
+        >>> _ = spark.sql("DROP TABLE IF EXISTS tblA")
+        >>> df = spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... )
+        >>> df.write.saveAsTable("tblA")
+
+        Insert the data into 'tblA' table but with different column names.
+
+        >>> df.selectExpr("age AS col1", "name AS col2").write.insertInto("tblA")
+        >>> spark.read.table("tblA").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE tblA")
+        """
+        if overwrite is not None:
+            self.mode("overwrite" if overwrite else "append")
+        self.saveAsTable(tableName)
+
+    def saveAsTable(
+        self,
+        name: str,
+        format: Optional[str] = None,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` as the specified table.
+
+        In the case the table already exists, behavior of this function depends on the
+        save mode, specified by the `mode` function (default to throwing an exception).
+        When `mode` is `Overwrite`, the schema of the :class:`DataFrame` does not need to be
+        the same as that of the existing table.
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Notes
+        -----
+        When `mode` is `Append`, if there is an existing table, we will use the format and
+        options of the existing table. The column order in the schema of the :class:`DataFrame`
+        doesn't need to be same as that of the existing table. Unlike
+        :meth:`DataFrameWriter.insertInto`, :meth:`DataFrameWriter.saveAsTable` will use the
+        column names to find the correct column positions.
+
+        Parameters
+        ----------
+        name : str
+            the table name
+        format : str, optional
+            the format used to save
+        mode : str, optional
+            one of `append`, `overwrite`, `error`, `errorifexists`, `ignore` \
+            (default: error)
+        partitionBy : str or list
+            names of partitioning columns
+        **options : dict
+            all other string options
+
+        Examples
+        --------
+        Creates a table from a DataFrame, and read it back.
+
+        >>> _ = spark.sql("DROP TABLE IF EXISTS tblA")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.saveAsTable("tblA")
+        >>> spark.read.table("tblA").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE tblA")
+        """
+        self.mode(mode).options(**options)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        if format is not None:
+            self.format(format)
+        self._write.table_name = name
+        self._spark.execute_command(self._write.command(self._spark))
+
+    def json(
+        self,
+        path: str,
+        mode: Optional[str] = None,
+        compression: Optional[str] = None,
+        dateFormat: Optional[str] = None,
+        timestampFormat: Optional[str] = None,
+        lineSep: Optional[str] = None,
+        encoding: Optional[str] = None,
+        ignoreNullFields: Optional[Union[bool, str]] = None,
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` in JSON format
+        (`JSON Lines text format or newline-delimited JSON <http://jsonlines.org/>`_) at the
+        specified path.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-json.html#data-source-option>`_
+            in the version you use.

Review Comment:
   for the version you use.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] bjornjorgensen commented on a diff in pull request #38801: [SPARK-40539] [CONNECT] [PYTHON] Add basic support for DataFrameWriter

Posted by GitBox <gi...@apache.org>.
bjornjorgensen commented on code in PR #38801:
URL: https://github.com/apache/spark/pull/38801#discussion_r1032662076


##########
python/pyspark/sql/connect/writer.py:
##########
@@ -0,0 +1,922 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+
+from typing import Optional, Union, List, overload, Tuple, cast, Any
+from typing import TYPE_CHECKING
+
+from pyspark.sql.connect.plan import WriteOperation, LogicalPlan
+from pyspark.sql.types import StructType
+from pyspark.sql.utils import to_str
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect._typing import OptionalPrimitiveType
+    from pyspark.sql.connect.client import RemoteSparkSession
+
+
+PathOrPaths = Union[str, List[str]]
+TupleOrListOfString = Union[List[str], Tuple[str, ...]]
+
+
+class OptionUtils:
+    def _set_opts(
+        self,
+        schema: Optional[Union[StructType, str]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """
+        Set named options (filter out those the value is None)
+        """
+        if schema is not None:
+            self.schema(schema)  # type: ignore[attr-defined]
+        for k, v in options.items():
+            if v is not None:
+                self.option(k, v)  # type: ignore[attr-defined]
+
+
+class DataFrameWriter(OptionUtils):
+    """
+    Interface used to write a :class:`DataFrame` to external storage systems
+    (e.g. file systems, key-value stores, etc). Use :attr:`DataFrame.write`
+    to access this.
+
+    .. versionadded:: 3.4.0
+    """
+
+    def __init__(self, plan: "LogicalPlan", session: "RemoteSparkSession"):
+        self._df: "LogicalPlan" = plan
+        self._spark: "RemoteSparkSession" = session
+        self._write: "WriteOperation" = WriteOperation(self._df)
+
+    def mode(self, saveMode: Optional[str]) -> "DataFrameWriter":
+        """Specifies the behavior when data or table already exists.
+
+        Options include:
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Examples
+        --------
+        Raise an error when writing to an existing path.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     spark.createDataFrame(
+        ...         [{"age": 80, "name": "Xinrong Meng"}]
+        ...     ).write.mode("error").format("parquet").save(d)
+        Traceback (most recent call last):
+            ...
+        pyspark.sql.utils.AnalysisException: ...
+
+        Write a Parquet file back with various options, and read it back.
+
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Overwrite the path with a new Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 120, "name": "Takuya Ueshin"}]
+        ...     ).write.mode("append").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 140, "name": "Haejoon Lee"}]
+        ...     ).write.mode("ignore").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |120|Takuya Ueshin|
+        |100| Hyukjin Kwon|
+        +---+-------------+
+        """
+        # At the JVM side, the default value of mode is already set to "error".
+        # So, if the given saveMode is None, we will not call JVM-side's mode method.
+        if saveMode is not None:
+            self._write.mode = saveMode
+        return self
+
+    def format(self, source: str) -> "DataFrameWriter":
+        """Specifies the underlying output data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        source : str
+            string, name of the data source, e.g. 'json', 'parquet'.
+
+        Examples
+        --------
+        >>> spark.range(1).write.format('parquet')
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Write a DataFrame into a Parquet file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.format('parquet').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.source = source
+        return self
+
+    def option(self, key: str, value: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds a output option for the underlying data source.

Review Comment:
   Adds a output



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HyukjinKwon commented on pull request #38801: [SPARK-41317][CONNECT][PYTHON] Add basic support for DataFrameWriter

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on PR #38801:
URL: https://github.com/apache/spark/pull/38801#issuecomment-1330372149

   Hm, I can't reproduce the error in my local:
   
   ```
   ======================================================================
   FAIL [0.355s]: test_write_operations (pyspark.sql.tests.connect.test_connect_basic.SparkConnectTests)
   ----------------------------------------------------------------------
   Traceback (most recent call last):
     File "/__w/spark/spark/python/pyspark/sql/tests/connect/test_connect_basic.py", line 615, in test_write_operations
       self.assertEqual(set(df.collect()), set(ndf.collect()))
   AssertionError: Items in the first set but not the second:
   Row(id=1)
   Row(id=75)
   Row(id=25)
   Row(id=50)
   
   ```
   
   ```
   Running PySpark tests. Output is in /.../spark/python/unit-tests.log
   Will test against the following Python executables: ['python3.9']
   Will test the following Python tests: ['pyspark.sql.tests.connect.test_connect_basic']
   python3.9 python_implementation is CPython
   python3.9 version is: Python 3.9.5
   Starting test(python3.9): pyspark.sql.tests.connect.test_connect_basic (temp output: /.../spark/python/target/13f63ed8-72bc-489a-b7c9-2b10edf0c94b/python3.9__pyspark.sql.tests.connect.test_connect_basic__63yhqjf1.log)
   Finished test(python3.9): pyspark.sql.tests.connect.test_connect_basic (20s)
   Tests passed in 20 seconds
   ```
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] grundprinzip commented on a diff in pull request #38801: [SPARK-40539] [CONNECT] [PYTHON] Add basic support for DataFrameWriter

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #38801:
URL: https://github.com/apache/spark/pull/38801#discussion_r1033220998


##########
python/pyspark/sql/connect/writer.py:
##########
@@ -0,0 +1,926 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+
+from typing import Optional, Union, List, overload, Tuple, cast, Any
+from typing import TYPE_CHECKING
+
+from pyspark.sql.connect.plan import WriteOperation, LogicalPlan
+from pyspark.sql.types import StructType
+from pyspark.sql.utils import to_str
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect._typing import OptionalPrimitiveType
+    from pyspark.sql.connect.session import SparkSession
+
+
+PathOrPaths = Union[str, List[str]]
+TupleOrListOfString = Union[List[str], Tuple[str, ...]]
+
+
+class OptionUtils:

Review Comment:
   It should when we support the additional methods. Right now, the reader is very bare bones. Once we address it we should use the same.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] grundprinzip commented on a diff in pull request #38801: [SPARK-40539] [CONNECT] [PYTHON] Add basic support for DataFrameWriter

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #38801:
URL: https://github.com/apache/spark/pull/38801#discussion_r1032791266


##########
python/pyspark/sql/connect/writer.py:
##########
@@ -0,0 +1,922 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+
+from typing import Optional, Union, List, overload, Tuple, cast, Any
+from typing import TYPE_CHECKING
+
+from pyspark.sql.connect.plan import WriteOperation, LogicalPlan
+from pyspark.sql.types import StructType
+from pyspark.sql.utils import to_str
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect._typing import OptionalPrimitiveType
+    from pyspark.sql.connect.client import RemoteSparkSession
+
+
+PathOrPaths = Union[str, List[str]]
+TupleOrListOfString = Union[List[str], Tuple[str, ...]]
+
+
+class OptionUtils:
+    def _set_opts(
+        self,
+        schema: Optional[Union[StructType, str]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """
+        Set named options (filter out those the value is None)
+        """
+        if schema is not None:
+            self.schema(schema)  # type: ignore[attr-defined]
+        for k, v in options.items():
+            if v is not None:
+                self.option(k, v)  # type: ignore[attr-defined]
+
+
+class DataFrameWriter(OptionUtils):
+    """
+    Interface used to write a :class:`DataFrame` to external storage systems
+    (e.g. file systems, key-value stores, etc). Use :attr:`DataFrame.write`
+    to access this.
+
+    .. versionadded:: 3.4.0
+    """
+
+    def __init__(self, plan: "LogicalPlan", session: "RemoteSparkSession"):
+        self._df: "LogicalPlan" = plan
+        self._spark: "RemoteSparkSession" = session
+        self._write: "WriteOperation" = WriteOperation(self._df)
+
+    def mode(self, saveMode: Optional[str]) -> "DataFrameWriter":
+        """Specifies the behavior when data or table already exists.
+
+        Options include:
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Examples
+        --------
+        Raise an error when writing to an existing path.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     spark.createDataFrame(
+        ...         [{"age": 80, "name": "Xinrong Meng"}]
+        ...     ).write.mode("error").format("parquet").save(d)
+        Traceback (most recent call last):
+            ...
+        pyspark.sql.utils.AnalysisException: ...
+
+        Write a Parquet file back with various options, and read it back.
+
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Overwrite the path with a new Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 120, "name": "Takuya Ueshin"}]
+        ...     ).write.mode("append").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 140, "name": "Haejoon Lee"}]
+        ...     ).write.mode("ignore").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |120|Takuya Ueshin|
+        |100| Hyukjin Kwon|
+        +---+-------------+
+        """
+        # At the JVM side, the default value of mode is already set to "error".
+        # So, if the given saveMode is None, we will not call JVM-side's mode method.
+        if saveMode is not None:
+            self._write.mode = saveMode
+        return self
+
+    def format(self, source: str) -> "DataFrameWriter":
+        """Specifies the underlying output data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        source : str
+            string, name of the data source, e.g. 'json', 'parquet'.
+
+        Examples
+        --------
+        >>> spark.range(1).write.format('parquet')
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Write a DataFrame into a Parquet file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.format('parquet').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.source = source
+        return self
+
+    def option(self, key: str, value: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds a output option for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        key : str
+            The key for the option to set.
+        value
+            The value for the option to set.
+
+        Examples
+        --------
+        >>> spark.range(1).write.option("key", "value")
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Specify the option 'nullValue' with writing a CSV file.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file with 'nullValue' option set to 'Hyukjin Kwon'.
+        ...     df = spark.createDataFrame([(100, None)], "age INT, name STRING")
+        ...     df.write.option("nullValue", "Hyukjin Kwon").mode("overwrite").format("csv").save(d)
+        ...
+        ...     # Read the CSV file as a DataFrame.
+        ...     spark.read.schema(df.schema).format('csv').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.options[key] = to_str(value)
+        return self
+
+    def options(self, **options: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds output options for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        **options : dict
+            The dictionary of string keys and prmitive-type values.
+
+        Examples
+        --------
+        >>> spark.range(1).write.option("key", "value")
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Specify the option 'nullValue' and 'header' with writing a CSV file.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file with 'nullValue' option set to 'Hyukjin Kwon',
+        ...     # and 'header' option set to `True`.
+        ...     df = spark.createDataFrame([(100, "Hyukjin Kwon")], ["age", "name"])
+        ...     df.write.options(nullValue="Hyukjin Kwon", header=True).mode(
+        ...         "overwrite").format("csv").save(d)
+        ...
+        ...     # Read the CSV file as a DataFrame.
+        ...     spark.read.option("header", True).format('csv').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        for k in options:
+            self._write.options[k] = to_str(options[k])
+        return self
+
+    @overload
+    def partitionBy(self, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def partitionBy(self, *cols: List[str]) -> "DataFrameWriter":
+        ...
+
+    def partitionBy(self, *cols: Union[str, List[str]]) -> "DataFrameWriter":
+        """Partitions the output by the given columns on the file system.
+
+        If specified, the output is laid out on the file system similar
+        to Hive's partitioning scheme.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        cols : str or list
+            name of columns
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a partitioned manner, and read it back.
+
+        >>> import tempfile
+        >>> import os
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file in a partitioned manner.
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}, {"age": 120, "name": "Ruifeng Zheng"}]
+        ...     ).write.partitionBy("name").mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).sort("age").show()
+        ...
+        ...     # Read one partition as a DataFrame.
+        ...     spark.read.parquet(f"{d}{os.path.sep}name=Hyukjin Kwon").show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |100| Hyukjin Kwon|
+        |120|Ruifeng Zheng|
+        +---+-------------+
+        +---+
+        |age|
+        +---+
+        |100|
+        +---+
+        """
+        if len(cols) == 1 and isinstance(cols[0], (list, tuple)):
+            cols = cols[0]  # type: ignore[assignment]
+
+        self._write.partitioning_cols = cast(List[str], cols)
+        return self
+
+    @overload
+    def bucketBy(self, numBuckets: int, col: str, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def bucketBy(self, numBuckets: int, col: TupleOrListOfString) -> "DataFrameWriter":
+        ...
+
+    def bucketBy(
+        self, numBuckets: int, col: Union[str, TupleOrListOfString], *cols: Optional[str]
+    ) -> "DataFrameWriter":
+        """Buckets the output by the given columns. If specified,
+        the output is laid out on the file system similar to Hive's bucketing scheme,
+        but with a different bucket hash function and is not compatible with Hive's bucketing.
+
+        .. versionadded:: 2.3.0
+
+        Parameters
+        ----------
+        numBuckets : int
+            the number of buckets to save
+        col : str, list or tuple
+            a name of a column, or a list of names.
+        cols : str
+            additional names (optional). If `col` is a list it should be empty.
+
+        Notes
+        -----
+        Applicable for file-based data sources in combination with
+        :py:meth:`DataFrameWriter.saveAsTable`.
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a buckted manner, and read it back.
+
+        >>> from pyspark.sql.functions import input_file_name
+        >>> # Write a DataFrame into a Parquet file in a bucketed manner.
+        ... _ = spark.sql("DROP TABLE IF EXISTS bucketed_table")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.bucketBy(2, "name").mode("overwrite").saveAsTable("bucketed_table")
+        >>> # Read the Parquet file as a DataFrame.
+        ... spark.read.table("bucketed_table").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE bucketed_table")
+        """
+        if not isinstance(numBuckets, int):
+            raise TypeError("numBuckets should be an int, got {0}.".format(type(numBuckets)))
+
+        if isinstance(col, (list, tuple)):
+            if cols:
+                raise ValueError("col is a {0} but cols are not empty".format(type(col)))
+
+            col, cols = col[0], col[1:]  # type: ignore[assignment]
+
+        if not all(isinstance(c, str) for c in cols) or not (isinstance(col, str)):
+            raise TypeError("all names should be `str`")
+
+        self._write.num_buckets = numBuckets
+        self._write.bucket_cols = cast(List[str], cols)
+        return self
+
+    @overload
+    def sortBy(self, col: str, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def sortBy(self, col: TupleOrListOfString) -> "DataFrameWriter":
+        ...
+
+    def sortBy(
+        self, col: Union[str, TupleOrListOfString], *cols: Optional[str]
+    ) -> "DataFrameWriter":
+        """Sorts the output in each bucket by the given columns on the file system.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        col : str, tuple or list
+            a name of a column, or a list of names.
+        cols : str
+            additional names (optional). If `col` is a list it should be empty.
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a sorted-buckted manner, and read it back.
+
+        >>> from pyspark.sql.functions import input_file_name
+        >>> # Write a DataFrame into a Parquet file in a sorted-bucketed manner.
+        ... _ = spark.sql("DROP TABLE IF EXISTS sorted_bucketed_table")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.bucketBy(1, "name").sortBy("age").mode(
+        ...     "overwrite").saveAsTable("sorted_bucketed_table")
+        >>> # Read the Parquet file as a DataFrame.
+        ... spark.read.table("sorted_bucketed_table").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE sorted_bucketed_table")
+        """
+        if isinstance(col, (list, tuple)):
+            if cols:
+                raise ValueError("col is a {0} but cols are not empty".format(type(col)))
+
+            col, cols = col[0], col[1:]  # type: ignore[assignment]
+
+        if not all(isinstance(c, str) for c in cols) or not (isinstance(col, str)):
+            raise TypeError("all names should be `str`")
+
+        self._write.sort_cols = cast(List[str], cols)
+        return self
+
+    def save(
+        self,
+        path: Optional[str] = None,
+        format: Optional[str] = None,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """Saves the contents of the :class:`DataFrame` to a data source.
+
+        The data source is specified by the ``format`` and a set of ``options``.
+        If ``format`` is not specified, the default data source configured by
+        ``spark.sql.sources.default`` will be used.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str, optional
+            the path in a Hadoop supported file system
+        format : str, optional
+            the format used to save
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+        partitionBy : list, optional
+            names of partitioning columns
+        **options : dict
+            all other string options
+
+        Examples
+        --------
+        Write a DataFrame into a JSON file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a JSON file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("json").save(d)
+        ...
+        ...     # Read the JSON file as a DataFrame.
+        ...     spark.read.format('json').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode).options(**options)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        if format is not None:
+            self.format(format)
+        self._write.path = path
+        self._spark.execute_command(self._write.command(self._spark))
+
+    def insertInto(self, tableName: str, overwrite: Optional[bool] = None) -> None:
+        """Inserts the content of the :class:`DataFrame` to the specified table.
+
+        It requires that the schema of the :class:`DataFrame` is the same as the
+        schema of the table.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        overwrite : bool, optional
+            If true, overwrites existing data. Disabled by default
+
+        Notes
+        -----
+        Unlike :meth:`DataFrameWriter.saveAsTable`, :meth:`DataFrameWriter.insertInto` ignores
+        the column names and just uses position-based resolution.
+
+        Examples
+        --------
+        >>> _ = spark.sql("DROP TABLE IF EXISTS tblA")
+        >>> df = spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... )
+        >>> df.write.saveAsTable("tblA")
+
+        Insert the data into 'tblA' table but with different column names.
+
+        >>> df.selectExpr("age AS col1", "name AS col2").write.insertInto("tblA")
+        >>> spark.read.table("tblA").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE tblA")
+        """
+        if overwrite is not None:
+            self.mode("overwrite" if overwrite else "append")
+        self.saveAsTable(tableName)
+
+    def saveAsTable(
+        self,
+        name: str,
+        format: Optional[str] = None,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` as the specified table.
+
+        In the case the table already exists, behavior of this function depends on the
+        save mode, specified by the `mode` function (default to throwing an exception).
+        When `mode` is `Overwrite`, the schema of the :class:`DataFrame` does not need to be
+        the same as that of the existing table.
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Notes
+        -----
+        When `mode` is `Append`, if there is an existing table, we will use the format and
+        options of the existing table. The column order in the schema of the :class:`DataFrame`
+        doesn't need to be same as that of the existing table. Unlike
+        :meth:`DataFrameWriter.insertInto`, :meth:`DataFrameWriter.saveAsTable` will use the
+        column names to find the correct column positions.
+
+        Parameters
+        ----------
+        name : str
+            the table name
+        format : str, optional
+            the format used to save
+        mode : str, optional
+            one of `append`, `overwrite`, `error`, `errorifexists`, `ignore` \
+            (default: error)
+        partitionBy : str or list
+            names of partitioning columns
+        **options : dict
+            all other string options
+
+        Examples
+        --------
+        Creates a table from a DataFrame, and read it back.
+
+        >>> _ = spark.sql("DROP TABLE IF EXISTS tblA")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.saveAsTable("tblA")
+        >>> spark.read.table("tblA").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE tblA")
+        """
+        self.mode(mode).options(**options)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        if format is not None:
+            self.format(format)
+        self._write.table_name = name
+        self._spark.execute_command(self._write.command(self._spark))
+
+    def json(
+        self,
+        path: str,
+        mode: Optional[str] = None,
+        compression: Optional[str] = None,
+        dateFormat: Optional[str] = None,
+        timestampFormat: Optional[str] = None,
+        lineSep: Optional[str] = None,
+        encoding: Optional[str] = None,
+        ignoreNullFields: Optional[Union[bool, str]] = None,
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` in JSON format
+        (`JSON Lines text format or newline-delimited JSON <http://jsonlines.org/>`_) at the
+        specified path.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-json.html#data-source-option>`_
+            in the version you use.
+
+            .. # noqa
+
+        Examples
+        --------
+        Write a DataFrame into a JSON file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a JSON file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.json(d, mode="overwrite")
+        ...
+        ...     # Read the JSON file as a DataFrame.
+        ...     spark.read.format("json").load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode)
+        self._set_opts(
+            compression=compression,
+            dateFormat=dateFormat,
+            timestampFormat=timestampFormat,
+            lineSep=lineSep,
+            encoding=encoding,
+            ignoreNullFields=ignoreNullFields,
+        )
+        self.format("json").save(path)
+
+    def parquet(
+        self,
+        path: str,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        compression: Optional[str] = None,
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` in Parquet format at the specified path.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+        partitionBy : str or list, optional
+            names of partitioning columns
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#data-source-option>`_
+            in the version you use.
+
+            .. # noqa
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.parquet(d, mode="overwrite")
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.format("parquet").load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        self.option("compression", compression)
+        self.format("parquet").save(path)
+
+    def text(
+        self, path: str, compression: Optional[str] = None, lineSep: Optional[str] = None
+    ) -> None:
+        """Saves the content of the DataFrame in a text file at the specified path.
+        The text files will be encoded as UTF-8.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-text.html#data-source-option>`_
+            in the version you use.
+
+            .. # noqa
+
+        Notes
+        -----
+        The DataFrame must have only one column that is of string type.
+        Each row becomes a new line in the output file.
+
+        Examples
+        --------
+        Write a DataFrame into a text file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a text file
+        ...     df = spark.createDataFrame([("a",), ("b",), ("c",)], schema=["alphabets"])
+        ...     df.write.mode("overwrite").text(d)
+        ...
+        ...     # Read the text file as a DataFrame.
+        ...     spark.read.schema(df.schema).format("text").load(d).sort("alphabets").show()
+        +---------+
+        |alphabets|
+        +---------+
+        |        a|
+        |        b|
+        |        c|
+        +---------+
+        """
+        self._set_opts(compression=compression, lineSep=lineSep)
+        self.format("text").save(path)
+
+    def csv(
+        self,
+        path: str,
+        mode: Optional[str] = None,
+        compression: Optional[str] = None,
+        sep: Optional[str] = None,
+        quote: Optional[str] = None,
+        escape: Optional[str] = None,
+        header: Optional[Union[bool, str]] = None,
+        nullValue: Optional[str] = None,
+        escapeQuotes: Optional[Union[bool, str]] = None,
+        quoteAll: Optional[Union[bool, str]] = None,
+        dateFormat: Optional[str] = None,
+        timestampFormat: Optional[str] = None,
+        ignoreLeadingWhiteSpace: Optional[Union[bool, str]] = None,
+        ignoreTrailingWhiteSpace: Optional[Union[bool, str]] = None,
+        charToEscapeQuoteEscaping: Optional[str] = None,
+        encoding: Optional[str] = None,
+        emptyValue: Optional[str] = None,
+        lineSep: Optional[str] = None,
+    ) -> None:
+        r"""Saves the content of the :class:`DataFrame` in CSV format at the specified path.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-csv.html#data-source-option>`_
+            in the version you use.
+
+            .. # noqa
+
+        Examples
+        --------
+        Write a DataFrame into a CSV file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file
+        ...     df = spark.createDataFrame([{"age": 100, "name": "Hyukjin Kwon"}])

Review Comment:
   The example is wrong here and in the original code. I'll fix them. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhengruifeng commented on a diff in pull request #38801: [SPARK-40539] [CONNECT] [PYTHON] Add basic support for DataFrameWriter

Posted by GitBox <gi...@apache.org>.
zhengruifeng commented on code in PR #38801:
URL: https://github.com/apache/spark/pull/38801#discussion_r1033169179


##########
python/pyspark/sql/tests/connect/test_connect_basic.py:
##########
@@ -566,6 +569,27 @@ def test_agg_with_two_agg_exprs(self):
             self.spark.read.table(self.tbl_name).agg({"name": "min", "id": "max"}).toPandas(),
         )
 
+    def test_write_operations(self):
+        with tempfile.TemporaryDirectory() as d:
+            df = self.connect.range(1, 100)
+            df.write.mode("overwrite").format("csv").save(d)
+
+            ndf = self.connect.read.load(d, format="csv")
+            df.toPandas().equals(ndf.toPandas())
+
+        with tempfile.TemporaryDirectory() as d:
+            df = self.connect.range(1, 100)
+            df.write.mode("overwrite").csv(d, lineSep="|")
+
+            ndf = self.connect.read.load(d, format="csv", lineSep="|")
+            df.toPandas().equals(ndf.toPandas())
+
+        df = self.connect.range(1, 100)
+        df.write.format("parquet").saveAsTable("parquet_test")
+
+        ndf = self.connect.read.table("parquet_test")
+        df.toPandas().equals(ndf.toPandas())

Review Comment:
   ditto



##########
python/pyspark/sql/tests/connect/test_connect_basic.py:
##########
@@ -566,6 +569,27 @@ def test_agg_with_two_agg_exprs(self):
             self.spark.read.table(self.tbl_name).agg({"name": "min", "id": "max"}).toPandas(),
         )
 
+    def test_write_operations(self):
+        with tempfile.TemporaryDirectory() as d:
+            df = self.connect.range(1, 100)
+            df.write.mode("overwrite").format("csv").save(d)
+
+            ndf = self.connect.read.load(d, format="csv")
+            df.toPandas().equals(ndf.toPandas())

Review Comment:
   we'd better use `self.assertEqual` instead, otherwise it will be painful to debug if it fails



##########
python/pyspark/sql/tests/connect/test_connect_basic.py:
##########
@@ -566,6 +569,27 @@ def test_agg_with_two_agg_exprs(self):
             self.spark.read.table(self.tbl_name).agg({"name": "min", "id": "max"}).toPandas(),
         )
 
+    def test_write_operations(self):
+        with tempfile.TemporaryDirectory() as d:
+            df = self.connect.range(1, 100)
+            df.write.mode("overwrite").format("csv").save(d)
+
+            ndf = self.connect.read.load(d, format="csv")
+            df.toPandas().equals(ndf.toPandas())
+
+        with tempfile.TemporaryDirectory() as d:
+            df = self.connect.range(1, 100)
+            df.write.mode("overwrite").csv(d, lineSep="|")
+
+            ndf = self.connect.read.load(d, format="csv", lineSep="|")
+            df.toPandas().equals(ndf.toPandas())

Review Comment:
   ditto



##########
python/pyspark/sql/connect/writer.py:
##########
@@ -0,0 +1,926 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+
+from typing import Optional, Union, List, overload, Tuple, cast, Any
+from typing import TYPE_CHECKING
+
+from pyspark.sql.connect.plan import WriteOperation, LogicalPlan
+from pyspark.sql.types import StructType
+from pyspark.sql.utils import to_str
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect._typing import OptionalPrimitiveType
+    from pyspark.sql.connect.session import SparkSession
+
+
+PathOrPaths = Union[str, List[str]]
+TupleOrListOfString = Union[List[str], Tuple[str, ...]]
+
+
+class OptionUtils:

Review Comment:
   can this also be used in `DataFrameReader`?



##########
python/pyspark/sql/connect/writer.py:
##########
@@ -0,0 +1,926 @@
+#

Review Comment:
   why creating a separate file?
   
   should it be in `readwriter.py`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] bjornjorgensen commented on a diff in pull request #38801: [SPARK-40539] [CONNECT] [PYTHON] Add basic support for DataFrameWriter

Posted by GitBox <gi...@apache.org>.
bjornjorgensen commented on code in PR #38801:
URL: https://github.com/apache/spark/pull/38801#discussion_r1032662683


##########
python/pyspark/sql/connect/writer.py:
##########
@@ -0,0 +1,922 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+
+from typing import Optional, Union, List, overload, Tuple, cast, Any
+from typing import TYPE_CHECKING
+
+from pyspark.sql.connect.plan import WriteOperation, LogicalPlan
+from pyspark.sql.types import StructType
+from pyspark.sql.utils import to_str
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect._typing import OptionalPrimitiveType
+    from pyspark.sql.connect.client import RemoteSparkSession
+
+
+PathOrPaths = Union[str, List[str]]
+TupleOrListOfString = Union[List[str], Tuple[str, ...]]
+
+
+class OptionUtils:
+    def _set_opts(
+        self,
+        schema: Optional[Union[StructType, str]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """
+        Set named options (filter out those the value is None)
+        """
+        if schema is not None:
+            self.schema(schema)  # type: ignore[attr-defined]
+        for k, v in options.items():
+            if v is not None:
+                self.option(k, v)  # type: ignore[attr-defined]
+
+
+class DataFrameWriter(OptionUtils):
+    """
+    Interface used to write a :class:`DataFrame` to external storage systems
+    (e.g. file systems, key-value stores, etc). Use :attr:`DataFrame.write`
+    to access this.
+
+    .. versionadded:: 3.4.0
+    """
+
+    def __init__(self, plan: "LogicalPlan", session: "RemoteSparkSession"):
+        self._df: "LogicalPlan" = plan
+        self._spark: "RemoteSparkSession" = session
+        self._write: "WriteOperation" = WriteOperation(self._df)
+
+    def mode(self, saveMode: Optional[str]) -> "DataFrameWriter":
+        """Specifies the behavior when data or table already exists.
+
+        Options include:
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Examples
+        --------
+        Raise an error when writing to an existing path.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     spark.createDataFrame(
+        ...         [{"age": 80, "name": "Xinrong Meng"}]
+        ...     ).write.mode("error").format("parquet").save(d)
+        Traceback (most recent call last):
+            ...
+        pyspark.sql.utils.AnalysisException: ...
+
+        Write a Parquet file back with various options, and read it back.
+
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Overwrite the path with a new Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 120, "name": "Takuya Ueshin"}]
+        ...     ).write.mode("append").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 140, "name": "Haejoon Lee"}]
+        ...     ).write.mode("ignore").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |120|Takuya Ueshin|
+        |100| Hyukjin Kwon|
+        +---+-------------+
+        """
+        # At the JVM side, the default value of mode is already set to "error".
+        # So, if the given saveMode is None, we will not call JVM-side's mode method.
+        if saveMode is not None:
+            self._write.mode = saveMode
+        return self
+
+    def format(self, source: str) -> "DataFrameWriter":
+        """Specifies the underlying output data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        source : str
+            string, name of the data source, e.g. 'json', 'parquet'.
+
+        Examples
+        --------
+        >>> spark.range(1).write.format('parquet')
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Write a DataFrame into a Parquet file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.format('parquet').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.source = source
+        return self
+
+    def option(self, key: str, value: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds a output option for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        key : str
+            The key for the option to set.
+        value
+            The value for the option to set.
+
+        Examples
+        --------
+        >>> spark.range(1).write.option("key", "value")
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Specify the option 'nullValue' with writing a CSV file.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file with 'nullValue' option set to 'Hyukjin Kwon'.
+        ...     df = spark.createDataFrame([(100, None)], "age INT, name STRING")
+        ...     df.write.option("nullValue", "Hyukjin Kwon").mode("overwrite").format("csv").save(d)
+        ...
+        ...     # Read the CSV file as a DataFrame.
+        ...     spark.read.schema(df.schema).format('csv').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.options[key] = to_str(value)
+        return self
+
+    def options(self, **options: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds output options for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        **options : dict
+            The dictionary of string keys and prmitive-type values.

Review Comment:
   prmitive -> primitive



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] grundprinzip commented on a diff in pull request #38801: [SPARK-41317][CONNECT][PYTHON] Add basic support for DataFrameWriter

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #38801:
URL: https://github.com/apache/spark/pull/38801#discussion_r1034571206


##########
python/pyspark/sql/connect/readwriter.py:
##########
@@ -140,9 +161,891 @@ def load(
             self.option("path", path)
 
         plan = DataSource(format=self._format, schema=self._schema, options=self._options)
-        df = DataFrame.withPlan(plan, self._client)
-        return df
+        return self._df(plan)
+
+    def _df(self, plan: LogicalPlan) -> "DataFrame":
+        # The import is needed here to avoid circular import issues.
+        from pyspark.sql.connect.dataframe import DataFrame
+
+        return DataFrame.withPlan(plan, self._client)
 
     def table(self, tableName: str) -> "DataFrame":
-        df = DataFrame.withPlan(Read(tableName), self._client)
-        return df
+        return self._df(Read(tableName))
+
+
+class DataFrameWriter(OptionUtils):
+    """
+    Interface used to write a :class:`DataFrame` to external storage systems
+    (e.g. file systems, key-value stores, etc). Use :attr:`DataFrame.write`
+    to access this.
+
+    .. versionadded:: 3.4.0
+    """
+
+    def __init__(self, plan: "LogicalPlan", session: "SparkSession"):
+        self._df: "LogicalPlan" = plan
+        self._spark: "SparkSession" = session
+        self._write: "WriteOperation" = WriteOperation(self._df)
+
+    def mode(self, saveMode: Optional[str]) -> "DataFrameWriter":
+        """Specifies the behavior when data or table already exists.
+
+        Options include:
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Examples
+        --------
+        Raise an error when writing to an existing path.
+
+        >>> import tempfile

Review Comment:
   Doctests is a todo, yes



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] bjornjorgensen commented on a diff in pull request #38801: [SPARK-40539] [CONNECT] [PYTHON] Add basic support for DataFrameWriter

Posted by GitBox <gi...@apache.org>.
bjornjorgensen commented on code in PR #38801:
URL: https://github.com/apache/spark/pull/38801#discussion_r1032667339


##########
python/pyspark/sql/connect/writer.py:
##########
@@ -0,0 +1,922 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+
+from typing import Optional, Union, List, overload, Tuple, cast, Any
+from typing import TYPE_CHECKING
+
+from pyspark.sql.connect.plan import WriteOperation, LogicalPlan
+from pyspark.sql.types import StructType
+from pyspark.sql.utils import to_str
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect._typing import OptionalPrimitiveType
+    from pyspark.sql.connect.client import RemoteSparkSession
+
+
+PathOrPaths = Union[str, List[str]]
+TupleOrListOfString = Union[List[str], Tuple[str, ...]]
+
+
+class OptionUtils:
+    def _set_opts(
+        self,
+        schema: Optional[Union[StructType, str]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """
+        Set named options (filter out those the value is None)
+        """
+        if schema is not None:
+            self.schema(schema)  # type: ignore[attr-defined]
+        for k, v in options.items():
+            if v is not None:
+                self.option(k, v)  # type: ignore[attr-defined]
+
+
+class DataFrameWriter(OptionUtils):
+    """
+    Interface used to write a :class:`DataFrame` to external storage systems
+    (e.g. file systems, key-value stores, etc). Use :attr:`DataFrame.write`
+    to access this.
+
+    .. versionadded:: 3.4.0
+    """
+
+    def __init__(self, plan: "LogicalPlan", session: "RemoteSparkSession"):
+        self._df: "LogicalPlan" = plan
+        self._spark: "RemoteSparkSession" = session
+        self._write: "WriteOperation" = WriteOperation(self._df)
+
+    def mode(self, saveMode: Optional[str]) -> "DataFrameWriter":
+        """Specifies the behavior when data or table already exists.
+
+        Options include:
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Examples
+        --------
+        Raise an error when writing to an existing path.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     spark.createDataFrame(
+        ...         [{"age": 80, "name": "Xinrong Meng"}]
+        ...     ).write.mode("error").format("parquet").save(d)
+        Traceback (most recent call last):
+            ...
+        pyspark.sql.utils.AnalysisException: ...
+
+        Write a Parquet file back with various options, and read it back.
+
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Overwrite the path with a new Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 120, "name": "Takuya Ueshin"}]
+        ...     ).write.mode("append").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 140, "name": "Haejoon Lee"}]
+        ...     ).write.mode("ignore").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |120|Takuya Ueshin|
+        |100| Hyukjin Kwon|
+        +---+-------------+
+        """
+        # At the JVM side, the default value of mode is already set to "error".
+        # So, if the given saveMode is None, we will not call JVM-side's mode method.
+        if saveMode is not None:
+            self._write.mode = saveMode
+        return self
+
+    def format(self, source: str) -> "DataFrameWriter":
+        """Specifies the underlying output data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        source : str
+            string, name of the data source, e.g. 'json', 'parquet'.
+
+        Examples
+        --------
+        >>> spark.range(1).write.format('parquet')
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Write a DataFrame into a Parquet file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.format('parquet').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.source = source
+        return self
+
+    def option(self, key: str, value: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds a output option for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        key : str
+            The key for the option to set.
+        value
+            The value for the option to set.
+
+        Examples
+        --------
+        >>> spark.range(1).write.option("key", "value")
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Specify the option 'nullValue' with writing a CSV file.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file with 'nullValue' option set to 'Hyukjin Kwon'.
+        ...     df = spark.createDataFrame([(100, None)], "age INT, name STRING")
+        ...     df.write.option("nullValue", "Hyukjin Kwon").mode("overwrite").format("csv").save(d)
+        ...
+        ...     # Read the CSV file as a DataFrame.
+        ...     spark.read.schema(df.schema).format('csv').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.options[key] = to_str(value)
+        return self
+
+    def options(self, **options: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds output options for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        **options : dict
+            The dictionary of string keys and prmitive-type values.
+
+        Examples
+        --------
+        >>> spark.range(1).write.option("key", "value")
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Specify the option 'nullValue' and 'header' with writing a CSV file.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file with 'nullValue' option set to 'Hyukjin Kwon',
+        ...     # and 'header' option set to `True`.
+        ...     df = spark.createDataFrame([(100, "Hyukjin Kwon")], ["age", "name"])
+        ...     df.write.options(nullValue="Hyukjin Kwon", header=True).mode(
+        ...         "overwrite").format("csv").save(d)
+        ...
+        ...     # Read the CSV file as a DataFrame.
+        ...     spark.read.option("header", True).format('csv').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        for k in options:
+            self._write.options[k] = to_str(options[k])
+        return self
+
+    @overload
+    def partitionBy(self, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def partitionBy(self, *cols: List[str]) -> "DataFrameWriter":
+        ...
+
+    def partitionBy(self, *cols: Union[str, List[str]]) -> "DataFrameWriter":
+        """Partitions the output by the given columns on the file system.
+
+        If specified, the output is laid out on the file system similar
+        to Hive's partitioning scheme.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        cols : str or list
+            name of columns
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a partitioned manner, and read it back.
+
+        >>> import tempfile
+        >>> import os
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file in a partitioned manner.
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}, {"age": 120, "name": "Ruifeng Zheng"}]
+        ...     ).write.partitionBy("name").mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).sort("age").show()
+        ...
+        ...     # Read one partition as a DataFrame.
+        ...     spark.read.parquet(f"{d}{os.path.sep}name=Hyukjin Kwon").show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |100| Hyukjin Kwon|
+        |120|Ruifeng Zheng|
+        +---+-------------+
+        +---+
+        |age|
+        +---+
+        |100|
+        +---+
+        """
+        if len(cols) == 1 and isinstance(cols[0], (list, tuple)):
+            cols = cols[0]  # type: ignore[assignment]
+
+        self._write.partitioning_cols = cast(List[str], cols)
+        return self
+
+    @overload
+    def bucketBy(self, numBuckets: int, col: str, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def bucketBy(self, numBuckets: int, col: TupleOrListOfString) -> "DataFrameWriter":
+        ...
+
+    def bucketBy(
+        self, numBuckets: int, col: Union[str, TupleOrListOfString], *cols: Optional[str]
+    ) -> "DataFrameWriter":
+        """Buckets the output by the given columns. If specified,
+        the output is laid out on the file system similar to Hive's bucketing scheme,
+        but with a different bucket hash function and is not compatible with Hive's bucketing.
+
+        .. versionadded:: 2.3.0
+
+        Parameters
+        ----------
+        numBuckets : int
+            the number of buckets to save
+        col : str, list or tuple
+            a name of a column, or a list of names.
+        cols : str
+            additional names (optional). If `col` is a list it should be empty.
+
+        Notes
+        -----
+        Applicable for file-based data sources in combination with
+        :py:meth:`DataFrameWriter.saveAsTable`.
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a buckted manner, and read it back.
+
+        >>> from pyspark.sql.functions import input_file_name
+        >>> # Write a DataFrame into a Parquet file in a bucketed manner.
+        ... _ = spark.sql("DROP TABLE IF EXISTS bucketed_table")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.bucketBy(2, "name").mode("overwrite").saveAsTable("bucketed_table")
+        >>> # Read the Parquet file as a DataFrame.
+        ... spark.read.table("bucketed_table").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE bucketed_table")
+        """
+        if not isinstance(numBuckets, int):
+            raise TypeError("numBuckets should be an int, got {0}.".format(type(numBuckets)))
+
+        if isinstance(col, (list, tuple)):
+            if cols:
+                raise ValueError("col is a {0} but cols are not empty".format(type(col)))
+
+            col, cols = col[0], col[1:]  # type: ignore[assignment]
+
+        if not all(isinstance(c, str) for c in cols) or not (isinstance(col, str)):
+            raise TypeError("all names should be `str`")
+
+        self._write.num_buckets = numBuckets
+        self._write.bucket_cols = cast(List[str], cols)
+        return self
+
+    @overload
+    def sortBy(self, col: str, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def sortBy(self, col: TupleOrListOfString) -> "DataFrameWriter":
+        ...
+
+    def sortBy(
+        self, col: Union[str, TupleOrListOfString], *cols: Optional[str]
+    ) -> "DataFrameWriter":
+        """Sorts the output in each bucket by the given columns on the file system.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        col : str, tuple or list
+            a name of a column, or a list of names.
+        cols : str
+            additional names (optional). If `col` is a list it should be empty.
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a sorted-buckted manner, and read it back.
+
+        >>> from pyspark.sql.functions import input_file_name
+        >>> # Write a DataFrame into a Parquet file in a sorted-bucketed manner.
+        ... _ = spark.sql("DROP TABLE IF EXISTS sorted_bucketed_table")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.bucketBy(1, "name").sortBy("age").mode(
+        ...     "overwrite").saveAsTable("sorted_bucketed_table")
+        >>> # Read the Parquet file as a DataFrame.
+        ... spark.read.table("sorted_bucketed_table").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE sorted_bucketed_table")
+        """
+        if isinstance(col, (list, tuple)):
+            if cols:
+                raise ValueError("col is a {0} but cols are not empty".format(type(col)))
+
+            col, cols = col[0], col[1:]  # type: ignore[assignment]
+
+        if not all(isinstance(c, str) for c in cols) or not (isinstance(col, str)):
+            raise TypeError("all names should be `str`")
+
+        self._write.sort_cols = cast(List[str], cols)
+        return self
+
+    def save(
+        self,
+        path: Optional[str] = None,
+        format: Optional[str] = None,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """Saves the contents of the :class:`DataFrame` to a data source.
+
+        The data source is specified by the ``format`` and a set of ``options``.
+        If ``format`` is not specified, the default data source configured by
+        ``spark.sql.sources.default`` will be used.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str, optional
+            the path in a Hadoop supported file system
+        format : str, optional
+            the format used to save
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+        partitionBy : list, optional
+            names of partitioning columns
+        **options : dict
+            all other string options
+
+        Examples
+        --------
+        Write a DataFrame into a JSON file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a JSON file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("json").save(d)
+        ...
+        ...     # Read the JSON file as a DataFrame.
+        ...     spark.read.format('json').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode).options(**options)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        if format is not None:
+            self.format(format)
+        self._write.path = path
+        self._spark.execute_command(self._write.command(self._spark))
+
+    def insertInto(self, tableName: str, overwrite: Optional[bool] = None) -> None:
+        """Inserts the content of the :class:`DataFrame` to the specified table.
+
+        It requires that the schema of the :class:`DataFrame` is the same as the
+        schema of the table.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        overwrite : bool, optional
+            If true, overwrites existing data. Disabled by default
+
+        Notes
+        -----
+        Unlike :meth:`DataFrameWriter.saveAsTable`, :meth:`DataFrameWriter.insertInto` ignores
+        the column names and just uses position-based resolution.
+
+        Examples
+        --------
+        >>> _ = spark.sql("DROP TABLE IF EXISTS tblA")
+        >>> df = spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... )
+        >>> df.write.saveAsTable("tblA")
+
+        Insert the data into 'tblA' table but with different column names.
+
+        >>> df.selectExpr("age AS col1", "name AS col2").write.insertInto("tblA")
+        >>> spark.read.table("tblA").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE tblA")
+        """
+        if overwrite is not None:
+            self.mode("overwrite" if overwrite else "append")
+        self.saveAsTable(tableName)
+
+    def saveAsTable(
+        self,
+        name: str,
+        format: Optional[str] = None,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` as the specified table.
+
+        In the case the table already exists, behavior of this function depends on the
+        save mode, specified by the `mode` function (default to throwing an exception).
+        When `mode` is `Overwrite`, the schema of the :class:`DataFrame` does not need to be
+        the same as that of the existing table.
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Notes
+        -----
+        When `mode` is `Append`, if there is an existing table, we will use the format and
+        options of the existing table. The column order in the schema of the :class:`DataFrame`
+        doesn't need to be same as that of the existing table. Unlike
+        :meth:`DataFrameWriter.insertInto`, :meth:`DataFrameWriter.saveAsTable` will use the
+        column names to find the correct column positions.
+
+        Parameters
+        ----------
+        name : str
+            the table name
+        format : str, optional
+            the format used to save
+        mode : str, optional
+            one of `append`, `overwrite`, `error`, `errorifexists`, `ignore` \
+            (default: error)
+        partitionBy : str or list
+            names of partitioning columns
+        **options : dict
+            all other string options
+
+        Examples
+        --------
+        Creates a table from a DataFrame, and read it back.
+
+        >>> _ = spark.sql("DROP TABLE IF EXISTS tblA")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.saveAsTable("tblA")
+        >>> spark.read.table("tblA").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE tblA")
+        """
+        self.mode(mode).options(**options)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        if format is not None:
+            self.format(format)
+        self._write.table_name = name
+        self._spark.execute_command(self._write.command(self._spark))
+
+    def json(
+        self,
+        path: str,
+        mode: Optional[str] = None,
+        compression: Optional[str] = None,
+        dateFormat: Optional[str] = None,
+        timestampFormat: Optional[str] = None,
+        lineSep: Optional[str] = None,
+        encoding: Optional[str] = None,
+        ignoreNullFields: Optional[Union[bool, str]] = None,
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` in JSON format
+        (`JSON Lines text format or newline-delimited JSON <http://jsonlines.org/>`_) at the
+        specified path.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-json.html#data-source-option>`_
+            in the version you use.

Review Comment:
   in -> for



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] bjornjorgensen commented on a diff in pull request #38801: [SPARK-40539] [CONNECT] [PYTHON] Add basic support for DataFrameWriter

Posted by GitBox <gi...@apache.org>.
bjornjorgensen commented on code in PR #38801:
URL: https://github.com/apache/spark/pull/38801#discussion_r1032662076


##########
python/pyspark/sql/connect/writer.py:
##########
@@ -0,0 +1,922 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+
+from typing import Optional, Union, List, overload, Tuple, cast, Any
+from typing import TYPE_CHECKING
+
+from pyspark.sql.connect.plan import WriteOperation, LogicalPlan
+from pyspark.sql.types import StructType
+from pyspark.sql.utils import to_str
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect._typing import OptionalPrimitiveType
+    from pyspark.sql.connect.client import RemoteSparkSession
+
+
+PathOrPaths = Union[str, List[str]]
+TupleOrListOfString = Union[List[str], Tuple[str, ...]]
+
+
+class OptionUtils:
+    def _set_opts(
+        self,
+        schema: Optional[Union[StructType, str]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """
+        Set named options (filter out those the value is None)
+        """
+        if schema is not None:
+            self.schema(schema)  # type: ignore[attr-defined]
+        for k, v in options.items():
+            if v is not None:
+                self.option(k, v)  # type: ignore[attr-defined]
+
+
+class DataFrameWriter(OptionUtils):
+    """
+    Interface used to write a :class:`DataFrame` to external storage systems
+    (e.g. file systems, key-value stores, etc). Use :attr:`DataFrame.write`
+    to access this.
+
+    .. versionadded:: 3.4.0
+    """
+
+    def __init__(self, plan: "LogicalPlan", session: "RemoteSparkSession"):
+        self._df: "LogicalPlan" = plan
+        self._spark: "RemoteSparkSession" = session
+        self._write: "WriteOperation" = WriteOperation(self._df)
+
+    def mode(self, saveMode: Optional[str]) -> "DataFrameWriter":
+        """Specifies the behavior when data or table already exists.
+
+        Options include:
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Examples
+        --------
+        Raise an error when writing to an existing path.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     spark.createDataFrame(
+        ...         [{"age": 80, "name": "Xinrong Meng"}]
+        ...     ).write.mode("error").format("parquet").save(d)
+        Traceback (most recent call last):
+            ...
+        pyspark.sql.utils.AnalysisException: ...
+
+        Write a Parquet file back with various options, and read it back.
+
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Overwrite the path with a new Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 120, "name": "Takuya Ueshin"}]
+        ...     ).write.mode("append").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 140, "name": "Haejoon Lee"}]
+        ...     ).write.mode("ignore").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |120|Takuya Ueshin|
+        |100| Hyukjin Kwon|
+        +---+-------------+
+        """
+        # At the JVM side, the default value of mode is already set to "error".
+        # So, if the given saveMode is None, we will not call JVM-side's mode method.
+        if saveMode is not None:
+            self._write.mode = saveMode
+        return self
+
+    def format(self, source: str) -> "DataFrameWriter":
+        """Specifies the underlying output data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        source : str
+            string, name of the data source, e.g. 'json', 'parquet'.
+
+        Examples
+        --------
+        >>> spark.range(1).write.format('parquet')
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Write a DataFrame into a Parquet file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.format('parquet').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.source = source
+        return self
+
+    def option(self, key: str, value: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds a output option for the underlying data source.

Review Comment:
   a -> an



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] bjornjorgensen commented on a diff in pull request #38801: [SPARK-40539] [CONNECT] [PYTHON] Add basic support for DataFrameWriter

Posted by GitBox <gi...@apache.org>.
bjornjorgensen commented on code in PR #38801:
URL: https://github.com/apache/spark/pull/38801#discussion_r1032813470


##########
python/pyspark/sql/connect/writer.py:
##########
@@ -0,0 +1,922 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+
+from typing import Optional, Union, List, overload, Tuple, cast, Any
+from typing import TYPE_CHECKING
+
+from pyspark.sql.connect.plan import WriteOperation, LogicalPlan
+from pyspark.sql.types import StructType
+from pyspark.sql.utils import to_str
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect._typing import OptionalPrimitiveType
+    from pyspark.sql.connect.client import RemoteSparkSession
+
+
+PathOrPaths = Union[str, List[str]]
+TupleOrListOfString = Union[List[str], Tuple[str, ...]]
+
+
+class OptionUtils:
+    def _set_opts(
+        self,
+        schema: Optional[Union[StructType, str]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """
+        Set named options (filter out those the value is None)
+        """
+        if schema is not None:
+            self.schema(schema)  # type: ignore[attr-defined]
+        for k, v in options.items():
+            if v is not None:
+                self.option(k, v)  # type: ignore[attr-defined]
+
+
+class DataFrameWriter(OptionUtils):
+    """
+    Interface used to write a :class:`DataFrame` to external storage systems
+    (e.g. file systems, key-value stores, etc). Use :attr:`DataFrame.write`
+    to access this.
+
+    .. versionadded:: 3.4.0
+    """
+
+    def __init__(self, plan: "LogicalPlan", session: "RemoteSparkSession"):
+        self._df: "LogicalPlan" = plan
+        self._spark: "RemoteSparkSession" = session
+        self._write: "WriteOperation" = WriteOperation(self._df)
+
+    def mode(self, saveMode: Optional[str]) -> "DataFrameWriter":
+        """Specifies the behavior when data or table already exists.
+
+        Options include:
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Examples
+        --------
+        Raise an error when writing to an existing path.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     spark.createDataFrame(
+        ...         [{"age": 80, "name": "Xinrong Meng"}]
+        ...     ).write.mode("error").format("parquet").save(d)
+        Traceback (most recent call last):
+            ...
+        pyspark.sql.utils.AnalysisException: ...
+
+        Write a Parquet file back with various options, and read it back.
+
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Overwrite the path with a new Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 120, "name": "Takuya Ueshin"}]
+        ...     ).write.mode("append").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 140, "name": "Haejoon Lee"}]
+        ...     ).write.mode("ignore").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |120|Takuya Ueshin|
+        |100| Hyukjin Kwon|
+        +---+-------------+
+        """
+        # At the JVM side, the default value of mode is already set to "error".
+        # So, if the given saveMode is None, we will not call JVM-side's mode method.
+        if saveMode is not None:
+            self._write.mode = saveMode
+        return self
+
+    def format(self, source: str) -> "DataFrameWriter":
+        """Specifies the underlying output data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        source : str
+            string, name of the data source, e.g. 'json', 'parquet'.
+
+        Examples
+        --------
+        >>> spark.range(1).write.format('parquet')
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Write a DataFrame into a Parquet file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.format('parquet').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.source = source
+        return self
+
+    def option(self, key: str, value: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds a output option for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        key : str
+            The key for the option to set.
+        value
+            The value for the option to set.
+
+        Examples
+        --------
+        >>> spark.range(1).write.option("key", "value")
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Specify the option 'nullValue' with writing a CSV file.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file with 'nullValue' option set to 'Hyukjin Kwon'.
+        ...     df = spark.createDataFrame([(100, None)], "age INT, name STRING")
+        ...     df.write.option("nullValue", "Hyukjin Kwon").mode("overwrite").format("csv").save(d)
+        ...
+        ...     # Read the CSV file as a DataFrame.
+        ...     spark.read.schema(df.schema).format('csv').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.options[key] = to_str(value)
+        return self
+
+    def options(self, **options: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds output options for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        **options : dict
+            The dictionary of string keys and prmitive-type values.

Review Comment:
   Yes, I have upgraded my comments now.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] bjornjorgensen commented on a diff in pull request #38801: [SPARK-40539] [CONNECT] [PYTHON] Add basic support for DataFrameWriter

Posted by GitBox <gi...@apache.org>.
bjornjorgensen commented on code in PR #38801:
URL: https://github.com/apache/spark/pull/38801#discussion_r1032913049


##########
python/pyspark/sql/connect/writer.py:
##########
@@ -0,0 +1,926 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+
+from typing import Optional, Union, List, overload, Tuple, cast, Any
+from typing import TYPE_CHECKING
+
+from pyspark.sql.connect.plan import WriteOperation, LogicalPlan
+from pyspark.sql.types import StructType
+from pyspark.sql.utils import to_str
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect._typing import OptionalPrimitiveType
+    from pyspark.sql.connect.session import SparkSession
+
+
+PathOrPaths = Union[str, List[str]]
+TupleOrListOfString = Union[List[str], Tuple[str, ...]]
+
+
+class OptionUtils:
+    def _set_opts(
+        self,
+        schema: Optional[Union[StructType, str]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """
+        Set named options (filter out those the value is None)
+        """
+        if schema is not None:
+            self.schema(schema)  # type: ignore[attr-defined]
+        for k, v in options.items():
+            if v is not None:
+                self.option(k, v)  # type: ignore[attr-defined]
+
+
+class DataFrameWriter(OptionUtils):
+    """
+    Interface used to write a :class:`DataFrame` to external storage systems
+    (e.g. file systems, key-value stores, etc). Use :attr:`DataFrame.write`
+    to access this.
+
+    .. versionadded:: 3.4.0
+    """
+
+    def __init__(self, plan: "LogicalPlan", session: "SparkSession"):
+        self._df: "LogicalPlan" = plan
+        self._spark: "SparkSession" = session
+        self._write: "WriteOperation" = WriteOperation(self._df)
+
+    def mode(self, saveMode: Optional[str]) -> "DataFrameWriter":
+        """Specifies the behavior when data or table already exists.
+
+        Options include:
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Examples
+        --------
+        Raise an error when writing to an existing path.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     spark.createDataFrame(
+        ...         [{"age": 80, "name": "Xinrong Meng"}]
+        ...     ).write.mode("error").format("parquet").save(d)
+        Traceback (most recent call last):
+            ...
+        pyspark.sql.utils.AnalysisException: ...
+
+        Write a Parquet file back with various options, and read it back.
+
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Overwrite the path with a new Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 120, "name": "Takuya Ueshin"}]
+        ...     ).write.mode("append").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 140, "name": "Haejoon Lee"}]
+        ...     ).write.mode("ignore").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |120|Takuya Ueshin|
+        |100| Hyukjin Kwon|
+        +---+-------------+
+        """
+        # At the JVM side, the default value of mode is already set to "error".
+        # So, if the given saveMode is None, we will not call JVM-side's mode method.
+        if saveMode is not None:
+            self._write.mode = saveMode
+        return self
+
+    def format(self, source: str) -> "DataFrameWriter":
+        """Specifies the underlying output data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        source : str
+            string, name of the data source, e.g. 'json', 'parquet'.
+
+        Examples
+        --------
+        >>> spark.range(1).write.format('parquet')
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Write a DataFrame into a Parquet file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.format('parquet').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.source = source
+        return self
+
+    def option(self, key: str, value: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds an output option for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        key : str
+            The key for the option to set.
+        value
+            The value for the option to set.
+
+        Examples
+        --------
+        >>> spark.range(1).write.option("key", "value")
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Specify the option 'nullValue' with writing a CSV file.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file with 'nullValue' option set to 'Hyukjin Kwon'.
+        ...     df = spark.createDataFrame([(100, None)], "age INT, name STRING")
+        ...     df.write.option("nullValue", "Hyukjin Kwon").mode("overwrite").format("csv").save(d)
+        ...
+        ...     # Read the CSV file as a DataFrame.
+        ...     spark.read.schema(df.schema).format('csv').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.options[key] = to_str(value)
+        return self
+
+    def options(self, **options: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds output options for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        **options : dict
+            The dictionary of string keys and primitive-type values.
+
+        Examples
+        --------
+        >>> spark.range(1).write.option("key", "value")
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Specify the option 'nullValue' and 'header' with writing a CSV file.
+        >>> from pyspark.sql.types import StructType,StructField, StringType, IntegerType
+        >>> schema = StructType([
+        ...     StructField("age",IntegerType(),True),
+        ...     StructField("name",StringType(),True),
+        ...])
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file with 'nullValue' option set to 'Hyukjin Kwon',
+        ...     # and 'header' option set to `True`.
+        ...     df = spark.createDataFrame([(100, None], schema=schema)
+        ...     df.write.options(nullValue="Hyukjin Kwon", header=True).mode(
+        ...         "overwrite").format("csv").save(d)
+        ...
+        ...     # Read the CSV file as a DataFrame.
+        ...     spark.read.option("header", True).format('csv').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        for k in options:
+            self._write.options[k] = to_str(options[k])
+        return self
+
+    @overload
+    def partitionBy(self, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def partitionBy(self, *cols: List[str]) -> "DataFrameWriter":
+        ...
+
+    def partitionBy(self, *cols: Union[str, List[str]]) -> "DataFrameWriter":
+        """Partitions the output by the given columns on the file system.
+
+        If specified, the output is laid out on the file system similar
+        to Hive's partitioning scheme.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        cols : str or list
+            name of columns
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a partitioned manner, and read it back.
+
+        >>> import tempfile
+        >>> import os
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file in a partitioned manner.
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}, {"age": 120, "name": "Ruifeng Zheng"}]
+        ...     ).write.partitionBy("name").mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).sort("age").show()
+        ...
+        ...     # Read one partition as a DataFrame.
+        ...     spark.read.parquet(f"{d}{os.path.sep}name=Hyukjin Kwon").show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |100| Hyukjin Kwon|
+        |120|Ruifeng Zheng|
+        +---+-------------+
+        +---+
+        |age|
+        +---+
+        |100|
+        +---+
+        """
+        if len(cols) == 1 and isinstance(cols[0], (list, tuple)):
+            cols = cols[0]  # type: ignore[assignment]
+
+        self._write.partitioning_cols = cast(List[str], cols)
+        return self
+
+    @overload
+    def bucketBy(self, numBuckets: int, col: str, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def bucketBy(self, numBuckets: int, col: TupleOrListOfString) -> "DataFrameWriter":
+        ...
+
+    def bucketBy(
+        self, numBuckets: int, col: Union[str, TupleOrListOfString], *cols: Optional[str]
+    ) -> "DataFrameWriter":
+        """Buckets the output by the given columns. If specified,
+        the output is laid out on the file system similar to Hive's bucketing scheme,
+        but with a different bucket hash function and is not compatible with Hive's bucketing.
+
+        .. versionadded:: 2.3.0
+
+        Parameters
+        ----------
+        numBuckets : int
+            the number of buckets to save
+        col : str, list or tuple
+            a name of a column, or a list of names.
+        cols : str
+            additional names (optional). If `col` is a list it should be empty.
+
+        Notes
+        -----
+        Applicable for file-based data sources in combination with
+        :py:meth:`DataFrameWriter.saveAsTable`.
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a buckted manner, and read it back.
+
+        >>> from pyspark.sql.functions import input_file_name
+        >>> # Write a DataFrame into a Parquet file in a bucketed manner.
+        ... _ = spark.sql("DROP TABLE IF EXISTS bucketed_table")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.bucketBy(2, "name").mode("overwrite").saveAsTable("bucketed_table")
+        >>> # Read the Parquet file as a DataFrame.
+        ... spark.read.table("bucketed_table").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE bucketed_table")
+        """
+        if not isinstance(numBuckets, int):
+            raise TypeError("numBuckets should be an int, got {0}.".format(type(numBuckets)))
+
+        if isinstance(col, (list, tuple)):
+            if cols:
+                raise ValueError("col is a {0} but cols are not empty".format(type(col)))
+
+            col, cols = col[0], col[1:]  # type: ignore[assignment]
+
+        if not all(isinstance(c, str) for c in cols) or not (isinstance(col, str)):
+            raise TypeError("all names should be `str`")
+
+        self._write.num_buckets = numBuckets
+        self._write.bucket_cols = cast(List[str], cols)
+        return self
+
+    @overload
+    def sortBy(self, col: str, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def sortBy(self, col: TupleOrListOfString) -> "DataFrameWriter":
+        ...
+
+    def sortBy(
+        self, col: Union[str, TupleOrListOfString], *cols: Optional[str]
+    ) -> "DataFrameWriter":
+        """Sorts the output in each bucket by the given columns on the file system.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        col : str, tuple or list
+            a name of a column, or a list of names.
+        cols : str
+            additional names (optional). If `col` is a list it should be empty.
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a sorted-buckted manner, and read it back.
+
+        >>> from pyspark.sql.functions import input_file_name
+        >>> # Write a DataFrame into a Parquet file in a sorted-bucketed manner.
+        ... _ = spark.sql("DROP TABLE IF EXISTS sorted_bucketed_table")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.bucketBy(1, "name").sortBy("age").mode(
+        ...     "overwrite").saveAsTable("sorted_bucketed_table")
+        >>> # Read the Parquet file as a DataFrame.
+        ... spark.read.table("sorted_bucketed_table").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE sorted_bucketed_table")
+        """
+        if isinstance(col, (list, tuple)):
+            if cols:
+                raise ValueError("col is a {0} but cols are not empty".format(type(col)))
+
+            col, cols = col[0], col[1:]  # type: ignore[assignment]
+
+        if not all(isinstance(c, str) for c in cols) or not (isinstance(col, str)):
+            raise TypeError("all names should be `str`")
+
+        self._write.sort_cols = cast(List[str], cols)
+        return self
+
+    def save(
+        self,
+        path: Optional[str] = None,
+        format: Optional[str] = None,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """Saves the contents of the :class:`DataFrame` to a data source.
+
+        The data source is specified by the ``format`` and a set of ``options``.
+        If ``format`` is not specified, the default data source configured by
+        ``spark.sql.sources.default`` will be used.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str, optional
+            the path in a Hadoop supported file system
+        format : str, optional
+            the format used to save
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+        partitionBy : list, optional
+            names of partitioning columns
+        **options : dict
+            all other string options
+
+        Examples
+        --------
+        Write a DataFrame into a JSON file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a JSON file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("json").save(d)
+        ...
+        ...     # Read the JSON file as a DataFrame.
+        ...     spark.read.format('json').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode).options(**options)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        if format is not None:
+            self.format(format)
+        self._write.path = path
+        self._spark.client.execute_command(self._write.command(self._spark.client))
+
+    def insertInto(self, tableName: str, overwrite: Optional[bool] = None) -> None:
+        """Inserts the content of the :class:`DataFrame` to the specified table.
+
+        It requires that the schema of the :class:`DataFrame` is the same as the
+        schema of the table.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        overwrite : bool, optional
+            If true, overwrites existing data. Disabled by default
+
+        Notes
+        -----
+        Unlike :meth:`DataFrameWriter.saveAsTable`, :meth:`DataFrameWriter.insertInto` ignores
+        the column names and just uses position-based resolution.
+
+        Examples
+        --------
+        >>> _ = spark.sql("DROP TABLE IF EXISTS tblA")
+        >>> df = spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... )
+        >>> df.write.saveAsTable("tblA")
+
+        Insert the data into 'tblA' table but with different column names.
+
+        >>> df.selectExpr("age AS col1", "name AS col2").write.insertInto("tblA")
+        >>> spark.read.table("tblA").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE tblA")
+        """
+        if overwrite is not None:
+            self.mode("overwrite" if overwrite else "append")
+        self.saveAsTable(tableName)
+
+    def saveAsTable(
+        self,
+        name: str,
+        format: Optional[str] = None,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` as the specified table.
+
+        In the case the table already exists, behavior of this function depends on the
+        save mode, specified by the `mode` function (default to throwing an exception).
+        When `mode` is `Overwrite`, the schema of the :class:`DataFrame` does not need to be
+        the same as that of the existing table.
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Notes
+        -----
+        When `mode` is `Append`, if there is an existing table, we will use the format and
+        options of the existing table. The column order in the schema of the :class:`DataFrame`
+        doesn't need to be the same as that of the existing table. Unlike
+        :meth:`DataFrameWriter.insertInto`, :meth:`DataFrameWriter.saveAsTable` will use the
+        column names to find the correct column positions.
+
+        Parameters
+        ----------
+        name : str
+            the table name
+        format : str, optional
+            the format used to save
+        mode : str, optional
+            one of `append`, `overwrite`, `error`, `errorifexists`, `ignore` \
+            (default: error)
+        partitionBy : str or list
+            names of partitioning columns
+        **options : dict
+            all other string options
+
+        Examples
+        --------
+        Creates a table from a DataFrame, and read it back.
+
+        >>> _ = spark.sql("DROP TABLE IF EXISTS tblA")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.saveAsTable("tblA")
+        >>> spark.read.table("tblA").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE tblA")
+        """
+        self.mode(mode).options(**options)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        if format is not None:
+            self.format(format)
+        self._write.table_name = name
+        self._spark.client.execute_command(self._write.command(self._spark.client))
+
+    def json(
+        self,
+        path: str,
+        mode: Optional[str] = None,
+        compression: Optional[str] = None,
+        dateFormat: Optional[str] = None,
+        timestampFormat: Optional[str] = None,
+        lineSep: Optional[str] = None,
+        encoding: Optional[str] = None,
+        ignoreNullFields: Optional[Union[bool, str]] = None,
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` in JSON format
+        (`JSON Lines text format or newline-delimited JSON <http://jsonlines.org/>`_) at the
+        specified path.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-json.html#data-source-option>`_
+            for the version you use.
+
+            .. # noqa
+
+        Examples
+        --------
+        Write a DataFrame into a JSON file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a JSON file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.json(d, mode="overwrite")
+        ...
+        ...     # Read the JSON file as a DataFrame.
+        ...     spark.read.format("json").load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode)
+        self._set_opts(
+            compression=compression,
+            dateFormat=dateFormat,
+            timestampFormat=timestampFormat,
+            lineSep=lineSep,
+            encoding=encoding,
+            ignoreNullFields=ignoreNullFields,
+        )
+        self.format("json").save(path)
+
+    def parquet(
+        self,
+        path: str,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        compression: Optional[str] = None,
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` in Parquet format at the specified path.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+        partitionBy : str or list, optional
+            names of partitioning columns
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#data-source-option>`_
+            in the version you use.
+
+            .. # noqa
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.parquet(d, mode="overwrite")
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.format("parquet").load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        self.option("compression", compression)
+        self.format("parquet").save(path)
+
+    def text(
+        self, path: str, compression: Optional[str] = None, lineSep: Optional[str] = None
+    ) -> None:
+        """Saves the content of the DataFrame in a text file at the specified path.
+        The text files will be encoded as UTF-8.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-text.html#data-source-option>`_
+            in the version you use.
+
+            .. # noqa
+
+        Notes
+        -----
+        The DataFrame must have only one column that is of string type.
+        Each row becomes a new line in the output file.
+
+        Examples
+        --------
+        Write a DataFrame into a text file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a text file
+        ...     df = spark.createDataFrame([("a",), ("b",), ("c",)], schema=["alphabets"])
+        ...     df.write.mode("overwrite").text(d)
+        ...
+        ...     # Read the text file as a DataFrame.
+        ...     spark.read.schema(df.schema).format("text").load(d).sort("alphabets").show()
+        +---------+
+        |alphabets|
+        +---------+
+        |        a|
+        |        b|
+        |        c|
+        +---------+
+        """
+        self._set_opts(compression=compression, lineSep=lineSep)
+        self.format("text").save(path)
+
+    def csv(
+        self,
+        path: str,
+        mode: Optional[str] = None,
+        compression: Optional[str] = None,
+        sep: Optional[str] = None,
+        quote: Optional[str] = None,
+        escape: Optional[str] = None,
+        header: Optional[Union[bool, str]] = None,
+        nullValue: Optional[str] = None,
+        escapeQuotes: Optional[Union[bool, str]] = None,
+        quoteAll: Optional[Union[bool, str]] = None,
+        dateFormat: Optional[str] = None,
+        timestampFormat: Optional[str] = None,
+        ignoreLeadingWhiteSpace: Optional[Union[bool, str]] = None,
+        ignoreTrailingWhiteSpace: Optional[Union[bool, str]] = None,
+        charToEscapeQuoteEscaping: Optional[str] = None,
+        encoding: Optional[str] = None,
+        emptyValue: Optional[str] = None,
+        lineSep: Optional[str] = None,
+    ) -> None:
+        r"""Saves the content of the :class:`DataFrame` in CSV format at the specified path.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-csv.html#data-source-option>`_
+            in the version you use.

Review Comment:
   in -> for



##########
python/pyspark/sql/connect/writer.py:
##########
@@ -0,0 +1,926 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+
+from typing import Optional, Union, List, overload, Tuple, cast, Any
+from typing import TYPE_CHECKING
+
+from pyspark.sql.connect.plan import WriteOperation, LogicalPlan
+from pyspark.sql.types import StructType
+from pyspark.sql.utils import to_str
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect._typing import OptionalPrimitiveType
+    from pyspark.sql.connect.session import SparkSession
+
+
+PathOrPaths = Union[str, List[str]]
+TupleOrListOfString = Union[List[str], Tuple[str, ...]]
+
+
+class OptionUtils:
+    def _set_opts(
+        self,
+        schema: Optional[Union[StructType, str]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """
+        Set named options (filter out those the value is None)
+        """
+        if schema is not None:
+            self.schema(schema)  # type: ignore[attr-defined]
+        for k, v in options.items():
+            if v is not None:
+                self.option(k, v)  # type: ignore[attr-defined]
+
+
+class DataFrameWriter(OptionUtils):
+    """
+    Interface used to write a :class:`DataFrame` to external storage systems
+    (e.g. file systems, key-value stores, etc). Use :attr:`DataFrame.write`
+    to access this.
+
+    .. versionadded:: 3.4.0
+    """
+
+    def __init__(self, plan: "LogicalPlan", session: "SparkSession"):
+        self._df: "LogicalPlan" = plan
+        self._spark: "SparkSession" = session
+        self._write: "WriteOperation" = WriteOperation(self._df)
+
+    def mode(self, saveMode: Optional[str]) -> "DataFrameWriter":
+        """Specifies the behavior when data or table already exists.
+
+        Options include:
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Examples
+        --------
+        Raise an error when writing to an existing path.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     spark.createDataFrame(
+        ...         [{"age": 80, "name": "Xinrong Meng"}]
+        ...     ).write.mode("error").format("parquet").save(d)
+        Traceback (most recent call last):
+            ...
+        pyspark.sql.utils.AnalysisException: ...
+
+        Write a Parquet file back with various options, and read it back.
+
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Overwrite the path with a new Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 120, "name": "Takuya Ueshin"}]
+        ...     ).write.mode("append").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 140, "name": "Haejoon Lee"}]
+        ...     ).write.mode("ignore").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |120|Takuya Ueshin|
+        |100| Hyukjin Kwon|
+        +---+-------------+
+        """
+        # At the JVM side, the default value of mode is already set to "error".
+        # So, if the given saveMode is None, we will not call JVM-side's mode method.
+        if saveMode is not None:
+            self._write.mode = saveMode
+        return self
+
+    def format(self, source: str) -> "DataFrameWriter":
+        """Specifies the underlying output data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        source : str
+            string, name of the data source, e.g. 'json', 'parquet'.
+
+        Examples
+        --------
+        >>> spark.range(1).write.format('parquet')
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Write a DataFrame into a Parquet file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.format('parquet').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.source = source
+        return self
+
+    def option(self, key: str, value: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds an output option for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        key : str
+            The key for the option to set.
+        value
+            The value for the option to set.
+
+        Examples
+        --------
+        >>> spark.range(1).write.option("key", "value")
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Specify the option 'nullValue' with writing a CSV file.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file with 'nullValue' option set to 'Hyukjin Kwon'.
+        ...     df = spark.createDataFrame([(100, None)], "age INT, name STRING")
+        ...     df.write.option("nullValue", "Hyukjin Kwon").mode("overwrite").format("csv").save(d)
+        ...
+        ...     # Read the CSV file as a DataFrame.
+        ...     spark.read.schema(df.schema).format('csv').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.options[key] = to_str(value)
+        return self
+
+    def options(self, **options: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds output options for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        **options : dict
+            The dictionary of string keys and primitive-type values.
+
+        Examples
+        --------
+        >>> spark.range(1).write.option("key", "value")
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Specify the option 'nullValue' and 'header' with writing a CSV file.
+        >>> from pyspark.sql.types import StructType,StructField, StringType, IntegerType
+        >>> schema = StructType([
+        ...     StructField("age",IntegerType(),True),
+        ...     StructField("name",StringType(),True),
+        ...])
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file with 'nullValue' option set to 'Hyukjin Kwon',
+        ...     # and 'header' option set to `True`.
+        ...     df = spark.createDataFrame([(100, None], schema=schema)
+        ...     df.write.options(nullValue="Hyukjin Kwon", header=True).mode(
+        ...         "overwrite").format("csv").save(d)
+        ...
+        ...     # Read the CSV file as a DataFrame.
+        ...     spark.read.option("header", True).format('csv').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        for k in options:
+            self._write.options[k] = to_str(options[k])
+        return self
+
+    @overload
+    def partitionBy(self, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def partitionBy(self, *cols: List[str]) -> "DataFrameWriter":
+        ...
+
+    def partitionBy(self, *cols: Union[str, List[str]]) -> "DataFrameWriter":
+        """Partitions the output by the given columns on the file system.
+
+        If specified, the output is laid out on the file system similar
+        to Hive's partitioning scheme.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        cols : str or list
+            name of columns
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a partitioned manner, and read it back.
+
+        >>> import tempfile
+        >>> import os
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file in a partitioned manner.
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}, {"age": 120, "name": "Ruifeng Zheng"}]
+        ...     ).write.partitionBy("name").mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).sort("age").show()
+        ...
+        ...     # Read one partition as a DataFrame.
+        ...     spark.read.parquet(f"{d}{os.path.sep}name=Hyukjin Kwon").show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |100| Hyukjin Kwon|
+        |120|Ruifeng Zheng|
+        +---+-------------+
+        +---+
+        |age|
+        +---+
+        |100|
+        +---+
+        """
+        if len(cols) == 1 and isinstance(cols[0], (list, tuple)):
+            cols = cols[0]  # type: ignore[assignment]
+
+        self._write.partitioning_cols = cast(List[str], cols)
+        return self
+
+    @overload
+    def bucketBy(self, numBuckets: int, col: str, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def bucketBy(self, numBuckets: int, col: TupleOrListOfString) -> "DataFrameWriter":
+        ...
+
+    def bucketBy(
+        self, numBuckets: int, col: Union[str, TupleOrListOfString], *cols: Optional[str]
+    ) -> "DataFrameWriter":
+        """Buckets the output by the given columns. If specified,
+        the output is laid out on the file system similar to Hive's bucketing scheme,
+        but with a different bucket hash function and is not compatible with Hive's bucketing.
+
+        .. versionadded:: 2.3.0
+
+        Parameters
+        ----------
+        numBuckets : int
+            the number of buckets to save
+        col : str, list or tuple
+            a name of a column, or a list of names.
+        cols : str
+            additional names (optional). If `col` is a list it should be empty.
+
+        Notes
+        -----
+        Applicable for file-based data sources in combination with
+        :py:meth:`DataFrameWriter.saveAsTable`.
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a buckted manner, and read it back.
+
+        >>> from pyspark.sql.functions import input_file_name
+        >>> # Write a DataFrame into a Parquet file in a bucketed manner.
+        ... _ = spark.sql("DROP TABLE IF EXISTS bucketed_table")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.bucketBy(2, "name").mode("overwrite").saveAsTable("bucketed_table")
+        >>> # Read the Parquet file as a DataFrame.
+        ... spark.read.table("bucketed_table").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE bucketed_table")
+        """
+        if not isinstance(numBuckets, int):
+            raise TypeError("numBuckets should be an int, got {0}.".format(type(numBuckets)))
+
+        if isinstance(col, (list, tuple)):
+            if cols:
+                raise ValueError("col is a {0} but cols are not empty".format(type(col)))
+
+            col, cols = col[0], col[1:]  # type: ignore[assignment]
+
+        if not all(isinstance(c, str) for c in cols) or not (isinstance(col, str)):
+            raise TypeError("all names should be `str`")
+
+        self._write.num_buckets = numBuckets
+        self._write.bucket_cols = cast(List[str], cols)
+        return self
+
+    @overload
+    def sortBy(self, col: str, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def sortBy(self, col: TupleOrListOfString) -> "DataFrameWriter":
+        ...
+
+    def sortBy(
+        self, col: Union[str, TupleOrListOfString], *cols: Optional[str]
+    ) -> "DataFrameWriter":
+        """Sorts the output in each bucket by the given columns on the file system.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        col : str, tuple or list
+            a name of a column, or a list of names.
+        cols : str
+            additional names (optional). If `col` is a list it should be empty.
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a sorted-buckted manner, and read it back.
+
+        >>> from pyspark.sql.functions import input_file_name
+        >>> # Write a DataFrame into a Parquet file in a sorted-bucketed manner.
+        ... _ = spark.sql("DROP TABLE IF EXISTS sorted_bucketed_table")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.bucketBy(1, "name").sortBy("age").mode(
+        ...     "overwrite").saveAsTable("sorted_bucketed_table")
+        >>> # Read the Parquet file as a DataFrame.
+        ... spark.read.table("sorted_bucketed_table").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE sorted_bucketed_table")
+        """
+        if isinstance(col, (list, tuple)):
+            if cols:
+                raise ValueError("col is a {0} but cols are not empty".format(type(col)))
+
+            col, cols = col[0], col[1:]  # type: ignore[assignment]
+
+        if not all(isinstance(c, str) for c in cols) or not (isinstance(col, str)):
+            raise TypeError("all names should be `str`")
+
+        self._write.sort_cols = cast(List[str], cols)
+        return self
+
+    def save(
+        self,
+        path: Optional[str] = None,
+        format: Optional[str] = None,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """Saves the contents of the :class:`DataFrame` to a data source.
+
+        The data source is specified by the ``format`` and a set of ``options``.
+        If ``format`` is not specified, the default data source configured by
+        ``spark.sql.sources.default`` will be used.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str, optional
+            the path in a Hadoop supported file system
+        format : str, optional
+            the format used to save
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+        partitionBy : list, optional
+            names of partitioning columns
+        **options : dict
+            all other string options
+
+        Examples
+        --------
+        Write a DataFrame into a JSON file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a JSON file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("json").save(d)
+        ...
+        ...     # Read the JSON file as a DataFrame.
+        ...     spark.read.format('json').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode).options(**options)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        if format is not None:
+            self.format(format)
+        self._write.path = path
+        self._spark.client.execute_command(self._write.command(self._spark.client))
+
+    def insertInto(self, tableName: str, overwrite: Optional[bool] = None) -> None:
+        """Inserts the content of the :class:`DataFrame` to the specified table.
+
+        It requires that the schema of the :class:`DataFrame` is the same as the
+        schema of the table.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        overwrite : bool, optional
+            If true, overwrites existing data. Disabled by default
+
+        Notes
+        -----
+        Unlike :meth:`DataFrameWriter.saveAsTable`, :meth:`DataFrameWriter.insertInto` ignores
+        the column names and just uses position-based resolution.
+
+        Examples
+        --------
+        >>> _ = spark.sql("DROP TABLE IF EXISTS tblA")
+        >>> df = spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... )
+        >>> df.write.saveAsTable("tblA")
+
+        Insert the data into 'tblA' table but with different column names.
+
+        >>> df.selectExpr("age AS col1", "name AS col2").write.insertInto("tblA")
+        >>> spark.read.table("tblA").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE tblA")
+        """
+        if overwrite is not None:
+            self.mode("overwrite" if overwrite else "append")
+        self.saveAsTable(tableName)
+
+    def saveAsTable(
+        self,
+        name: str,
+        format: Optional[str] = None,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` as the specified table.
+
+        In the case the table already exists, behavior of this function depends on the
+        save mode, specified by the `mode` function (default to throwing an exception).
+        When `mode` is `Overwrite`, the schema of the :class:`DataFrame` does not need to be
+        the same as that of the existing table.
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Notes
+        -----
+        When `mode` is `Append`, if there is an existing table, we will use the format and
+        options of the existing table. The column order in the schema of the :class:`DataFrame`
+        doesn't need to be the same as that of the existing table. Unlike
+        :meth:`DataFrameWriter.insertInto`, :meth:`DataFrameWriter.saveAsTable` will use the
+        column names to find the correct column positions.
+
+        Parameters
+        ----------
+        name : str
+            the table name
+        format : str, optional
+            the format used to save
+        mode : str, optional
+            one of `append`, `overwrite`, `error`, `errorifexists`, `ignore` \
+            (default: error)
+        partitionBy : str or list
+            names of partitioning columns
+        **options : dict
+            all other string options
+
+        Examples
+        --------
+        Creates a table from a DataFrame, and read it back.
+
+        >>> _ = spark.sql("DROP TABLE IF EXISTS tblA")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.saveAsTable("tblA")
+        >>> spark.read.table("tblA").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE tblA")
+        """
+        self.mode(mode).options(**options)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        if format is not None:
+            self.format(format)
+        self._write.table_name = name
+        self._spark.client.execute_command(self._write.command(self._spark.client))
+
+    def json(
+        self,
+        path: str,
+        mode: Optional[str] = None,
+        compression: Optional[str] = None,
+        dateFormat: Optional[str] = None,
+        timestampFormat: Optional[str] = None,
+        lineSep: Optional[str] = None,
+        encoding: Optional[str] = None,
+        ignoreNullFields: Optional[Union[bool, str]] = None,
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` in JSON format
+        (`JSON Lines text format or newline-delimited JSON <http://jsonlines.org/>`_) at the
+        specified path.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-json.html#data-source-option>`_
+            for the version you use.
+
+            .. # noqa
+
+        Examples
+        --------
+        Write a DataFrame into a JSON file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a JSON file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.json(d, mode="overwrite")
+        ...
+        ...     # Read the JSON file as a DataFrame.
+        ...     spark.read.format("json").load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode)
+        self._set_opts(
+            compression=compression,
+            dateFormat=dateFormat,
+            timestampFormat=timestampFormat,
+            lineSep=lineSep,
+            encoding=encoding,
+            ignoreNullFields=ignoreNullFields,
+        )
+        self.format("json").save(path)
+
+    def parquet(
+        self,
+        path: str,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        compression: Optional[str] = None,
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` in Parquet format at the specified path.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+        partitionBy : str or list, optional
+            names of partitioning columns
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#data-source-option>`_
+            in the version you use.
+
+            .. # noqa
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.parquet(d, mode="overwrite")
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.format("parquet").load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        self.option("compression", compression)
+        self.format("parquet").save(path)
+
+    def text(
+        self, path: str, compression: Optional[str] = None, lineSep: Optional[str] = None
+    ) -> None:
+        """Saves the content of the DataFrame in a text file at the specified path.
+        The text files will be encoded as UTF-8.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-text.html#data-source-option>`_
+            in the version you use.

Review Comment:
   in -> for



##########
python/pyspark/sql/connect/writer.py:
##########
@@ -0,0 +1,926 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+
+from typing import Optional, Union, List, overload, Tuple, cast, Any
+from typing import TYPE_CHECKING
+
+from pyspark.sql.connect.plan import WriteOperation, LogicalPlan
+from pyspark.sql.types import StructType
+from pyspark.sql.utils import to_str
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect._typing import OptionalPrimitiveType
+    from pyspark.sql.connect.session import SparkSession
+
+
+PathOrPaths = Union[str, List[str]]
+TupleOrListOfString = Union[List[str], Tuple[str, ...]]
+
+
+class OptionUtils:
+    def _set_opts(
+        self,
+        schema: Optional[Union[StructType, str]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """
+        Set named options (filter out those the value is None)
+        """
+        if schema is not None:
+            self.schema(schema)  # type: ignore[attr-defined]
+        for k, v in options.items():
+            if v is not None:
+                self.option(k, v)  # type: ignore[attr-defined]
+
+
+class DataFrameWriter(OptionUtils):
+    """
+    Interface used to write a :class:`DataFrame` to external storage systems
+    (e.g. file systems, key-value stores, etc). Use :attr:`DataFrame.write`
+    to access this.
+
+    .. versionadded:: 3.4.0
+    """
+
+    def __init__(self, plan: "LogicalPlan", session: "SparkSession"):
+        self._df: "LogicalPlan" = plan
+        self._spark: "SparkSession" = session
+        self._write: "WriteOperation" = WriteOperation(self._df)
+
+    def mode(self, saveMode: Optional[str]) -> "DataFrameWriter":
+        """Specifies the behavior when data or table already exists.
+
+        Options include:
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Examples
+        --------
+        Raise an error when writing to an existing path.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     spark.createDataFrame(
+        ...         [{"age": 80, "name": "Xinrong Meng"}]
+        ...     ).write.mode("error").format("parquet").save(d)
+        Traceback (most recent call last):
+            ...
+        pyspark.sql.utils.AnalysisException: ...
+
+        Write a Parquet file back with various options, and read it back.
+
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Overwrite the path with a new Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 120, "name": "Takuya Ueshin"}]
+        ...     ).write.mode("append").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 140, "name": "Haejoon Lee"}]
+        ...     ).write.mode("ignore").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |120|Takuya Ueshin|
+        |100| Hyukjin Kwon|
+        +---+-------------+
+        """
+        # At the JVM side, the default value of mode is already set to "error".
+        # So, if the given saveMode is None, we will not call JVM-side's mode method.
+        if saveMode is not None:
+            self._write.mode = saveMode
+        return self
+
+    def format(self, source: str) -> "DataFrameWriter":
+        """Specifies the underlying output data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        source : str
+            string, name of the data source, e.g. 'json', 'parquet'.
+
+        Examples
+        --------
+        >>> spark.range(1).write.format('parquet')
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Write a DataFrame into a Parquet file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.format('parquet').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.source = source
+        return self
+
+    def option(self, key: str, value: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds an output option for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        key : str
+            The key for the option to set.
+        value
+            The value for the option to set.
+
+        Examples
+        --------
+        >>> spark.range(1).write.option("key", "value")
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Specify the option 'nullValue' with writing a CSV file.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file with 'nullValue' option set to 'Hyukjin Kwon'.
+        ...     df = spark.createDataFrame([(100, None)], "age INT, name STRING")
+        ...     df.write.option("nullValue", "Hyukjin Kwon").mode("overwrite").format("csv").save(d)
+        ...
+        ...     # Read the CSV file as a DataFrame.
+        ...     spark.read.schema(df.schema).format('csv').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.options[key] = to_str(value)
+        return self
+
+    def options(self, **options: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds output options for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        **options : dict
+            The dictionary of string keys and primitive-type values.
+
+        Examples
+        --------
+        >>> spark.range(1).write.option("key", "value")
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Specify the option 'nullValue' and 'header' with writing a CSV file.
+        >>> from pyspark.sql.types import StructType,StructField, StringType, IntegerType
+        >>> schema = StructType([
+        ...     StructField("age",IntegerType(),True),
+        ...     StructField("name",StringType(),True),
+        ...])
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file with 'nullValue' option set to 'Hyukjin Kwon',
+        ...     # and 'header' option set to `True`.
+        ...     df = spark.createDataFrame([(100, None], schema=schema)
+        ...     df.write.options(nullValue="Hyukjin Kwon", header=True).mode(
+        ...         "overwrite").format("csv").save(d)
+        ...
+        ...     # Read the CSV file as a DataFrame.
+        ...     spark.read.option("header", True).format('csv').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        for k in options:
+            self._write.options[k] = to_str(options[k])
+        return self
+
+    @overload
+    def partitionBy(self, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def partitionBy(self, *cols: List[str]) -> "DataFrameWriter":
+        ...
+
+    def partitionBy(self, *cols: Union[str, List[str]]) -> "DataFrameWriter":
+        """Partitions the output by the given columns on the file system.
+
+        If specified, the output is laid out on the file system similar
+        to Hive's partitioning scheme.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        cols : str or list
+            name of columns
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a partitioned manner, and read it back.
+
+        >>> import tempfile
+        >>> import os
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file in a partitioned manner.
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}, {"age": 120, "name": "Ruifeng Zheng"}]
+        ...     ).write.partitionBy("name").mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).sort("age").show()
+        ...
+        ...     # Read one partition as a DataFrame.
+        ...     spark.read.parquet(f"{d}{os.path.sep}name=Hyukjin Kwon").show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |100| Hyukjin Kwon|
+        |120|Ruifeng Zheng|
+        +---+-------------+
+        +---+
+        |age|
+        +---+
+        |100|
+        +---+
+        """
+        if len(cols) == 1 and isinstance(cols[0], (list, tuple)):
+            cols = cols[0]  # type: ignore[assignment]
+
+        self._write.partitioning_cols = cast(List[str], cols)
+        return self
+
+    @overload
+    def bucketBy(self, numBuckets: int, col: str, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def bucketBy(self, numBuckets: int, col: TupleOrListOfString) -> "DataFrameWriter":
+        ...
+
+    def bucketBy(
+        self, numBuckets: int, col: Union[str, TupleOrListOfString], *cols: Optional[str]
+    ) -> "DataFrameWriter":
+        """Buckets the output by the given columns. If specified,
+        the output is laid out on the file system similar to Hive's bucketing scheme,
+        but with a different bucket hash function and is not compatible with Hive's bucketing.
+
+        .. versionadded:: 2.3.0
+
+        Parameters
+        ----------
+        numBuckets : int
+            the number of buckets to save
+        col : str, list or tuple
+            a name of a column, or a list of names.
+        cols : str
+            additional names (optional). If `col` is a list it should be empty.
+
+        Notes
+        -----
+        Applicable for file-based data sources in combination with
+        :py:meth:`DataFrameWriter.saveAsTable`.
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a buckted manner, and read it back.
+
+        >>> from pyspark.sql.functions import input_file_name
+        >>> # Write a DataFrame into a Parquet file in a bucketed manner.
+        ... _ = spark.sql("DROP TABLE IF EXISTS bucketed_table")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.bucketBy(2, "name").mode("overwrite").saveAsTable("bucketed_table")
+        >>> # Read the Parquet file as a DataFrame.
+        ... spark.read.table("bucketed_table").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE bucketed_table")
+        """
+        if not isinstance(numBuckets, int):
+            raise TypeError("numBuckets should be an int, got {0}.".format(type(numBuckets)))
+
+        if isinstance(col, (list, tuple)):
+            if cols:
+                raise ValueError("col is a {0} but cols are not empty".format(type(col)))
+
+            col, cols = col[0], col[1:]  # type: ignore[assignment]
+
+        if not all(isinstance(c, str) for c in cols) or not (isinstance(col, str)):
+            raise TypeError("all names should be `str`")
+
+        self._write.num_buckets = numBuckets
+        self._write.bucket_cols = cast(List[str], cols)
+        return self
+
+    @overload
+    def sortBy(self, col: str, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def sortBy(self, col: TupleOrListOfString) -> "DataFrameWriter":
+        ...
+
+    def sortBy(
+        self, col: Union[str, TupleOrListOfString], *cols: Optional[str]
+    ) -> "DataFrameWriter":
+        """Sorts the output in each bucket by the given columns on the file system.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        col : str, tuple or list
+            a name of a column, or a list of names.
+        cols : str
+            additional names (optional). If `col` is a list it should be empty.
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a sorted-buckted manner, and read it back.
+
+        >>> from pyspark.sql.functions import input_file_name
+        >>> # Write a DataFrame into a Parquet file in a sorted-bucketed manner.
+        ... _ = spark.sql("DROP TABLE IF EXISTS sorted_bucketed_table")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.bucketBy(1, "name").sortBy("age").mode(
+        ...     "overwrite").saveAsTable("sorted_bucketed_table")
+        >>> # Read the Parquet file as a DataFrame.
+        ... spark.read.table("sorted_bucketed_table").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE sorted_bucketed_table")
+        """
+        if isinstance(col, (list, tuple)):
+            if cols:
+                raise ValueError("col is a {0} but cols are not empty".format(type(col)))
+
+            col, cols = col[0], col[1:]  # type: ignore[assignment]
+
+        if not all(isinstance(c, str) for c in cols) or not (isinstance(col, str)):
+            raise TypeError("all names should be `str`")
+
+        self._write.sort_cols = cast(List[str], cols)
+        return self
+
+    def save(
+        self,
+        path: Optional[str] = None,
+        format: Optional[str] = None,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """Saves the contents of the :class:`DataFrame` to a data source.
+
+        The data source is specified by the ``format`` and a set of ``options``.
+        If ``format`` is not specified, the default data source configured by
+        ``spark.sql.sources.default`` will be used.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str, optional
+            the path in a Hadoop supported file system
+        format : str, optional
+            the format used to save
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+        partitionBy : list, optional
+            names of partitioning columns
+        **options : dict
+            all other string options
+
+        Examples
+        --------
+        Write a DataFrame into a JSON file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a JSON file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("json").save(d)
+        ...
+        ...     # Read the JSON file as a DataFrame.
+        ...     spark.read.format('json').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode).options(**options)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        if format is not None:
+            self.format(format)
+        self._write.path = path
+        self._spark.client.execute_command(self._write.command(self._spark.client))
+
+    def insertInto(self, tableName: str, overwrite: Optional[bool] = None) -> None:
+        """Inserts the content of the :class:`DataFrame` to the specified table.
+
+        It requires that the schema of the :class:`DataFrame` is the same as the
+        schema of the table.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        overwrite : bool, optional
+            If true, overwrites existing data. Disabled by default
+
+        Notes
+        -----
+        Unlike :meth:`DataFrameWriter.saveAsTable`, :meth:`DataFrameWriter.insertInto` ignores
+        the column names and just uses position-based resolution.
+
+        Examples
+        --------
+        >>> _ = spark.sql("DROP TABLE IF EXISTS tblA")
+        >>> df = spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... )
+        >>> df.write.saveAsTable("tblA")
+
+        Insert the data into 'tblA' table but with different column names.
+
+        >>> df.selectExpr("age AS col1", "name AS col2").write.insertInto("tblA")
+        >>> spark.read.table("tblA").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE tblA")
+        """
+        if overwrite is not None:
+            self.mode("overwrite" if overwrite else "append")
+        self.saveAsTable(tableName)
+
+    def saveAsTable(
+        self,
+        name: str,
+        format: Optional[str] = None,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` as the specified table.
+
+        In the case the table already exists, behavior of this function depends on the
+        save mode, specified by the `mode` function (default to throwing an exception).
+        When `mode` is `Overwrite`, the schema of the :class:`DataFrame` does not need to be
+        the same as that of the existing table.
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Notes
+        -----
+        When `mode` is `Append`, if there is an existing table, we will use the format and
+        options of the existing table. The column order in the schema of the :class:`DataFrame`
+        doesn't need to be the same as that of the existing table. Unlike
+        :meth:`DataFrameWriter.insertInto`, :meth:`DataFrameWriter.saveAsTable` will use the
+        column names to find the correct column positions.
+
+        Parameters
+        ----------
+        name : str
+            the table name
+        format : str, optional
+            the format used to save
+        mode : str, optional
+            one of `append`, `overwrite`, `error`, `errorifexists`, `ignore` \
+            (default: error)
+        partitionBy : str or list
+            names of partitioning columns
+        **options : dict
+            all other string options
+
+        Examples
+        --------
+        Creates a table from a DataFrame, and read it back.
+
+        >>> _ = spark.sql("DROP TABLE IF EXISTS tblA")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.saveAsTable("tblA")
+        >>> spark.read.table("tblA").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE tblA")
+        """
+        self.mode(mode).options(**options)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        if format is not None:
+            self.format(format)
+        self._write.table_name = name
+        self._spark.client.execute_command(self._write.command(self._spark.client))
+
+    def json(
+        self,
+        path: str,
+        mode: Optional[str] = None,
+        compression: Optional[str] = None,
+        dateFormat: Optional[str] = None,
+        timestampFormat: Optional[str] = None,
+        lineSep: Optional[str] = None,
+        encoding: Optional[str] = None,
+        ignoreNullFields: Optional[Union[bool, str]] = None,
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` in JSON format
+        (`JSON Lines text format or newline-delimited JSON <http://jsonlines.org/>`_) at the
+        specified path.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-json.html#data-source-option>`_
+            for the version you use.
+
+            .. # noqa
+
+        Examples
+        --------
+        Write a DataFrame into a JSON file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a JSON file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.json(d, mode="overwrite")
+        ...
+        ...     # Read the JSON file as a DataFrame.
+        ...     spark.read.format("json").load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode)
+        self._set_opts(
+            compression=compression,
+            dateFormat=dateFormat,
+            timestampFormat=timestampFormat,
+            lineSep=lineSep,
+            encoding=encoding,
+            ignoreNullFields=ignoreNullFields,
+        )
+        self.format("json").save(path)
+
+    def parquet(
+        self,
+        path: str,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        compression: Optional[str] = None,
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` in Parquet format at the specified path.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+        partitionBy : str or list, optional
+            names of partitioning columns
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#data-source-option>`_
+            in the version you use.

Review Comment:
   in -> for



##########
python/pyspark/sql/connect/writer.py:
##########
@@ -0,0 +1,926 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+
+from typing import Optional, Union, List, overload, Tuple, cast, Any
+from typing import TYPE_CHECKING
+
+from pyspark.sql.connect.plan import WriteOperation, LogicalPlan
+from pyspark.sql.types import StructType
+from pyspark.sql.utils import to_str
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect._typing import OptionalPrimitiveType
+    from pyspark.sql.connect.session import SparkSession
+
+
+PathOrPaths = Union[str, List[str]]
+TupleOrListOfString = Union[List[str], Tuple[str, ...]]
+
+
+class OptionUtils:
+    def _set_opts(
+        self,
+        schema: Optional[Union[StructType, str]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """
+        Set named options (filter out those the value is None)
+        """
+        if schema is not None:
+            self.schema(schema)  # type: ignore[attr-defined]
+        for k, v in options.items():
+            if v is not None:
+                self.option(k, v)  # type: ignore[attr-defined]
+
+
+class DataFrameWriter(OptionUtils):
+    """
+    Interface used to write a :class:`DataFrame` to external storage systems
+    (e.g. file systems, key-value stores, etc). Use :attr:`DataFrame.write`
+    to access this.
+
+    .. versionadded:: 3.4.0
+    """
+
+    def __init__(self, plan: "LogicalPlan", session: "SparkSession"):
+        self._df: "LogicalPlan" = plan
+        self._spark: "SparkSession" = session
+        self._write: "WriteOperation" = WriteOperation(self._df)
+
+    def mode(self, saveMode: Optional[str]) -> "DataFrameWriter":
+        """Specifies the behavior when data or table already exists.
+
+        Options include:
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Examples
+        --------
+        Raise an error when writing to an existing path.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     spark.createDataFrame(
+        ...         [{"age": 80, "name": "Xinrong Meng"}]
+        ...     ).write.mode("error").format("parquet").save(d)
+        Traceback (most recent call last):
+            ...
+        pyspark.sql.utils.AnalysisException: ...
+
+        Write a Parquet file back with various options, and read it back.
+
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Overwrite the path with a new Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 120, "name": "Takuya Ueshin"}]
+        ...     ).write.mode("append").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 140, "name": "Haejoon Lee"}]
+        ...     ).write.mode("ignore").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |120|Takuya Ueshin|
+        |100| Hyukjin Kwon|
+        +---+-------------+
+        """
+        # At the JVM side, the default value of mode is already set to "error".
+        # So, if the given saveMode is None, we will not call JVM-side's mode method.
+        if saveMode is not None:
+            self._write.mode = saveMode
+        return self
+
+    def format(self, source: str) -> "DataFrameWriter":
+        """Specifies the underlying output data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        source : str
+            string, name of the data source, e.g. 'json', 'parquet'.
+
+        Examples
+        --------
+        >>> spark.range(1).write.format('parquet')
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Write a DataFrame into a Parquet file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.format('parquet').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.source = source
+        return self
+
+    def option(self, key: str, value: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds an output option for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        key : str
+            The key for the option to set.
+        value
+            The value for the option to set.
+
+        Examples
+        --------
+        >>> spark.range(1).write.option("key", "value")
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Specify the option 'nullValue' with writing a CSV file.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file with 'nullValue' option set to 'Hyukjin Kwon'.
+        ...     df = spark.createDataFrame([(100, None)], "age INT, name STRING")
+        ...     df.write.option("nullValue", "Hyukjin Kwon").mode("overwrite").format("csv").save(d)
+        ...
+        ...     # Read the CSV file as a DataFrame.
+        ...     spark.read.schema(df.schema).format('csv').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.options[key] = to_str(value)
+        return self
+
+    def options(self, **options: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds output options for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        **options : dict
+            The dictionary of string keys and primitive-type values.
+
+        Examples
+        --------
+        >>> spark.range(1).write.option("key", "value")
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Specify the option 'nullValue' and 'header' with writing a CSV file.
+        >>> from pyspark.sql.types import StructType,StructField, StringType, IntegerType
+        >>> schema = StructType([
+        ...     StructField("age",IntegerType(),True),
+        ...     StructField("name",StringType(),True),
+        ...])
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file with 'nullValue' option set to 'Hyukjin Kwon',
+        ...     # and 'header' option set to `True`.
+        ...     df = spark.createDataFrame([(100, None], schema=schema)
+        ...     df.write.options(nullValue="Hyukjin Kwon", header=True).mode(
+        ...         "overwrite").format("csv").save(d)
+        ...
+        ...     # Read the CSV file as a DataFrame.
+        ...     spark.read.option("header", True).format('csv').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        for k in options:
+            self._write.options[k] = to_str(options[k])
+        return self
+
+    @overload
+    def partitionBy(self, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def partitionBy(self, *cols: List[str]) -> "DataFrameWriter":
+        ...
+
+    def partitionBy(self, *cols: Union[str, List[str]]) -> "DataFrameWriter":
+        """Partitions the output by the given columns on the file system.
+
+        If specified, the output is laid out on the file system similar
+        to Hive's partitioning scheme.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        cols : str or list
+            name of columns
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a partitioned manner, and read it back.
+
+        >>> import tempfile
+        >>> import os
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file in a partitioned manner.
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}, {"age": 120, "name": "Ruifeng Zheng"}]
+        ...     ).write.partitionBy("name").mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).sort("age").show()
+        ...
+        ...     # Read one partition as a DataFrame.
+        ...     spark.read.parquet(f"{d}{os.path.sep}name=Hyukjin Kwon").show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |100| Hyukjin Kwon|
+        |120|Ruifeng Zheng|
+        +---+-------------+
+        +---+
+        |age|
+        +---+
+        |100|
+        +---+
+        """
+        if len(cols) == 1 and isinstance(cols[0], (list, tuple)):
+            cols = cols[0]  # type: ignore[assignment]
+
+        self._write.partitioning_cols = cast(List[str], cols)
+        return self
+
+    @overload
+    def bucketBy(self, numBuckets: int, col: str, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def bucketBy(self, numBuckets: int, col: TupleOrListOfString) -> "DataFrameWriter":
+        ...
+
+    def bucketBy(
+        self, numBuckets: int, col: Union[str, TupleOrListOfString], *cols: Optional[str]
+    ) -> "DataFrameWriter":
+        """Buckets the output by the given columns. If specified,
+        the output is laid out on the file system similar to Hive's bucketing scheme,
+        but with a different bucket hash function and is not compatible with Hive's bucketing.
+
+        .. versionadded:: 2.3.0
+
+        Parameters
+        ----------
+        numBuckets : int
+            the number of buckets to save
+        col : str, list or tuple
+            a name of a column, or a list of names.
+        cols : str
+            additional names (optional). If `col` is a list it should be empty.
+
+        Notes
+        -----
+        Applicable for file-based data sources in combination with
+        :py:meth:`DataFrameWriter.saveAsTable`.
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a buckted manner, and read it back.
+
+        >>> from pyspark.sql.functions import input_file_name
+        >>> # Write a DataFrame into a Parquet file in a bucketed manner.
+        ... _ = spark.sql("DROP TABLE IF EXISTS bucketed_table")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.bucketBy(2, "name").mode("overwrite").saveAsTable("bucketed_table")
+        >>> # Read the Parquet file as a DataFrame.
+        ... spark.read.table("bucketed_table").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE bucketed_table")
+        """
+        if not isinstance(numBuckets, int):
+            raise TypeError("numBuckets should be an int, got {0}.".format(type(numBuckets)))
+
+        if isinstance(col, (list, tuple)):
+            if cols:
+                raise ValueError("col is a {0} but cols are not empty".format(type(col)))
+
+            col, cols = col[0], col[1:]  # type: ignore[assignment]
+
+        if not all(isinstance(c, str) for c in cols) or not (isinstance(col, str)):
+            raise TypeError("all names should be `str`")
+
+        self._write.num_buckets = numBuckets
+        self._write.bucket_cols = cast(List[str], cols)
+        return self
+
+    @overload
+    def sortBy(self, col: str, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def sortBy(self, col: TupleOrListOfString) -> "DataFrameWriter":
+        ...
+
+    def sortBy(
+        self, col: Union[str, TupleOrListOfString], *cols: Optional[str]
+    ) -> "DataFrameWriter":
+        """Sorts the output in each bucket by the given columns on the file system.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        col : str, tuple or list
+            a name of a column, or a list of names.
+        cols : str
+            additional names (optional). If `col` is a list it should be empty.
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a sorted-buckted manner, and read it back.
+
+        >>> from pyspark.sql.functions import input_file_name
+        >>> # Write a DataFrame into a Parquet file in a sorted-bucketed manner.
+        ... _ = spark.sql("DROP TABLE IF EXISTS sorted_bucketed_table")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.bucketBy(1, "name").sortBy("age").mode(
+        ...     "overwrite").saveAsTable("sorted_bucketed_table")
+        >>> # Read the Parquet file as a DataFrame.
+        ... spark.read.table("sorted_bucketed_table").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE sorted_bucketed_table")
+        """
+        if isinstance(col, (list, tuple)):
+            if cols:
+                raise ValueError("col is a {0} but cols are not empty".format(type(col)))
+
+            col, cols = col[0], col[1:]  # type: ignore[assignment]
+
+        if not all(isinstance(c, str) for c in cols) or not (isinstance(col, str)):
+            raise TypeError("all names should be `str`")
+
+        self._write.sort_cols = cast(List[str], cols)
+        return self
+
+    def save(
+        self,
+        path: Optional[str] = None,
+        format: Optional[str] = None,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """Saves the contents of the :class:`DataFrame` to a data source.
+
+        The data source is specified by the ``format`` and a set of ``options``.
+        If ``format`` is not specified, the default data source configured by
+        ``spark.sql.sources.default`` will be used.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str, optional
+            the path in a Hadoop supported file system
+        format : str, optional
+            the format used to save
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+        partitionBy : list, optional
+            names of partitioning columns
+        **options : dict
+            all other string options
+
+        Examples
+        --------
+        Write a DataFrame into a JSON file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a JSON file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("json").save(d)
+        ...
+        ...     # Read the JSON file as a DataFrame.
+        ...     spark.read.format('json').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode).options(**options)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        if format is not None:
+            self.format(format)
+        self._write.path = path
+        self._spark.client.execute_command(self._write.command(self._spark.client))
+
+    def insertInto(self, tableName: str, overwrite: Optional[bool] = None) -> None:
+        """Inserts the content of the :class:`DataFrame` to the specified table.
+
+        It requires that the schema of the :class:`DataFrame` is the same as the
+        schema of the table.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        overwrite : bool, optional
+            If true, overwrites existing data. Disabled by default
+
+        Notes
+        -----
+        Unlike :meth:`DataFrameWriter.saveAsTable`, :meth:`DataFrameWriter.insertInto` ignores
+        the column names and just uses position-based resolution.
+
+        Examples
+        --------
+        >>> _ = spark.sql("DROP TABLE IF EXISTS tblA")
+        >>> df = spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... )
+        >>> df.write.saveAsTable("tblA")
+
+        Insert the data into 'tblA' table but with different column names.
+
+        >>> df.selectExpr("age AS col1", "name AS col2").write.insertInto("tblA")
+        >>> spark.read.table("tblA").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE tblA")
+        """
+        if overwrite is not None:
+            self.mode("overwrite" if overwrite else "append")
+        self.saveAsTable(tableName)
+
+    def saveAsTable(
+        self,
+        name: str,
+        format: Optional[str] = None,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` as the specified table.
+
+        In the case the table already exists, behavior of this function depends on the
+        save mode, specified by the `mode` function (default to throwing an exception).
+        When `mode` is `Overwrite`, the schema of the :class:`DataFrame` does not need to be
+        the same as that of the existing table.
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Notes
+        -----
+        When `mode` is `Append`, if there is an existing table, we will use the format and
+        options of the existing table. The column order in the schema of the :class:`DataFrame`
+        doesn't need to be the same as that of the existing table. Unlike
+        :meth:`DataFrameWriter.insertInto`, :meth:`DataFrameWriter.saveAsTable` will use the
+        column names to find the correct column positions.
+
+        Parameters
+        ----------
+        name : str
+            the table name
+        format : str, optional
+            the format used to save
+        mode : str, optional
+            one of `append`, `overwrite`, `error`, `errorifexists`, `ignore` \
+            (default: error)
+        partitionBy : str or list
+            names of partitioning columns
+        **options : dict
+            all other string options
+
+        Examples
+        --------
+        Creates a table from a DataFrame, and read it back.
+
+        >>> _ = spark.sql("DROP TABLE IF EXISTS tblA")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.saveAsTable("tblA")
+        >>> spark.read.table("tblA").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE tblA")
+        """
+        self.mode(mode).options(**options)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        if format is not None:
+            self.format(format)
+        self._write.table_name = name
+        self._spark.client.execute_command(self._write.command(self._spark.client))
+
+    def json(
+        self,
+        path: str,
+        mode: Optional[str] = None,
+        compression: Optional[str] = None,
+        dateFormat: Optional[str] = None,
+        timestampFormat: Optional[str] = None,
+        lineSep: Optional[str] = None,
+        encoding: Optional[str] = None,
+        ignoreNullFields: Optional[Union[bool, str]] = None,
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` in JSON format
+        (`JSON Lines text format or newline-delimited JSON <http://jsonlines.org/>`_) at the
+        specified path.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-json.html#data-source-option>`_
+            for the version you use.
+
+            .. # noqa
+
+        Examples
+        --------
+        Write a DataFrame into a JSON file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a JSON file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.json(d, mode="overwrite")
+        ...
+        ...     # Read the JSON file as a DataFrame.
+        ...     spark.read.format("json").load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode)
+        self._set_opts(
+            compression=compression,
+            dateFormat=dateFormat,
+            timestampFormat=timestampFormat,
+            lineSep=lineSep,
+            encoding=encoding,
+            ignoreNullFields=ignoreNullFields,
+        )
+        self.format("json").save(path)
+
+    def parquet(
+        self,
+        path: str,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        compression: Optional[str] = None,
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` in Parquet format at the specified path.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+        partitionBy : str or list, optional
+            names of partitioning columns
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#data-source-option>`_
+            in the version you use.
+
+            .. # noqa
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.parquet(d, mode="overwrite")
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.format("parquet").load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        self.option("compression", compression)
+        self.format("parquet").save(path)
+
+    def text(
+        self, path: str, compression: Optional[str] = None, lineSep: Optional[str] = None
+    ) -> None:
+        """Saves the content of the DataFrame in a text file at the specified path.
+        The text files will be encoded as UTF-8.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-text.html#data-source-option>`_
+            in the version you use.
+
+            .. # noqa
+
+        Notes
+        -----
+        The DataFrame must have only one column that is of string type.
+        Each row becomes a new line in the output file.
+
+        Examples
+        --------
+        Write a DataFrame into a text file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a text file
+        ...     df = spark.createDataFrame([("a",), ("b",), ("c",)], schema=["alphabets"])
+        ...     df.write.mode("overwrite").text(d)
+        ...
+        ...     # Read the text file as a DataFrame.
+        ...     spark.read.schema(df.schema).format("text").load(d).sort("alphabets").show()
+        +---------+
+        |alphabets|
+        +---------+
+        |        a|
+        |        b|
+        |        c|
+        +---------+
+        """
+        self._set_opts(compression=compression, lineSep=lineSep)
+        self.format("text").save(path)
+
+    def csv(
+        self,
+        path: str,
+        mode: Optional[str] = None,
+        compression: Optional[str] = None,
+        sep: Optional[str] = None,
+        quote: Optional[str] = None,
+        escape: Optional[str] = None,
+        header: Optional[Union[bool, str]] = None,
+        nullValue: Optional[str] = None,
+        escapeQuotes: Optional[Union[bool, str]] = None,
+        quoteAll: Optional[Union[bool, str]] = None,
+        dateFormat: Optional[str] = None,
+        timestampFormat: Optional[str] = None,
+        ignoreLeadingWhiteSpace: Optional[Union[bool, str]] = None,
+        ignoreTrailingWhiteSpace: Optional[Union[bool, str]] = None,
+        charToEscapeQuoteEscaping: Optional[str] = None,
+        encoding: Optional[str] = None,
+        emptyValue: Optional[str] = None,
+        lineSep: Optional[str] = None,
+    ) -> None:
+        r"""Saves the content of the :class:`DataFrame` in CSV format at the specified path.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-csv.html#data-source-option>`_
+            in the version you use.
+
+            .. # noqa
+
+        Examples
+        --------
+        Write a DataFrame into a CSV file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file
+        ...     df = spark.createDataFrame([{"age": 100, "name": "Hyukjin Kwon"}])
+        ...     df.write.csv(d, mode="overwrite")
+        ...
+        ...     # Read the CSV file as a DataFrame with 'nullValue' option set to 'Hyukjin Kwon'.
+        ...     spark.read.schema(df.schema).format("csv").option(
+        ...         "nullValue", "Hyukjin Kwon").load(d).show()
+        +---+----+
+        |age|name|
+        +---+----+
+        |100|null|
+        +---+----+
+        """
+        self.mode(mode)
+        self._set_opts(
+            compression=compression,
+            sep=sep,
+            quote=quote,
+            escape=escape,
+            header=header,
+            nullValue=nullValue,
+            escapeQuotes=escapeQuotes,
+            quoteAll=quoteAll,
+            dateFormat=dateFormat,
+            timestampFormat=timestampFormat,
+            ignoreLeadingWhiteSpace=ignoreLeadingWhiteSpace,
+            ignoreTrailingWhiteSpace=ignoreTrailingWhiteSpace,
+            charToEscapeQuoteEscaping=charToEscapeQuoteEscaping,
+            encoding=encoding,
+            emptyValue=emptyValue,
+            lineSep=lineSep,
+        )
+        self.format("csv").save(path)
+
+    def orc(
+        self,
+        path: str,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        compression: Optional[str] = None,
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` in ORC format at the specified path.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+        partitionBy : str or list, optional
+            names of partitioning columns
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-orc.html#data-source-option>`_
+            in the version you use.

Review Comment:
   in -> for



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] grundprinzip commented on a diff in pull request #38801: [SPARK-40539] [CONNECT] [PYTHON] Add basic support for DataFrameWriter

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #38801:
URL: https://github.com/apache/spark/pull/38801#discussion_r1033194288


##########
python/pyspark/sql/tests/connect/test_connect_basic.py:
##########
@@ -566,6 +569,27 @@ def test_agg_with_two_agg_exprs(self):
             self.spark.read.table(self.tbl_name).agg({"name": "min", "id": "max"}).toPandas(),
         )
 
+    def test_write_operations(self):
+        with tempfile.TemporaryDirectory() as d:
+            df = self.connect.range(1, 100)
+            df.write.mode("overwrite").format("csv").save(d)
+
+            ndf = self.connect.read.load(d, format="csv")
+            df.toPandas().equals(ndf.toPandas())
+
+        with tempfile.TemporaryDirectory() as d:
+            df = self.connect.range(1, 100)
+            df.write.mode("overwrite").csv(d, lineSep="|")
+
+            ndf = self.connect.read.load(d, format="csv", lineSep="|")
+            df.toPandas().equals(ndf.toPandas())

Review Comment:
   Done



##########
python/pyspark/sql/tests/connect/test_connect_basic.py:
##########
@@ -566,6 +569,27 @@ def test_agg_with_two_agg_exprs(self):
             self.spark.read.table(self.tbl_name).agg({"name": "min", "id": "max"}).toPandas(),
         )
 
+    def test_write_operations(self):
+        with tempfile.TemporaryDirectory() as d:
+            df = self.connect.range(1, 100)
+            df.write.mode("overwrite").format("csv").save(d)
+
+            ndf = self.connect.read.load(d, format="csv")
+            df.toPandas().equals(ndf.toPandas())
+
+        with tempfile.TemporaryDirectory() as d:
+            df = self.connect.range(1, 100)
+            df.write.mode("overwrite").csv(d, lineSep="|")
+
+            ndf = self.connect.read.load(d, format="csv", lineSep="|")
+            df.toPandas().equals(ndf.toPandas())
+
+        df = self.connect.range(1, 100)
+        df.write.format("parquet").saveAsTable("parquet_test")
+
+        ndf = self.connect.read.table("parquet_test")
+        df.toPandas().equals(ndf.toPandas())

Review Comment:
   Done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] grundprinzip commented on pull request #38801: [SPARK-41317][CONNECT][PYTHON] Add basic support for DataFrameWriter

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on PR #38801:
URL: https://github.com/apache/spark/pull/38801#issuecomment-1331114227

   > Overall look good to me.
   > 
   > 
   > 
   > This is based on the past scala and proto side writer API work so no big concerns. 
   > 
   > 
   > 
   > IIRC there were still some proto API level design comments that worth to take a look. I probably could re-visit that later after expression work.
   
   These comments were independent of this patch though? Because this patch was only Python. Just want to make sure if we missed something. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhengruifeng commented on pull request #38801: [SPARK-41317][CONNECT][PYTHON] Add basic support for DataFrameWriter

Posted by GitBox <gi...@apache.org>.
zhengruifeng commented on PR #38801:
URL: https://github.com/apache/spark/pull/38801#issuecomment-1330524688

   merged into master


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] bjornjorgensen commented on a diff in pull request #38801: [SPARK-40539] [CONNECT] [PYTHON] Add basic support for DataFrameWriter

Posted by GitBox <gi...@apache.org>.
bjornjorgensen commented on code in PR #38801:
URL: https://github.com/apache/spark/pull/38801#discussion_r1032674832


##########
python/pyspark/sql/connect/writer.py:
##########
@@ -0,0 +1,922 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+
+from typing import Optional, Union, List, overload, Tuple, cast, Any
+from typing import TYPE_CHECKING
+
+from pyspark.sql.connect.plan import WriteOperation, LogicalPlan
+from pyspark.sql.types import StructType
+from pyspark.sql.utils import to_str
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect._typing import OptionalPrimitiveType
+    from pyspark.sql.connect.client import RemoteSparkSession
+
+
+PathOrPaths = Union[str, List[str]]
+TupleOrListOfString = Union[List[str], Tuple[str, ...]]
+
+
+class OptionUtils:
+    def _set_opts(
+        self,
+        schema: Optional[Union[StructType, str]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """
+        Set named options (filter out those the value is None)
+        """
+        if schema is not None:
+            self.schema(schema)  # type: ignore[attr-defined]
+        for k, v in options.items():
+            if v is not None:
+                self.option(k, v)  # type: ignore[attr-defined]
+
+
+class DataFrameWriter(OptionUtils):
+    """
+    Interface used to write a :class:`DataFrame` to external storage systems
+    (e.g. file systems, key-value stores, etc). Use :attr:`DataFrame.write`
+    to access this.
+
+    .. versionadded:: 3.4.0
+    """
+
+    def __init__(self, plan: "LogicalPlan", session: "RemoteSparkSession"):
+        self._df: "LogicalPlan" = plan
+        self._spark: "RemoteSparkSession" = session
+        self._write: "WriteOperation" = WriteOperation(self._df)
+
+    def mode(self, saveMode: Optional[str]) -> "DataFrameWriter":
+        """Specifies the behavior when data or table already exists.
+
+        Options include:
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Examples
+        --------
+        Raise an error when writing to an existing path.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     spark.createDataFrame(
+        ...         [{"age": 80, "name": "Xinrong Meng"}]
+        ...     ).write.mode("error").format("parquet").save(d)
+        Traceback (most recent call last):
+            ...
+        pyspark.sql.utils.AnalysisException: ...
+
+        Write a Parquet file back with various options, and read it back.
+
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Overwrite the path with a new Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 120, "name": "Takuya Ueshin"}]
+        ...     ).write.mode("append").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 140, "name": "Haejoon Lee"}]
+        ...     ).write.mode("ignore").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |120|Takuya Ueshin|
+        |100| Hyukjin Kwon|
+        +---+-------------+
+        """
+        # At the JVM side, the default value of mode is already set to "error".
+        # So, if the given saveMode is None, we will not call JVM-side's mode method.
+        if saveMode is not None:
+            self._write.mode = saveMode
+        return self
+
+    def format(self, source: str) -> "DataFrameWriter":
+        """Specifies the underlying output data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        source : str
+            string, name of the data source, e.g. 'json', 'parquet'.
+
+        Examples
+        --------
+        >>> spark.range(1).write.format('parquet')
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Write a DataFrame into a Parquet file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.format('parquet').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.source = source
+        return self
+
+    def option(self, key: str, value: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds a output option for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        key : str
+            The key for the option to set.
+        value
+            The value for the option to set.
+
+        Examples
+        --------
+        >>> spark.range(1).write.option("key", "value")
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Specify the option 'nullValue' with writing a CSV file.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file with 'nullValue' option set to 'Hyukjin Kwon'.
+        ...     df = spark.createDataFrame([(100, None)], "age INT, name STRING")
+        ...     df.write.option("nullValue", "Hyukjin Kwon").mode("overwrite").format("csv").save(d)
+        ...
+        ...     # Read the CSV file as a DataFrame.
+        ...     spark.read.schema(df.schema).format('csv').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.options[key] = to_str(value)
+        return self
+
+    def options(self, **options: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds output options for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        **options : dict
+            The dictionary of string keys and prmitive-type values.
+
+        Examples
+        --------
+        >>> spark.range(1).write.option("key", "value")
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Specify the option 'nullValue' and 'header' with writing a CSV file.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file with 'nullValue' option set to 'Hyukjin Kwon',
+        ...     # and 'header' option set to `True`.
+        ...     df = spark.createDataFrame([(100, "Hyukjin Kwon")], ["age", "name"])
+        ...     df.write.options(nullValue="Hyukjin Kwon", header=True).mode(
+        ...         "overwrite").format("csv").save(d)
+        ...
+        ...     # Read the CSV file as a DataFrame.
+        ...     spark.read.option("header", True).format('csv').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        for k in options:
+            self._write.options[k] = to_str(options[k])
+        return self
+
+    @overload
+    def partitionBy(self, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def partitionBy(self, *cols: List[str]) -> "DataFrameWriter":
+        ...
+
+    def partitionBy(self, *cols: Union[str, List[str]]) -> "DataFrameWriter":
+        """Partitions the output by the given columns on the file system.
+
+        If specified, the output is laid out on the file system similar
+        to Hive's partitioning scheme.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        cols : str or list
+            name of columns
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a partitioned manner, and read it back.
+
+        >>> import tempfile
+        >>> import os
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file in a partitioned manner.
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}, {"age": 120, "name": "Ruifeng Zheng"}]
+        ...     ).write.partitionBy("name").mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).sort("age").show()
+        ...
+        ...     # Read one partition as a DataFrame.
+        ...     spark.read.parquet(f"{d}{os.path.sep}name=Hyukjin Kwon").show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |100| Hyukjin Kwon|
+        |120|Ruifeng Zheng|
+        +---+-------------+
+        +---+
+        |age|
+        +---+
+        |100|
+        +---+
+        """
+        if len(cols) == 1 and isinstance(cols[0], (list, tuple)):
+            cols = cols[0]  # type: ignore[assignment]
+
+        self._write.partitioning_cols = cast(List[str], cols)
+        return self
+
+    @overload
+    def bucketBy(self, numBuckets: int, col: str, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def bucketBy(self, numBuckets: int, col: TupleOrListOfString) -> "DataFrameWriter":
+        ...
+
+    def bucketBy(
+        self, numBuckets: int, col: Union[str, TupleOrListOfString], *cols: Optional[str]
+    ) -> "DataFrameWriter":
+        """Buckets the output by the given columns. If specified,
+        the output is laid out on the file system similar to Hive's bucketing scheme,
+        but with a different bucket hash function and is not compatible with Hive's bucketing.
+
+        .. versionadded:: 2.3.0
+
+        Parameters
+        ----------
+        numBuckets : int
+            the number of buckets to save
+        col : str, list or tuple
+            a name of a column, or a list of names.
+        cols : str
+            additional names (optional). If `col` is a list it should be empty.
+
+        Notes
+        -----
+        Applicable for file-based data sources in combination with
+        :py:meth:`DataFrameWriter.saveAsTable`.
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a buckted manner, and read it back.
+
+        >>> from pyspark.sql.functions import input_file_name
+        >>> # Write a DataFrame into a Parquet file in a bucketed manner.
+        ... _ = spark.sql("DROP TABLE IF EXISTS bucketed_table")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.bucketBy(2, "name").mode("overwrite").saveAsTable("bucketed_table")
+        >>> # Read the Parquet file as a DataFrame.
+        ... spark.read.table("bucketed_table").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE bucketed_table")
+        """
+        if not isinstance(numBuckets, int):
+            raise TypeError("numBuckets should be an int, got {0}.".format(type(numBuckets)))
+
+        if isinstance(col, (list, tuple)):
+            if cols:
+                raise ValueError("col is a {0} but cols are not empty".format(type(col)))
+
+            col, cols = col[0], col[1:]  # type: ignore[assignment]
+
+        if not all(isinstance(c, str) for c in cols) or not (isinstance(col, str)):
+            raise TypeError("all names should be `str`")
+
+        self._write.num_buckets = numBuckets
+        self._write.bucket_cols = cast(List[str], cols)
+        return self
+
+    @overload
+    def sortBy(self, col: str, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def sortBy(self, col: TupleOrListOfString) -> "DataFrameWriter":
+        ...
+
+    def sortBy(
+        self, col: Union[str, TupleOrListOfString], *cols: Optional[str]
+    ) -> "DataFrameWriter":
+        """Sorts the output in each bucket by the given columns on the file system.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        col : str, tuple or list
+            a name of a column, or a list of names.
+        cols : str
+            additional names (optional). If `col` is a list it should be empty.
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a sorted-buckted manner, and read it back.
+
+        >>> from pyspark.sql.functions import input_file_name
+        >>> # Write a DataFrame into a Parquet file in a sorted-bucketed manner.
+        ... _ = spark.sql("DROP TABLE IF EXISTS sorted_bucketed_table")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.bucketBy(1, "name").sortBy("age").mode(
+        ...     "overwrite").saveAsTable("sorted_bucketed_table")
+        >>> # Read the Parquet file as a DataFrame.
+        ... spark.read.table("sorted_bucketed_table").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE sorted_bucketed_table")
+        """
+        if isinstance(col, (list, tuple)):
+            if cols:
+                raise ValueError("col is a {0} but cols are not empty".format(type(col)))
+
+            col, cols = col[0], col[1:]  # type: ignore[assignment]
+
+        if not all(isinstance(c, str) for c in cols) or not (isinstance(col, str)):
+            raise TypeError("all names should be `str`")
+
+        self._write.sort_cols = cast(List[str], cols)
+        return self
+
+    def save(
+        self,
+        path: Optional[str] = None,
+        format: Optional[str] = None,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """Saves the contents of the :class:`DataFrame` to a data source.
+
+        The data source is specified by the ``format`` and a set of ``options``.
+        If ``format`` is not specified, the default data source configured by
+        ``spark.sql.sources.default`` will be used.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str, optional
+            the path in a Hadoop supported file system
+        format : str, optional
+            the format used to save
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+        partitionBy : list, optional
+            names of partitioning columns
+        **options : dict
+            all other string options
+
+        Examples
+        --------
+        Write a DataFrame into a JSON file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a JSON file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("json").save(d)
+        ...
+        ...     # Read the JSON file as a DataFrame.
+        ...     spark.read.format('json').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode).options(**options)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        if format is not None:
+            self.format(format)
+        self._write.path = path
+        self._spark.execute_command(self._write.command(self._spark))
+
+    def insertInto(self, tableName: str, overwrite: Optional[bool] = None) -> None:
+        """Inserts the content of the :class:`DataFrame` to the specified table.
+
+        It requires that the schema of the :class:`DataFrame` is the same as the
+        schema of the table.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        overwrite : bool, optional
+            If true, overwrites existing data. Disabled by default
+
+        Notes
+        -----
+        Unlike :meth:`DataFrameWriter.saveAsTable`, :meth:`DataFrameWriter.insertInto` ignores
+        the column names and just uses position-based resolution.
+
+        Examples
+        --------
+        >>> _ = spark.sql("DROP TABLE IF EXISTS tblA")
+        >>> df = spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... )
+        >>> df.write.saveAsTable("tblA")
+
+        Insert the data into 'tblA' table but with different column names.
+
+        >>> df.selectExpr("age AS col1", "name AS col2").write.insertInto("tblA")
+        >>> spark.read.table("tblA").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE tblA")
+        """
+        if overwrite is not None:
+            self.mode("overwrite" if overwrite else "append")
+        self.saveAsTable(tableName)
+
+    def saveAsTable(
+        self,
+        name: str,
+        format: Optional[str] = None,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` as the specified table.
+
+        In the case the table already exists, behavior of this function depends on the
+        save mode, specified by the `mode` function (default to throwing an exception).
+        When `mode` is `Overwrite`, the schema of the :class:`DataFrame` does not need to be
+        the same as that of the existing table.
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Notes
+        -----
+        When `mode` is `Append`, if there is an existing table, we will use the format and
+        options of the existing table. The column order in the schema of the :class:`DataFrame`
+        doesn't need to be same as that of the existing table. Unlike
+        :meth:`DataFrameWriter.insertInto`, :meth:`DataFrameWriter.saveAsTable` will use the
+        column names to find the correct column positions.
+
+        Parameters
+        ----------
+        name : str
+            the table name
+        format : str, optional
+            the format used to save
+        mode : str, optional
+            one of `append`, `overwrite`, `error`, `errorifexists`, `ignore` \
+            (default: error)
+        partitionBy : str or list
+            names of partitioning columns
+        **options : dict
+            all other string options
+
+        Examples
+        --------
+        Creates a table from a DataFrame, and read it back.
+
+        >>> _ = spark.sql("DROP TABLE IF EXISTS tblA")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.saveAsTable("tblA")
+        >>> spark.read.table("tblA").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE tblA")
+        """
+        self.mode(mode).options(**options)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        if format is not None:
+            self.format(format)
+        self._write.table_name = name
+        self._spark.execute_command(self._write.command(self._spark))
+
+    def json(
+        self,
+        path: str,
+        mode: Optional[str] = None,
+        compression: Optional[str] = None,
+        dateFormat: Optional[str] = None,
+        timestampFormat: Optional[str] = None,
+        lineSep: Optional[str] = None,
+        encoding: Optional[str] = None,
+        ignoreNullFields: Optional[Union[bool, str]] = None,
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` in JSON format
+        (`JSON Lines text format or newline-delimited JSON <http://jsonlines.org/>`_) at the
+        specified path.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-json.html#data-source-option>`_
+            in the version you use.
+
+            .. # noqa
+
+        Examples
+        --------
+        Write a DataFrame into a JSON file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a JSON file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.json(d, mode="overwrite")
+        ...
+        ...     # Read the JSON file as a DataFrame.
+        ...     spark.read.format("json").load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode)
+        self._set_opts(
+            compression=compression,
+            dateFormat=dateFormat,
+            timestampFormat=timestampFormat,
+            lineSep=lineSep,
+            encoding=encoding,
+            ignoreNullFields=ignoreNullFields,
+        )
+        self.format("json").save(path)
+
+    def parquet(
+        self,
+        path: str,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        compression: Optional[str] = None,
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` in Parquet format at the specified path.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+        partitionBy : str or list, optional
+            names of partitioning columns
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#data-source-option>`_
+            in the version you use.
+
+            .. # noqa
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.parquet(d, mode="overwrite")
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.format("parquet").load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        self.option("compression", compression)
+        self.format("parquet").save(path)
+
+    def text(
+        self, path: str, compression: Optional[str] = None, lineSep: Optional[str] = None
+    ) -> None:
+        """Saves the content of the DataFrame in a text file at the specified path.
+        The text files will be encoded as UTF-8.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-text.html#data-source-option>`_
+            in the version you use.
+
+            .. # noqa
+
+        Notes
+        -----
+        The DataFrame must have only one column that is of string type.
+        Each row becomes a new line in the output file.
+
+        Examples
+        --------
+        Write a DataFrame into a text file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a text file
+        ...     df = spark.createDataFrame([("a",), ("b",), ("c",)], schema=["alphabets"])
+        ...     df.write.mode("overwrite").text(d)
+        ...
+        ...     # Read the text file as a DataFrame.
+        ...     spark.read.schema(df.schema).format("text").load(d).sort("alphabets").show()
+        +---------+
+        |alphabets|
+        +---------+
+        |        a|
+        |        b|
+        |        c|
+        +---------+
+        """
+        self._set_opts(compression=compression, lineSep=lineSep)
+        self.format("text").save(path)
+
+    def csv(
+        self,
+        path: str,
+        mode: Optional[str] = None,
+        compression: Optional[str] = None,
+        sep: Optional[str] = None,
+        quote: Optional[str] = None,
+        escape: Optional[str] = None,
+        header: Optional[Union[bool, str]] = None,
+        nullValue: Optional[str] = None,
+        escapeQuotes: Optional[Union[bool, str]] = None,
+        quoteAll: Optional[Union[bool, str]] = None,
+        dateFormat: Optional[str] = None,
+        timestampFormat: Optional[str] = None,
+        ignoreLeadingWhiteSpace: Optional[Union[bool, str]] = None,
+        ignoreTrailingWhiteSpace: Optional[Union[bool, str]] = None,
+        charToEscapeQuoteEscaping: Optional[str] = None,
+        encoding: Optional[str] = None,
+        emptyValue: Optional[str] = None,
+        lineSep: Optional[str] = None,
+    ) -> None:
+        r"""Saves the content of the :class:`DataFrame` in CSV format at the specified path.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-csv.html#data-source-option>`_
+            in the version you use.
+
+            .. # noqa
+
+        Examples
+        --------
+        Write a DataFrame into a CSV file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file
+        ...     df = spark.createDataFrame([{"age": 100, "name": "Hyukjin Kwon"}])

Review Comment:
   None? 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] hvanhovell commented on a diff in pull request #38801: [SPARK-40539] [CONNECT] [PYTHON] Add basic support for DataFrameWriter

Posted by GitBox <gi...@apache.org>.
hvanhovell commented on code in PR #38801:
URL: https://github.com/apache/spark/pull/38801#discussion_r1032700242


##########
python/pyspark/sql/connect/writer.py:
##########
@@ -0,0 +1,922 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+
+from typing import Optional, Union, List, overload, Tuple, cast, Any
+from typing import TYPE_CHECKING
+
+from pyspark.sql.connect.plan import WriteOperation, LogicalPlan
+from pyspark.sql.types import StructType
+from pyspark.sql.utils import to_str
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect._typing import OptionalPrimitiveType
+    from pyspark.sql.connect.client import RemoteSparkSession
+
+
+PathOrPaths = Union[str, List[str]]
+TupleOrListOfString = Union[List[str], Tuple[str, ...]]
+
+
+class OptionUtils:
+    def _set_opts(
+        self,
+        schema: Optional[Union[StructType, str]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """
+        Set named options (filter out those the value is None)
+        """
+        if schema is not None:
+            self.schema(schema)  # type: ignore[attr-defined]
+        for k, v in options.items():
+            if v is not None:
+                self.option(k, v)  # type: ignore[attr-defined]
+
+
+class DataFrameWriter(OptionUtils):
+    """
+    Interface used to write a :class:`DataFrame` to external storage systems
+    (e.g. file systems, key-value stores, etc). Use :attr:`DataFrame.write`
+    to access this.
+
+    .. versionadded:: 3.4.0
+    """
+
+    def __init__(self, plan: "LogicalPlan", session: "RemoteSparkSession"):
+        self._df: "LogicalPlan" = plan
+        self._spark: "RemoteSparkSession" = session
+        self._write: "WriteOperation" = WriteOperation(self._df)
+
+    def mode(self, saveMode: Optional[str]) -> "DataFrameWriter":
+        """Specifies the behavior when data or table already exists.
+
+        Options include:
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Examples
+        --------
+        Raise an error when writing to an existing path.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     spark.createDataFrame(
+        ...         [{"age": 80, "name": "Xinrong Meng"}]
+        ...     ).write.mode("error").format("parquet").save(d)
+        Traceback (most recent call last):
+            ...
+        pyspark.sql.utils.AnalysisException: ...
+
+        Write a Parquet file back with various options, and read it back.
+
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Overwrite the path with a new Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 120, "name": "Takuya Ueshin"}]
+        ...     ).write.mode("append").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 140, "name": "Haejoon Lee"}]
+        ...     ).write.mode("ignore").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |120|Takuya Ueshin|
+        |100| Hyukjin Kwon|
+        +---+-------------+
+        """
+        # At the JVM side, the default value of mode is already set to "error".
+        # So, if the given saveMode is None, we will not call JVM-side's mode method.
+        if saveMode is not None:
+            self._write.mode = saveMode
+        return self
+
+    def format(self, source: str) -> "DataFrameWriter":
+        """Specifies the underlying output data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        source : str
+            string, name of the data source, e.g. 'json', 'parquet'.
+
+        Examples
+        --------
+        >>> spark.range(1).write.format('parquet')
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Write a DataFrame into a Parquet file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.format('parquet').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.source = source
+        return self
+
+    def option(self, key: str, value: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds a output option for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        key : str
+            The key for the option to set.
+        value
+            The value for the option to set.
+
+        Examples
+        --------
+        >>> spark.range(1).write.option("key", "value")
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Specify the option 'nullValue' with writing a CSV file.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file with 'nullValue' option set to 'Hyukjin Kwon'.
+        ...     df = spark.createDataFrame([(100, None)], "age INT, name STRING")
+        ...     df.write.option("nullValue", "Hyukjin Kwon").mode("overwrite").format("csv").save(d)
+        ...
+        ...     # Read the CSV file as a DataFrame.
+        ...     spark.read.schema(df.schema).format('csv').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.options[key] = to_str(value)
+        return self
+
+    def options(self, **options: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds output options for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        **options : dict
+            The dictionary of string keys and prmitive-type values.

Review Comment:
   It is easier to show what is wrong and how we should correct it, e.g.: `prmtive` -> `primitive`. It took me 30 seconds to discover what was wrong.
   
   BTW we should update the original writer.



##########
python/pyspark/sql/connect/writer.py:
##########
@@ -0,0 +1,922 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+
+from typing import Optional, Union, List, overload, Tuple, cast, Any
+from typing import TYPE_CHECKING
+
+from pyspark.sql.connect.plan import WriteOperation, LogicalPlan
+from pyspark.sql.types import StructType
+from pyspark.sql.utils import to_str
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect._typing import OptionalPrimitiveType
+    from pyspark.sql.connect.client import RemoteSparkSession
+
+
+PathOrPaths = Union[str, List[str]]
+TupleOrListOfString = Union[List[str], Tuple[str, ...]]
+
+
+class OptionUtils:
+    def _set_opts(
+        self,
+        schema: Optional[Union[StructType, str]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """
+        Set named options (filter out those the value is None)
+        """
+        if schema is not None:
+            self.schema(schema)  # type: ignore[attr-defined]
+        for k, v in options.items():
+            if v is not None:
+                self.option(k, v)  # type: ignore[attr-defined]
+
+
+class DataFrameWriter(OptionUtils):
+    """
+    Interface used to write a :class:`DataFrame` to external storage systems
+    (e.g. file systems, key-value stores, etc). Use :attr:`DataFrame.write`
+    to access this.
+
+    .. versionadded:: 3.4.0
+    """
+
+    def __init__(self, plan: "LogicalPlan", session: "RemoteSparkSession"):
+        self._df: "LogicalPlan" = plan
+        self._spark: "RemoteSparkSession" = session
+        self._write: "WriteOperation" = WriteOperation(self._df)
+
+    def mode(self, saveMode: Optional[str]) -> "DataFrameWriter":
+        """Specifies the behavior when data or table already exists.
+
+        Options include:
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Examples
+        --------
+        Raise an error when writing to an existing path.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     spark.createDataFrame(
+        ...         [{"age": 80, "name": "Xinrong Meng"}]
+        ...     ).write.mode("error").format("parquet").save(d)
+        Traceback (most recent call last):
+            ...
+        pyspark.sql.utils.AnalysisException: ...
+
+        Write a Parquet file back with various options, and read it back.
+
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Overwrite the path with a new Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 120, "name": "Takuya Ueshin"}]
+        ...     ).write.mode("append").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 140, "name": "Haejoon Lee"}]
+        ...     ).write.mode("ignore").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |120|Takuya Ueshin|
+        |100| Hyukjin Kwon|
+        +---+-------------+
+        """
+        # At the JVM side, the default value of mode is already set to "error".
+        # So, if the given saveMode is None, we will not call JVM-side's mode method.
+        if saveMode is not None:
+            self._write.mode = saveMode
+        return self
+
+    def format(self, source: str) -> "DataFrameWriter":
+        """Specifies the underlying output data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        source : str
+            string, name of the data source, e.g. 'json', 'parquet'.
+
+        Examples
+        --------
+        >>> spark.range(1).write.format('parquet')
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Write a DataFrame into a Parquet file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.format('parquet').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.source = source
+        return self
+
+    def option(self, key: str, value: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds a output option for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        key : str
+            The key for the option to set.
+        value
+            The value for the option to set.
+
+        Examples
+        --------
+        >>> spark.range(1).write.option("key", "value")
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Specify the option 'nullValue' with writing a CSV file.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file with 'nullValue' option set to 'Hyukjin Kwon'.
+        ...     df = spark.createDataFrame([(100, None)], "age INT, name STRING")
+        ...     df.write.option("nullValue", "Hyukjin Kwon").mode("overwrite").format("csv").save(d)
+        ...
+        ...     # Read the CSV file as a DataFrame.
+        ...     spark.read.schema(df.schema).format('csv').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.options[key] = to_str(value)
+        return self
+
+    def options(self, **options: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds output options for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        **options : dict
+            The dictionary of string keys and prmitive-type values.
+
+        Examples
+        --------
+        >>> spark.range(1).write.option("key", "value")
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Specify the option 'nullValue' and 'header' with writing a CSV file.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file with 'nullValue' option set to 'Hyukjin Kwon',
+        ...     # and 'header' option set to `True`.
+        ...     df = spark.createDataFrame([(100, "Hyukjin Kwon")], ["age", "name"])
+        ...     df.write.options(nullValue="Hyukjin Kwon", header=True).mode(
+        ...         "overwrite").format("csv").save(d)
+        ...
+        ...     # Read the CSV file as a DataFrame.
+        ...     spark.read.option("header", True).format('csv').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        for k in options:
+            self._write.options[k] = to_str(options[k])
+        return self
+
+    @overload
+    def partitionBy(self, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def partitionBy(self, *cols: List[str]) -> "DataFrameWriter":
+        ...
+
+    def partitionBy(self, *cols: Union[str, List[str]]) -> "DataFrameWriter":
+        """Partitions the output by the given columns on the file system.
+
+        If specified, the output is laid out on the file system similar
+        to Hive's partitioning scheme.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        cols : str or list
+            name of columns
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a partitioned manner, and read it back.
+
+        >>> import tempfile
+        >>> import os
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file in a partitioned manner.
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}, {"age": 120, "name": "Ruifeng Zheng"}]
+        ...     ).write.partitionBy("name").mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).sort("age").show()
+        ...
+        ...     # Read one partition as a DataFrame.
+        ...     spark.read.parquet(f"{d}{os.path.sep}name=Hyukjin Kwon").show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |100| Hyukjin Kwon|
+        |120|Ruifeng Zheng|
+        +---+-------------+
+        +---+
+        |age|
+        +---+
+        |100|
+        +---+
+        """
+        if len(cols) == 1 and isinstance(cols[0], (list, tuple)):
+            cols = cols[0]  # type: ignore[assignment]
+
+        self._write.partitioning_cols = cast(List[str], cols)
+        return self
+
+    @overload
+    def bucketBy(self, numBuckets: int, col: str, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def bucketBy(self, numBuckets: int, col: TupleOrListOfString) -> "DataFrameWriter":
+        ...
+
+    def bucketBy(
+        self, numBuckets: int, col: Union[str, TupleOrListOfString], *cols: Optional[str]
+    ) -> "DataFrameWriter":
+        """Buckets the output by the given columns. If specified,
+        the output is laid out on the file system similar to Hive's bucketing scheme,
+        but with a different bucket hash function and is not compatible with Hive's bucketing.
+
+        .. versionadded:: 2.3.0
+
+        Parameters
+        ----------
+        numBuckets : int
+            the number of buckets to save
+        col : str, list or tuple
+            a name of a column, or a list of names.
+        cols : str
+            additional names (optional). If `col` is a list it should be empty.
+
+        Notes
+        -----
+        Applicable for file-based data sources in combination with
+        :py:meth:`DataFrameWriter.saveAsTable`.
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a buckted manner, and read it back.
+
+        >>> from pyspark.sql.functions import input_file_name
+        >>> # Write a DataFrame into a Parquet file in a bucketed manner.
+        ... _ = spark.sql("DROP TABLE IF EXISTS bucketed_table")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.bucketBy(2, "name").mode("overwrite").saveAsTable("bucketed_table")
+        >>> # Read the Parquet file as a DataFrame.
+        ... spark.read.table("bucketed_table").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE bucketed_table")
+        """
+        if not isinstance(numBuckets, int):
+            raise TypeError("numBuckets should be an int, got {0}.".format(type(numBuckets)))
+
+        if isinstance(col, (list, tuple)):
+            if cols:
+                raise ValueError("col is a {0} but cols are not empty".format(type(col)))
+
+            col, cols = col[0], col[1:]  # type: ignore[assignment]
+
+        if not all(isinstance(c, str) for c in cols) or not (isinstance(col, str)):
+            raise TypeError("all names should be `str`")
+
+        self._write.num_buckets = numBuckets
+        self._write.bucket_cols = cast(List[str], cols)
+        return self
+
+    @overload
+    def sortBy(self, col: str, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def sortBy(self, col: TupleOrListOfString) -> "DataFrameWriter":
+        ...
+
+    def sortBy(
+        self, col: Union[str, TupleOrListOfString], *cols: Optional[str]
+    ) -> "DataFrameWriter":
+        """Sorts the output in each bucket by the given columns on the file system.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        col : str, tuple or list
+            a name of a column, or a list of names.
+        cols : str
+            additional names (optional). If `col` is a list it should be empty.
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a sorted-buckted manner, and read it back.
+
+        >>> from pyspark.sql.functions import input_file_name
+        >>> # Write a DataFrame into a Parquet file in a sorted-bucketed manner.
+        ... _ = spark.sql("DROP TABLE IF EXISTS sorted_bucketed_table")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.bucketBy(1, "name").sortBy("age").mode(
+        ...     "overwrite").saveAsTable("sorted_bucketed_table")
+        >>> # Read the Parquet file as a DataFrame.
+        ... spark.read.table("sorted_bucketed_table").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE sorted_bucketed_table")
+        """
+        if isinstance(col, (list, tuple)):
+            if cols:
+                raise ValueError("col is a {0} but cols are not empty".format(type(col)))
+
+            col, cols = col[0], col[1:]  # type: ignore[assignment]
+
+        if not all(isinstance(c, str) for c in cols) or not (isinstance(col, str)):
+            raise TypeError("all names should be `str`")
+
+        self._write.sort_cols = cast(List[str], cols)
+        return self
+
+    def save(
+        self,
+        path: Optional[str] = None,
+        format: Optional[str] = None,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """Saves the contents of the :class:`DataFrame` to a data source.
+
+        The data source is specified by the ``format`` and a set of ``options``.
+        If ``format`` is not specified, the default data source configured by
+        ``spark.sql.sources.default`` will be used.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str, optional
+            the path in a Hadoop supported file system
+        format : str, optional
+            the format used to save
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+        partitionBy : list, optional
+            names of partitioning columns
+        **options : dict
+            all other string options
+
+        Examples
+        --------
+        Write a DataFrame into a JSON file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a JSON file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("json").save(d)
+        ...
+        ...     # Read the JSON file as a DataFrame.
+        ...     spark.read.format('json').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode).options(**options)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        if format is not None:
+            self.format(format)
+        self._write.path = path
+        self._spark.execute_command(self._write.command(self._spark))
+
+    def insertInto(self, tableName: str, overwrite: Optional[bool] = None) -> None:
+        """Inserts the content of the :class:`DataFrame` to the specified table.
+
+        It requires that the schema of the :class:`DataFrame` is the same as the
+        schema of the table.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        overwrite : bool, optional
+            If true, overwrites existing data. Disabled by default
+
+        Notes
+        -----
+        Unlike :meth:`DataFrameWriter.saveAsTable`, :meth:`DataFrameWriter.insertInto` ignores
+        the column names and just uses position-based resolution.
+
+        Examples
+        --------
+        >>> _ = spark.sql("DROP TABLE IF EXISTS tblA")
+        >>> df = spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... )
+        >>> df.write.saveAsTable("tblA")
+
+        Insert the data into 'tblA' table but with different column names.
+
+        >>> df.selectExpr("age AS col1", "name AS col2").write.insertInto("tblA")
+        >>> spark.read.table("tblA").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE tblA")
+        """
+        if overwrite is not None:
+            self.mode("overwrite" if overwrite else "append")
+        self.saveAsTable(tableName)
+
+    def saveAsTable(
+        self,
+        name: str,
+        format: Optional[str] = None,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` as the specified table.
+
+        In the case the table already exists, behavior of this function depends on the
+        save mode, specified by the `mode` function (default to throwing an exception).
+        When `mode` is `Overwrite`, the schema of the :class:`DataFrame` does not need to be
+        the same as that of the existing table.
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Notes
+        -----
+        When `mode` is `Append`, if there is an existing table, we will use the format and
+        options of the existing table. The column order in the schema of the :class:`DataFrame`
+        doesn't need to be same as that of the existing table. Unlike
+        :meth:`DataFrameWriter.insertInto`, :meth:`DataFrameWriter.saveAsTable` will use the
+        column names to find the correct column positions.
+
+        Parameters
+        ----------
+        name : str
+            the table name
+        format : str, optional
+            the format used to save
+        mode : str, optional
+            one of `append`, `overwrite`, `error`, `errorifexists`, `ignore` \
+            (default: error)
+        partitionBy : str or list
+            names of partitioning columns
+        **options : dict
+            all other string options
+
+        Examples
+        --------
+        Creates a table from a DataFrame, and read it back.
+
+        >>> _ = spark.sql("DROP TABLE IF EXISTS tblA")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.saveAsTable("tblA")
+        >>> spark.read.table("tblA").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE tblA")
+        """
+        self.mode(mode).options(**options)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        if format is not None:
+            self.format(format)
+        self._write.table_name = name
+        self._spark.execute_command(self._write.command(self._spark))
+
+    def json(
+        self,
+        path: str,
+        mode: Optional[str] = None,
+        compression: Optional[str] = None,
+        dateFormat: Optional[str] = None,
+        timestampFormat: Optional[str] = None,
+        lineSep: Optional[str] = None,
+        encoding: Optional[str] = None,
+        ignoreNullFields: Optional[Union[bool, str]] = None,
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` in JSON format
+        (`JSON Lines text format or newline-delimited JSON <http://jsonlines.org/>`_) at the
+        specified path.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-json.html#data-source-option>`_
+            in the version you use.
+
+            .. # noqa
+
+        Examples
+        --------
+        Write a DataFrame into a JSON file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a JSON file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.json(d, mode="overwrite")
+        ...
+        ...     # Read the JSON file as a DataFrame.
+        ...     spark.read.format("json").load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode)
+        self._set_opts(
+            compression=compression,
+            dateFormat=dateFormat,
+            timestampFormat=timestampFormat,
+            lineSep=lineSep,
+            encoding=encoding,
+            ignoreNullFields=ignoreNullFields,
+        )
+        self.format("json").save(path)
+
+    def parquet(
+        self,
+        path: str,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        compression: Optional[str] = None,
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` in Parquet format at the specified path.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+        partitionBy : str or list, optional
+            names of partitioning columns
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#data-source-option>`_
+            in the version you use.
+
+            .. # noqa
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.parquet(d, mode="overwrite")
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.format("parquet").load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        self.option("compression", compression)
+        self.format("parquet").save(path)
+
+    def text(
+        self, path: str, compression: Optional[str] = None, lineSep: Optional[str] = None
+    ) -> None:
+        """Saves the content of the DataFrame in a text file at the specified path.
+        The text files will be encoded as UTF-8.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-text.html#data-source-option>`_
+            in the version you use.
+
+            .. # noqa
+
+        Notes
+        -----
+        The DataFrame must have only one column that is of string type.
+        Each row becomes a new line in the output file.
+
+        Examples
+        --------
+        Write a DataFrame into a text file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a text file
+        ...     df = spark.createDataFrame([("a",), ("b",), ("c",)], schema=["alphabets"])
+        ...     df.write.mode("overwrite").text(d)
+        ...
+        ...     # Read the text file as a DataFrame.
+        ...     spark.read.schema(df.schema).format("text").load(d).sort("alphabets").show()
+        +---------+
+        |alphabets|
+        +---------+
+        |        a|
+        |        b|
+        |        c|
+        +---------+
+        """
+        self._set_opts(compression=compression, lineSep=lineSep)
+        self.format("text").save(path)
+
+    def csv(
+        self,
+        path: str,
+        mode: Optional[str] = None,
+        compression: Optional[str] = None,
+        sep: Optional[str] = None,
+        quote: Optional[str] = None,
+        escape: Optional[str] = None,
+        header: Optional[Union[bool, str]] = None,
+        nullValue: Optional[str] = None,
+        escapeQuotes: Optional[Union[bool, str]] = None,
+        quoteAll: Optional[Union[bool, str]] = None,
+        dateFormat: Optional[str] = None,
+        timestampFormat: Optional[str] = None,
+        ignoreLeadingWhiteSpace: Optional[Union[bool, str]] = None,
+        ignoreTrailingWhiteSpace: Optional[Union[bool, str]] = None,
+        charToEscapeQuoteEscaping: Optional[str] = None,
+        encoding: Optional[str] = None,
+        emptyValue: Optional[str] = None,
+        lineSep: Optional[str] = None,
+    ) -> None:
+        r"""Saves the content of the :class:`DataFrame` in CSV format at the specified path.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-csv.html#data-source-option>`_
+            in the version you use.
+
+            .. # noqa
+
+        Examples
+        --------
+        Write a DataFrame into a CSV file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file
+        ...     df = spark.createDataFrame([{"age": 100, "name": "Hyukjin Kwon"}])
+        ...     df.write.csv(d, mode="overwrite")
+        ...
+        ...     # Read the CSV file as a DataFrame with 'nullValue' option set to 'Hyukjin Kwon'.
+        ...     spark.read.schema(df.schema).format("csv").option(
+        ...         "nullValue", "Hyukjin Kwon").load(d).show()
+        +---+----+
+        |age|name|
+        +---+----+
+        |100|null|
+        +---+----+
+        """
+        self.mode(mode)
+        self._set_opts(
+            compression=compression,
+            sep=sep,
+            quote=quote,
+            escape=escape,
+            header=header,
+            nullValue=nullValue,
+            escapeQuotes=escapeQuotes,
+            quoteAll=quoteAll,
+            dateFormat=dateFormat,
+            timestampFormat=timestampFormat,
+            ignoreLeadingWhiteSpace=ignoreLeadingWhiteSpace,
+            ignoreTrailingWhiteSpace=ignoreTrailingWhiteSpace,
+            charToEscapeQuoteEscaping=charToEscapeQuoteEscaping,
+            encoding=encoding,
+            emptyValue=emptyValue,
+            lineSep=lineSep,
+        )
+        self.format("csv").save(path)
+
+    def orc(
+        self,
+        path: str,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        compression: Optional[str] = None,
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` in ORC format at the specified path.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+        partitionBy : str or list, optional
+            names of partitioning columns
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-orc.html#data-source-option>`_
+            in the version you use.
+
+            .. # noqa
+
+        Examples
+        --------
+        Write a DataFrame into a ORC file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a ORC file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.orc(d, mode="overwrite")
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.format("orc").load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        self._set_opts(compression=compression)
+        self.format("orc").save(path)
+
+    def jdbc(self, *args: Any, **kwargs: Any) -> None:

Review Comment:
   SGTM



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] bjornjorgensen commented on a diff in pull request #38801: [SPARK-40539] [CONNECT] [PYTHON] Add basic support for DataFrameWriter

Posted by GitBox <gi...@apache.org>.
bjornjorgensen commented on code in PR #38801:
URL: https://github.com/apache/spark/pull/38801#discussion_r1032786273


##########
python/pyspark/sql/connect/writer.py:
##########
@@ -0,0 +1,922 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+
+from typing import Optional, Union, List, overload, Tuple, cast, Any
+from typing import TYPE_CHECKING
+
+from pyspark.sql.connect.plan import WriteOperation, LogicalPlan
+from pyspark.sql.types import StructType
+from pyspark.sql.utils import to_str
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect._typing import OptionalPrimitiveType
+    from pyspark.sql.connect.client import RemoteSparkSession
+
+
+PathOrPaths = Union[str, List[str]]
+TupleOrListOfString = Union[List[str], Tuple[str, ...]]
+
+
+class OptionUtils:
+    def _set_opts(
+        self,
+        schema: Optional[Union[StructType, str]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """
+        Set named options (filter out those the value is None)
+        """
+        if schema is not None:
+            self.schema(schema)  # type: ignore[attr-defined]
+        for k, v in options.items():
+            if v is not None:
+                self.option(k, v)  # type: ignore[attr-defined]
+
+
+class DataFrameWriter(OptionUtils):
+    """
+    Interface used to write a :class:`DataFrame` to external storage systems
+    (e.g. file systems, key-value stores, etc). Use :attr:`DataFrame.write`
+    to access this.
+
+    .. versionadded:: 3.4.0
+    """
+
+    def __init__(self, plan: "LogicalPlan", session: "RemoteSparkSession"):
+        self._df: "LogicalPlan" = plan
+        self._spark: "RemoteSparkSession" = session
+        self._write: "WriteOperation" = WriteOperation(self._df)
+
+    def mode(self, saveMode: Optional[str]) -> "DataFrameWriter":
+        """Specifies the behavior when data or table already exists.
+
+        Options include:
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Examples
+        --------
+        Raise an error when writing to an existing path.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     spark.createDataFrame(
+        ...         [{"age": 80, "name": "Xinrong Meng"}]
+        ...     ).write.mode("error").format("parquet").save(d)
+        Traceback (most recent call last):
+            ...
+        pyspark.sql.utils.AnalysisException: ...
+
+        Write a Parquet file back with various options, and read it back.
+
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Overwrite the path with a new Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 120, "name": "Takuya Ueshin"}]
+        ...     ).write.mode("append").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 140, "name": "Haejoon Lee"}]
+        ...     ).write.mode("ignore").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |120|Takuya Ueshin|
+        |100| Hyukjin Kwon|
+        +---+-------------+
+        """
+        # At the JVM side, the default value of mode is already set to "error".
+        # So, if the given saveMode is None, we will not call JVM-side's mode method.
+        if saveMode is not None:
+            self._write.mode = saveMode
+        return self
+
+    def format(self, source: str) -> "DataFrameWriter":
+        """Specifies the underlying output data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        source : str
+            string, name of the data source, e.g. 'json', 'parquet'.
+
+        Examples
+        --------
+        >>> spark.range(1).write.format('parquet')
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Write a DataFrame into a Parquet file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.format('parquet').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.source = source
+        return self
+
+    def option(self, key: str, value: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds a output option for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        key : str
+            The key for the option to set.
+        value
+            The value for the option to set.
+
+        Examples
+        --------
+        >>> spark.range(1).write.option("key", "value")
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Specify the option 'nullValue' with writing a CSV file.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file with 'nullValue' option set to 'Hyukjin Kwon'.
+        ...     df = spark.createDataFrame([(100, None)], "age INT, name STRING")
+        ...     df.write.option("nullValue", "Hyukjin Kwon").mode("overwrite").format("csv").save(d)
+        ...
+        ...     # Read the CSV file as a DataFrame.
+        ...     spark.read.schema(df.schema).format('csv').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.options[key] = to_str(value)
+        return self
+
+    def options(self, **options: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds output options for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        **options : dict
+            The dictionary of string keys and prmitive-type values.
+
+        Examples
+        --------
+        >>> spark.range(1).write.option("key", "value")
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Specify the option 'nullValue' and 'header' with writing a CSV file.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file with 'nullValue' option set to 'Hyukjin Kwon',
+        ...     # and 'header' option set to `True`.
+        ...     df = spark.createDataFrame([(100, "Hyukjin Kwon")], ["age", "name"])
+        ...     df.write.options(nullValue="Hyukjin Kwon", header=True).mode(
+        ...         "overwrite").format("csv").save(d)
+        ...
+        ...     # Read the CSV file as a DataFrame.
+        ...     spark.read.option("header", True).format('csv').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        for k in options:
+            self._write.options[k] = to_str(options[k])
+        return self
+
+    @overload
+    def partitionBy(self, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def partitionBy(self, *cols: List[str]) -> "DataFrameWriter":
+        ...
+
+    def partitionBy(self, *cols: Union[str, List[str]]) -> "DataFrameWriter":
+        """Partitions the output by the given columns on the file system.
+
+        If specified, the output is laid out on the file system similar
+        to Hive's partitioning scheme.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        cols : str or list
+            name of columns
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a partitioned manner, and read it back.
+
+        >>> import tempfile
+        >>> import os
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file in a partitioned manner.
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}, {"age": 120, "name": "Ruifeng Zheng"}]
+        ...     ).write.partitionBy("name").mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).sort("age").show()
+        ...
+        ...     # Read one partition as a DataFrame.
+        ...     spark.read.parquet(f"{d}{os.path.sep}name=Hyukjin Kwon").show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |100| Hyukjin Kwon|
+        |120|Ruifeng Zheng|
+        +---+-------------+
+        +---+
+        |age|
+        +---+
+        |100|
+        +---+
+        """
+        if len(cols) == 1 and isinstance(cols[0], (list, tuple)):
+            cols = cols[0]  # type: ignore[assignment]
+
+        self._write.partitioning_cols = cast(List[str], cols)
+        return self
+
+    @overload
+    def bucketBy(self, numBuckets: int, col: str, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def bucketBy(self, numBuckets: int, col: TupleOrListOfString) -> "DataFrameWriter":
+        ...
+
+    def bucketBy(
+        self, numBuckets: int, col: Union[str, TupleOrListOfString], *cols: Optional[str]
+    ) -> "DataFrameWriter":
+        """Buckets the output by the given columns. If specified,
+        the output is laid out on the file system similar to Hive's bucketing scheme,
+        but with a different bucket hash function and is not compatible with Hive's bucketing.
+
+        .. versionadded:: 2.3.0
+
+        Parameters
+        ----------
+        numBuckets : int
+            the number of buckets to save
+        col : str, list or tuple
+            a name of a column, or a list of names.
+        cols : str
+            additional names (optional). If `col` is a list it should be empty.
+
+        Notes
+        -----
+        Applicable for file-based data sources in combination with
+        :py:meth:`DataFrameWriter.saveAsTable`.
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a buckted manner, and read it back.
+
+        >>> from pyspark.sql.functions import input_file_name
+        >>> # Write a DataFrame into a Parquet file in a bucketed manner.
+        ... _ = spark.sql("DROP TABLE IF EXISTS bucketed_table")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.bucketBy(2, "name").mode("overwrite").saveAsTable("bucketed_table")
+        >>> # Read the Parquet file as a DataFrame.
+        ... spark.read.table("bucketed_table").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE bucketed_table")
+        """
+        if not isinstance(numBuckets, int):
+            raise TypeError("numBuckets should be an int, got {0}.".format(type(numBuckets)))
+
+        if isinstance(col, (list, tuple)):
+            if cols:
+                raise ValueError("col is a {0} but cols are not empty".format(type(col)))
+
+            col, cols = col[0], col[1:]  # type: ignore[assignment]
+
+        if not all(isinstance(c, str) for c in cols) or not (isinstance(col, str)):
+            raise TypeError("all names should be `str`")
+
+        self._write.num_buckets = numBuckets
+        self._write.bucket_cols = cast(List[str], cols)
+        return self
+
+    @overload
+    def sortBy(self, col: str, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def sortBy(self, col: TupleOrListOfString) -> "DataFrameWriter":
+        ...
+
+    def sortBy(
+        self, col: Union[str, TupleOrListOfString], *cols: Optional[str]
+    ) -> "DataFrameWriter":
+        """Sorts the output in each bucket by the given columns on the file system.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        col : str, tuple or list
+            a name of a column, or a list of names.
+        cols : str
+            additional names (optional). If `col` is a list it should be empty.
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a sorted-buckted manner, and read it back.
+
+        >>> from pyspark.sql.functions import input_file_name
+        >>> # Write a DataFrame into a Parquet file in a sorted-bucketed manner.
+        ... _ = spark.sql("DROP TABLE IF EXISTS sorted_bucketed_table")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.bucketBy(1, "name").sortBy("age").mode(
+        ...     "overwrite").saveAsTable("sorted_bucketed_table")
+        >>> # Read the Parquet file as a DataFrame.
+        ... spark.read.table("sorted_bucketed_table").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE sorted_bucketed_table")
+        """
+        if isinstance(col, (list, tuple)):
+            if cols:
+                raise ValueError("col is a {0} but cols are not empty".format(type(col)))
+
+            col, cols = col[0], col[1:]  # type: ignore[assignment]
+
+        if not all(isinstance(c, str) for c in cols) or not (isinstance(col, str)):
+            raise TypeError("all names should be `str`")
+
+        self._write.sort_cols = cast(List[str], cols)
+        return self
+
+    def save(
+        self,
+        path: Optional[str] = None,
+        format: Optional[str] = None,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """Saves the contents of the :class:`DataFrame` to a data source.
+
+        The data source is specified by the ``format`` and a set of ``options``.
+        If ``format`` is not specified, the default data source configured by
+        ``spark.sql.sources.default`` will be used.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str, optional
+            the path in a Hadoop supported file system
+        format : str, optional
+            the format used to save
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+        partitionBy : list, optional
+            names of partitioning columns
+        **options : dict
+            all other string options
+
+        Examples
+        --------
+        Write a DataFrame into a JSON file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a JSON file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("json").save(d)
+        ...
+        ...     # Read the JSON file as a DataFrame.
+        ...     spark.read.format('json').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode).options(**options)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        if format is not None:
+            self.format(format)
+        self._write.path = path
+        self._spark.execute_command(self._write.command(self._spark))
+
+    def insertInto(self, tableName: str, overwrite: Optional[bool] = None) -> None:
+        """Inserts the content of the :class:`DataFrame` to the specified table.
+
+        It requires that the schema of the :class:`DataFrame` is the same as the
+        schema of the table.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        overwrite : bool, optional
+            If true, overwrites existing data. Disabled by default
+
+        Notes
+        -----
+        Unlike :meth:`DataFrameWriter.saveAsTable`, :meth:`DataFrameWriter.insertInto` ignores
+        the column names and just uses position-based resolution.
+
+        Examples
+        --------
+        >>> _ = spark.sql("DROP TABLE IF EXISTS tblA")
+        >>> df = spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... )
+        >>> df.write.saveAsTable("tblA")
+
+        Insert the data into 'tblA' table but with different column names.
+
+        >>> df.selectExpr("age AS col1", "name AS col2").write.insertInto("tblA")
+        >>> spark.read.table("tblA").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE tblA")
+        """
+        if overwrite is not None:
+            self.mode("overwrite" if overwrite else "append")
+        self.saveAsTable(tableName)
+
+    def saveAsTable(
+        self,
+        name: str,
+        format: Optional[str] = None,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` as the specified table.
+
+        In the case the table already exists, behavior of this function depends on the
+        save mode, specified by the `mode` function (default to throwing an exception).
+        When `mode` is `Overwrite`, the schema of the :class:`DataFrame` does not need to be
+        the same as that of the existing table.
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Notes
+        -----
+        When `mode` is `Append`, if there is an existing table, we will use the format and
+        options of the existing table. The column order in the schema of the :class:`DataFrame`
+        doesn't need to be same as that of the existing table. Unlike
+        :meth:`DataFrameWriter.insertInto`, :meth:`DataFrameWriter.saveAsTable` will use the
+        column names to find the correct column positions.
+
+        Parameters
+        ----------
+        name : str
+            the table name
+        format : str, optional
+            the format used to save
+        mode : str, optional
+            one of `append`, `overwrite`, `error`, `errorifexists`, `ignore` \
+            (default: error)
+        partitionBy : str or list
+            names of partitioning columns
+        **options : dict
+            all other string options
+
+        Examples
+        --------
+        Creates a table from a DataFrame, and read it back.
+
+        >>> _ = spark.sql("DROP TABLE IF EXISTS tblA")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.saveAsTable("tblA")
+        >>> spark.read.table("tblA").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE tblA")
+        """
+        self.mode(mode).options(**options)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        if format is not None:
+            self.format(format)
+        self._write.table_name = name
+        self._spark.execute_command(self._write.command(self._spark))
+
+    def json(
+        self,
+        path: str,
+        mode: Optional[str] = None,
+        compression: Optional[str] = None,
+        dateFormat: Optional[str] = None,
+        timestampFormat: Optional[str] = None,
+        lineSep: Optional[str] = None,
+        encoding: Optional[str] = None,
+        ignoreNullFields: Optional[Union[bool, str]] = None,
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` in JSON format
+        (`JSON Lines text format or newline-delimited JSON <http://jsonlines.org/>`_) at the
+        specified path.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-json.html#data-source-option>`_
+            in the version you use.
+
+            .. # noqa
+
+        Examples
+        --------
+        Write a DataFrame into a JSON file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a JSON file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.json(d, mode="overwrite")
+        ...
+        ...     # Read the JSON file as a DataFrame.
+        ...     spark.read.format("json").load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode)
+        self._set_opts(
+            compression=compression,
+            dateFormat=dateFormat,
+            timestampFormat=timestampFormat,
+            lineSep=lineSep,
+            encoding=encoding,
+            ignoreNullFields=ignoreNullFields,
+        )
+        self.format("json").save(path)
+
+    def parquet(
+        self,
+        path: str,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        compression: Optional[str] = None,
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` in Parquet format at the specified path.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+        partitionBy : str or list, optional
+            names of partitioning columns
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#data-source-option>`_
+            in the version you use.
+
+            .. # noqa
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.parquet(d, mode="overwrite")
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.format("parquet").load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        self.option("compression", compression)
+        self.format("parquet").save(path)
+
+    def text(
+        self, path: str, compression: Optional[str] = None, lineSep: Optional[str] = None
+    ) -> None:
+        """Saves the content of the DataFrame in a text file at the specified path.
+        The text files will be encoded as UTF-8.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-text.html#data-source-option>`_
+            in the version you use.
+
+            .. # noqa
+
+        Notes
+        -----
+        The DataFrame must have only one column that is of string type.
+        Each row becomes a new line in the output file.
+
+        Examples
+        --------
+        Write a DataFrame into a text file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a text file
+        ...     df = spark.createDataFrame([("a",), ("b",), ("c",)], schema=["alphabets"])
+        ...     df.write.mode("overwrite").text(d)
+        ...
+        ...     # Read the text file as a DataFrame.
+        ...     spark.read.schema(df.schema).format("text").load(d).sort("alphabets").show()
+        +---------+
+        |alphabets|
+        +---------+
+        |        a|
+        |        b|
+        |        c|
+        +---------+
+        """
+        self._set_opts(compression=compression, lineSep=lineSep)
+        self.format("text").save(path)
+
+    def csv(
+        self,
+        path: str,
+        mode: Optional[str] = None,
+        compression: Optional[str] = None,
+        sep: Optional[str] = None,
+        quote: Optional[str] = None,
+        escape: Optional[str] = None,
+        header: Optional[Union[bool, str]] = None,
+        nullValue: Optional[str] = None,
+        escapeQuotes: Optional[Union[bool, str]] = None,
+        quoteAll: Optional[Union[bool, str]] = None,
+        dateFormat: Optional[str] = None,
+        timestampFormat: Optional[str] = None,
+        ignoreLeadingWhiteSpace: Optional[Union[bool, str]] = None,
+        ignoreTrailingWhiteSpace: Optional[Union[bool, str]] = None,
+        charToEscapeQuoteEscaping: Optional[str] = None,
+        encoding: Optional[str] = None,
+        emptyValue: Optional[str] = None,
+        lineSep: Optional[str] = None,
+    ) -> None:
+        r"""Saves the content of the :class:`DataFrame` in CSV format at the specified path.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-csv.html#data-source-option>`_
+            in the version you use.
+
+            .. # noqa
+
+        Examples
+        --------
+        Write a DataFrame into a CSV file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file
+        ...     df = spark.createDataFrame([{"age": 100, "name": "Hyukjin Kwon"}])

Review Comment:
   The dataframe is [{"age": 100, "name": "Hyukjin Kwon"}]  and it's Read the CSV file as a DataFrame with 'nullValue' option set to 'Hyukjin Kwon'  why do we get a datafreme where name is null? 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] grundprinzip commented on a diff in pull request #38801: [SPARK-40539] [CONNECT] [PYTHON] Add basic support for DataFrameWriter

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #38801:
URL: https://github.com/apache/spark/pull/38801#discussion_r1032817398


##########
python/pyspark/sql/connect/writer.py:
##########
@@ -0,0 +1,922 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+
+from typing import Optional, Union, List, overload, Tuple, cast, Any
+from typing import TYPE_CHECKING
+
+from pyspark.sql.connect.plan import WriteOperation, LogicalPlan
+from pyspark.sql.types import StructType
+from pyspark.sql.utils import to_str
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect._typing import OptionalPrimitiveType
+    from pyspark.sql.connect.client import RemoteSparkSession
+
+
+PathOrPaths = Union[str, List[str]]
+TupleOrListOfString = Union[List[str], Tuple[str, ...]]
+
+
+class OptionUtils:
+    def _set_opts(
+        self,
+        schema: Optional[Union[StructType, str]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """
+        Set named options (filter out those the value is None)
+        """
+        if schema is not None:
+            self.schema(schema)  # type: ignore[attr-defined]
+        for k, v in options.items():
+            if v is not None:
+                self.option(k, v)  # type: ignore[attr-defined]
+
+
+class DataFrameWriter(OptionUtils):
+    """
+    Interface used to write a :class:`DataFrame` to external storage systems
+    (e.g. file systems, key-value stores, etc). Use :attr:`DataFrame.write`
+    to access this.
+
+    .. versionadded:: 3.4.0
+    """
+
+    def __init__(self, plan: "LogicalPlan", session: "RemoteSparkSession"):
+        self._df: "LogicalPlan" = plan
+        self._spark: "RemoteSparkSession" = session
+        self._write: "WriteOperation" = WriteOperation(self._df)
+
+    def mode(self, saveMode: Optional[str]) -> "DataFrameWriter":
+        """Specifies the behavior when data or table already exists.
+
+        Options include:
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Examples
+        --------
+        Raise an error when writing to an existing path.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     spark.createDataFrame(
+        ...         [{"age": 80, "name": "Xinrong Meng"}]
+        ...     ).write.mode("error").format("parquet").save(d)
+        Traceback (most recent call last):
+            ...
+        pyspark.sql.utils.AnalysisException: ...
+
+        Write a Parquet file back with various options, and read it back.
+
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Overwrite the path with a new Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 120, "name": "Takuya Ueshin"}]
+        ...     ).write.mode("append").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 140, "name": "Haejoon Lee"}]
+        ...     ).write.mode("ignore").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |120|Takuya Ueshin|
+        |100| Hyukjin Kwon|
+        +---+-------------+
+        """
+        # At the JVM side, the default value of mode is already set to "error".
+        # So, if the given saveMode is None, we will not call JVM-side's mode method.
+        if saveMode is not None:
+            self._write.mode = saveMode
+        return self
+
+    def format(self, source: str) -> "DataFrameWriter":
+        """Specifies the underlying output data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        source : str
+            string, name of the data source, e.g. 'json', 'parquet'.
+
+        Examples
+        --------
+        >>> spark.range(1).write.format('parquet')
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Write a DataFrame into a Parquet file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.format('parquet').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.source = source
+        return self
+
+    def option(self, key: str, value: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds a output option for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        key : str
+            The key for the option to set.
+        value
+            The value for the option to set.
+
+        Examples
+        --------
+        >>> spark.range(1).write.option("key", "value")
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Specify the option 'nullValue' with writing a CSV file.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file with 'nullValue' option set to 'Hyukjin Kwon'.
+        ...     df = spark.createDataFrame([(100, None)], "age INT, name STRING")
+        ...     df.write.option("nullValue", "Hyukjin Kwon").mode("overwrite").format("csv").save(d)
+        ...
+        ...     # Read the CSV file as a DataFrame.
+        ...     spark.read.schema(df.schema).format('csv').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.options[key] = to_str(value)
+        return self
+
+    def options(self, **options: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds output options for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        **options : dict
+            The dictionary of string keys and prmitive-type values.
+
+        Examples
+        --------
+        >>> spark.range(1).write.option("key", "value")
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Specify the option 'nullValue' and 'header' with writing a CSV file.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file with 'nullValue' option set to 'Hyukjin Kwon',
+        ...     # and 'header' option set to `True`.
+        ...     df = spark.createDataFrame([(100, "Hyukjin Kwon")], ["age", "name"])
+        ...     df.write.options(nullValue="Hyukjin Kwon", header=True).mode(
+        ...         "overwrite").format("csv").save(d)
+        ...
+        ...     # Read the CSV file as a DataFrame.
+        ...     spark.read.option("header", True).format('csv').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        for k in options:
+            self._write.options[k] = to_str(options[k])
+        return self
+
+    @overload
+    def partitionBy(self, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def partitionBy(self, *cols: List[str]) -> "DataFrameWriter":
+        ...
+
+    def partitionBy(self, *cols: Union[str, List[str]]) -> "DataFrameWriter":
+        """Partitions the output by the given columns on the file system.
+
+        If specified, the output is laid out on the file system similar
+        to Hive's partitioning scheme.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        cols : str or list
+            name of columns
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a partitioned manner, and read it back.
+
+        >>> import tempfile
+        >>> import os
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file in a partitioned manner.
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}, {"age": 120, "name": "Ruifeng Zheng"}]
+        ...     ).write.partitionBy("name").mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).sort("age").show()
+        ...
+        ...     # Read one partition as a DataFrame.
+        ...     spark.read.parquet(f"{d}{os.path.sep}name=Hyukjin Kwon").show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |100| Hyukjin Kwon|
+        |120|Ruifeng Zheng|
+        +---+-------------+
+        +---+
+        |age|
+        +---+
+        |100|
+        +---+
+        """
+        if len(cols) == 1 and isinstance(cols[0], (list, tuple)):
+            cols = cols[0]  # type: ignore[assignment]
+
+        self._write.partitioning_cols = cast(List[str], cols)
+        return self
+
+    @overload
+    def bucketBy(self, numBuckets: int, col: str, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def bucketBy(self, numBuckets: int, col: TupleOrListOfString) -> "DataFrameWriter":
+        ...
+
+    def bucketBy(
+        self, numBuckets: int, col: Union[str, TupleOrListOfString], *cols: Optional[str]
+    ) -> "DataFrameWriter":
+        """Buckets the output by the given columns. If specified,
+        the output is laid out on the file system similar to Hive's bucketing scheme,
+        but with a different bucket hash function and is not compatible with Hive's bucketing.
+
+        .. versionadded:: 2.3.0
+
+        Parameters
+        ----------
+        numBuckets : int
+            the number of buckets to save
+        col : str, list or tuple
+            a name of a column, or a list of names.
+        cols : str
+            additional names (optional). If `col` is a list it should be empty.
+
+        Notes
+        -----
+        Applicable for file-based data sources in combination with
+        :py:meth:`DataFrameWriter.saveAsTable`.
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a buckted manner, and read it back.
+
+        >>> from pyspark.sql.functions import input_file_name
+        >>> # Write a DataFrame into a Parquet file in a bucketed manner.
+        ... _ = spark.sql("DROP TABLE IF EXISTS bucketed_table")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.bucketBy(2, "name").mode("overwrite").saveAsTable("bucketed_table")
+        >>> # Read the Parquet file as a DataFrame.
+        ... spark.read.table("bucketed_table").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE bucketed_table")
+        """
+        if not isinstance(numBuckets, int):
+            raise TypeError("numBuckets should be an int, got {0}.".format(type(numBuckets)))
+
+        if isinstance(col, (list, tuple)):
+            if cols:
+                raise ValueError("col is a {0} but cols are not empty".format(type(col)))
+
+            col, cols = col[0], col[1:]  # type: ignore[assignment]
+
+        if not all(isinstance(c, str) for c in cols) or not (isinstance(col, str)):
+            raise TypeError("all names should be `str`")
+
+        self._write.num_buckets = numBuckets
+        self._write.bucket_cols = cast(List[str], cols)
+        return self
+
+    @overload
+    def sortBy(self, col: str, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def sortBy(self, col: TupleOrListOfString) -> "DataFrameWriter":
+        ...
+
+    def sortBy(
+        self, col: Union[str, TupleOrListOfString], *cols: Optional[str]
+    ) -> "DataFrameWriter":
+        """Sorts the output in each bucket by the given columns on the file system.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        col : str, tuple or list
+            a name of a column, or a list of names.
+        cols : str
+            additional names (optional). If `col` is a list it should be empty.
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a sorted-buckted manner, and read it back.
+
+        >>> from pyspark.sql.functions import input_file_name
+        >>> # Write a DataFrame into a Parquet file in a sorted-bucketed manner.
+        ... _ = spark.sql("DROP TABLE IF EXISTS sorted_bucketed_table")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.bucketBy(1, "name").sortBy("age").mode(
+        ...     "overwrite").saveAsTable("sorted_bucketed_table")
+        >>> # Read the Parquet file as a DataFrame.
+        ... spark.read.table("sorted_bucketed_table").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE sorted_bucketed_table")
+        """
+        if isinstance(col, (list, tuple)):
+            if cols:
+                raise ValueError("col is a {0} but cols are not empty".format(type(col)))
+
+            col, cols = col[0], col[1:]  # type: ignore[assignment]
+
+        if not all(isinstance(c, str) for c in cols) or not (isinstance(col, str)):
+            raise TypeError("all names should be `str`")
+
+        self._write.sort_cols = cast(List[str], cols)
+        return self
+
+    def save(
+        self,
+        path: Optional[str] = None,
+        format: Optional[str] = None,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """Saves the contents of the :class:`DataFrame` to a data source.
+
+        The data source is specified by the ``format`` and a set of ``options``.
+        If ``format`` is not specified, the default data source configured by
+        ``spark.sql.sources.default`` will be used.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str, optional
+            the path in a Hadoop supported file system
+        format : str, optional
+            the format used to save
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+        partitionBy : list, optional
+            names of partitioning columns
+        **options : dict
+            all other string options
+
+        Examples
+        --------
+        Write a DataFrame into a JSON file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a JSON file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("json").save(d)
+        ...
+        ...     # Read the JSON file as a DataFrame.
+        ...     spark.read.format('json').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode).options(**options)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        if format is not None:
+            self.format(format)
+        self._write.path = path
+        self._spark.execute_command(self._write.command(self._spark))
+
+    def insertInto(self, tableName: str, overwrite: Optional[bool] = None) -> None:
+        """Inserts the content of the :class:`DataFrame` to the specified table.
+
+        It requires that the schema of the :class:`DataFrame` is the same as the
+        schema of the table.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        overwrite : bool, optional
+            If true, overwrites existing data. Disabled by default
+
+        Notes
+        -----
+        Unlike :meth:`DataFrameWriter.saveAsTable`, :meth:`DataFrameWriter.insertInto` ignores
+        the column names and just uses position-based resolution.
+
+        Examples
+        --------
+        >>> _ = spark.sql("DROP TABLE IF EXISTS tblA")
+        >>> df = spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... )
+        >>> df.write.saveAsTable("tblA")
+
+        Insert the data into 'tblA' table but with different column names.
+
+        >>> df.selectExpr("age AS col1", "name AS col2").write.insertInto("tblA")
+        >>> spark.read.table("tblA").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE tblA")
+        """
+        if overwrite is not None:
+            self.mode("overwrite" if overwrite else "append")
+        self.saveAsTable(tableName)
+
+    def saveAsTable(
+        self,
+        name: str,
+        format: Optional[str] = None,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` as the specified table.
+
+        In the case the table already exists, behavior of this function depends on the
+        save mode, specified by the `mode` function (default to throwing an exception).
+        When `mode` is `Overwrite`, the schema of the :class:`DataFrame` does not need to be
+        the same as that of the existing table.
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Notes
+        -----
+        When `mode` is `Append`, if there is an existing table, we will use the format and
+        options of the existing table. The column order in the schema of the :class:`DataFrame`
+        doesn't need to be same as that of the existing table. Unlike
+        :meth:`DataFrameWriter.insertInto`, :meth:`DataFrameWriter.saveAsTable` will use the
+        column names to find the correct column positions.
+
+        Parameters
+        ----------
+        name : str
+            the table name
+        format : str, optional
+            the format used to save
+        mode : str, optional
+            one of `append`, `overwrite`, `error`, `errorifexists`, `ignore` \
+            (default: error)
+        partitionBy : str or list
+            names of partitioning columns
+        **options : dict
+            all other string options
+
+        Examples
+        --------
+        Creates a table from a DataFrame, and read it back.
+
+        >>> _ = spark.sql("DROP TABLE IF EXISTS tblA")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.saveAsTable("tblA")
+        >>> spark.read.table("tblA").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE tblA")
+        """
+        self.mode(mode).options(**options)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        if format is not None:
+            self.format(format)
+        self._write.table_name = name
+        self._spark.execute_command(self._write.command(self._spark))
+
+    def json(
+        self,
+        path: str,
+        mode: Optional[str] = None,
+        compression: Optional[str] = None,
+        dateFormat: Optional[str] = None,
+        timestampFormat: Optional[str] = None,
+        lineSep: Optional[str] = None,
+        encoding: Optional[str] = None,
+        ignoreNullFields: Optional[Union[bool, str]] = None,
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` in JSON format
+        (`JSON Lines text format or newline-delimited JSON <http://jsonlines.org/>`_) at the
+        specified path.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-json.html#data-source-option>`_
+            in the version you use.
+
+            .. # noqa
+
+        Examples
+        --------
+        Write a DataFrame into a JSON file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a JSON file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.json(d, mode="overwrite")
+        ...
+        ...     # Read the JSON file as a DataFrame.
+        ...     spark.read.format("json").load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode)
+        self._set_opts(
+            compression=compression,
+            dateFormat=dateFormat,
+            timestampFormat=timestampFormat,
+            lineSep=lineSep,
+            encoding=encoding,
+            ignoreNullFields=ignoreNullFields,
+        )
+        self.format("json").save(path)
+
+    def parquet(
+        self,
+        path: str,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        compression: Optional[str] = None,
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` in Parquet format at the specified path.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+        partitionBy : str or list, optional
+            names of partitioning columns
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#data-source-option>`_
+            in the version you use.
+
+            .. # noqa
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.parquet(d, mode="overwrite")
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.format("parquet").load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        self.option("compression", compression)
+        self.format("parquet").save(path)
+
+    def text(
+        self, path: str, compression: Optional[str] = None, lineSep: Optional[str] = None
+    ) -> None:
+        """Saves the content of the DataFrame in a text file at the specified path.
+        The text files will be encoded as UTF-8.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-text.html#data-source-option>`_
+            in the version you use.
+
+            .. # noqa
+
+        Notes
+        -----
+        The DataFrame must have only one column that is of string type.
+        Each row becomes a new line in the output file.
+
+        Examples
+        --------
+        Write a DataFrame into a text file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a text file
+        ...     df = spark.createDataFrame([("a",), ("b",), ("c",)], schema=["alphabets"])
+        ...     df.write.mode("overwrite").text(d)
+        ...
+        ...     # Read the text file as a DataFrame.
+        ...     spark.read.schema(df.schema).format("text").load(d).sort("alphabets").show()
+        +---------+
+        |alphabets|
+        +---------+
+        |        a|
+        |        b|
+        |        c|
+        +---------+
+        """
+        self._set_opts(compression=compression, lineSep=lineSep)
+        self.format("text").save(path)
+
+    def csv(
+        self,
+        path: str,
+        mode: Optional[str] = None,
+        compression: Optional[str] = None,
+        sep: Optional[str] = None,
+        quote: Optional[str] = None,
+        escape: Optional[str] = None,
+        header: Optional[Union[bool, str]] = None,
+        nullValue: Optional[str] = None,
+        escapeQuotes: Optional[Union[bool, str]] = None,
+        quoteAll: Optional[Union[bool, str]] = None,
+        dateFormat: Optional[str] = None,
+        timestampFormat: Optional[str] = None,
+        ignoreLeadingWhiteSpace: Optional[Union[bool, str]] = None,
+        ignoreTrailingWhiteSpace: Optional[Union[bool, str]] = None,
+        charToEscapeQuoteEscaping: Optional[str] = None,
+        encoding: Optional[str] = None,
+        emptyValue: Optional[str] = None,
+        lineSep: Optional[str] = None,
+    ) -> None:
+        r"""Saves the content of the :class:`DataFrame` in CSV format at the specified path.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str
+            the path in any Hadoop supported file system
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+
+        Other Parameters
+        ----------------
+        Extra options
+            For the extra options, refer to
+            `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-csv.html#data-source-option>`_
+            in the version you use.
+
+            .. # noqa
+
+        Examples
+        --------
+        Write a DataFrame into a CSV file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file
+        ...     df = spark.createDataFrame([{"age": 100, "name": "Hyukjin Kwon"}])

Review Comment:
   Actually, I think there is confusion with the example in line 214ff. Here the code is actually correct, the col value is "Hyukjin Kwon", and when reading it's set as a `nullValue` so it's read as `Null`, but in the above example, the `None` was never used.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] grundprinzip commented on a diff in pull request #38801: [SPARK-40539] [CONNECT] [PYTHON] Add basic support for DataFrameWriter

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #38801:
URL: https://github.com/apache/spark/pull/38801#discussion_r1032817074


##########
python/pyspark/sql/connect/writer.py:
##########
@@ -0,0 +1,922 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+
+from typing import Optional, Union, List, overload, Tuple, cast, Any
+from typing import TYPE_CHECKING
+
+from pyspark.sql.connect.plan import WriteOperation, LogicalPlan
+from pyspark.sql.types import StructType
+from pyspark.sql.utils import to_str
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect._typing import OptionalPrimitiveType
+    from pyspark.sql.connect.client import RemoteSparkSession
+
+
+PathOrPaths = Union[str, List[str]]
+TupleOrListOfString = Union[List[str], Tuple[str, ...]]
+
+
+class OptionUtils:
+    def _set_opts(
+        self,
+        schema: Optional[Union[StructType, str]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """
+        Set named options (filter out those the value is None)
+        """
+        if schema is not None:
+            self.schema(schema)  # type: ignore[attr-defined]
+        for k, v in options.items():
+            if v is not None:
+                self.option(k, v)  # type: ignore[attr-defined]
+
+
+class DataFrameWriter(OptionUtils):
+    """
+    Interface used to write a :class:`DataFrame` to external storage systems
+    (e.g. file systems, key-value stores, etc). Use :attr:`DataFrame.write`
+    to access this.
+
+    .. versionadded:: 3.4.0
+    """
+
+    def __init__(self, plan: "LogicalPlan", session: "RemoteSparkSession"):
+        self._df: "LogicalPlan" = plan
+        self._spark: "RemoteSparkSession" = session
+        self._write: "WriteOperation" = WriteOperation(self._df)
+
+    def mode(self, saveMode: Optional[str]) -> "DataFrameWriter":
+        """Specifies the behavior when data or table already exists.
+
+        Options include:
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Examples
+        --------
+        Raise an error when writing to an existing path.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     spark.createDataFrame(
+        ...         [{"age": 80, "name": "Xinrong Meng"}]
+        ...     ).write.mode("error").format("parquet").save(d)
+        Traceback (most recent call last):
+            ...
+        pyspark.sql.utils.AnalysisException: ...
+
+        Write a Parquet file back with various options, and read it back.
+
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Overwrite the path with a new Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 120, "name": "Takuya Ueshin"}]
+        ...     ).write.mode("append").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 140, "name": "Haejoon Lee"}]
+        ...     ).write.mode("ignore").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |120|Takuya Ueshin|
+        |100| Hyukjin Kwon|
+        +---+-------------+
+        """
+        # At the JVM side, the default value of mode is already set to "error".
+        # So, if the given saveMode is None, we will not call JVM-side's mode method.
+        if saveMode is not None:
+            self._write.mode = saveMode
+        return self
+
+    def format(self, source: str) -> "DataFrameWriter":
+        """Specifies the underlying output data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        source : str
+            string, name of the data source, e.g. 'json', 'parquet'.
+
+        Examples
+        --------
+        >>> spark.range(1).write.format('parquet')
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Write a DataFrame into a Parquet file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.format('parquet').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.source = source
+        return self
+
+    def option(self, key: str, value: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds a output option for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        key : str
+            The key for the option to set.
+        value
+            The value for the option to set.
+
+        Examples
+        --------
+        >>> spark.range(1).write.option("key", "value")
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Specify the option 'nullValue' with writing a CSV file.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file with 'nullValue' option set to 'Hyukjin Kwon'.
+        ...     df = spark.createDataFrame([(100, None)], "age INT, name STRING")
+        ...     df.write.option("nullValue", "Hyukjin Kwon").mode("overwrite").format("csv").save(d)
+        ...
+        ...     # Read the CSV file as a DataFrame.
+        ...     spark.read.schema(df.schema).format('csv').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.options[key] = to_str(value)
+        return self
+
+    def options(self, **options: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds output options for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        **options : dict
+            The dictionary of string keys and prmitive-type values.
+
+        Examples
+        --------
+        >>> spark.range(1).write.option("key", "value")
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Specify the option 'nullValue' and 'header' with writing a CSV file.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file with 'nullValue' option set to 'Hyukjin Kwon',
+        ...     # and 'header' option set to `True`.
+        ...     df = spark.createDataFrame([(100, "Hyukjin Kwon")], ["age", "name"])

Review Comment:
   With just the `None` there is an error about the schema not being inferrable, I fixed the example.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] bjornjorgensen commented on a diff in pull request #38801: [SPARK-40539] [CONNECT] [PYTHON] Add basic support for DataFrameWriter

Posted by GitBox <gi...@apache.org>.
bjornjorgensen commented on code in PR #38801:
URL: https://github.com/apache/spark/pull/38801#discussion_r1032662683


##########
python/pyspark/sql/connect/writer.py:
##########
@@ -0,0 +1,922 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+
+from typing import Optional, Union, List, overload, Tuple, cast, Any
+from typing import TYPE_CHECKING
+
+from pyspark.sql.connect.plan import WriteOperation, LogicalPlan
+from pyspark.sql.types import StructType
+from pyspark.sql.utils import to_str
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect._typing import OptionalPrimitiveType
+    from pyspark.sql.connect.client import RemoteSparkSession
+
+
+PathOrPaths = Union[str, List[str]]
+TupleOrListOfString = Union[List[str], Tuple[str, ...]]
+
+
+class OptionUtils:
+    def _set_opts(
+        self,
+        schema: Optional[Union[StructType, str]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """
+        Set named options (filter out those the value is None)
+        """
+        if schema is not None:
+            self.schema(schema)  # type: ignore[attr-defined]
+        for k, v in options.items():
+            if v is not None:
+                self.option(k, v)  # type: ignore[attr-defined]
+
+
+class DataFrameWriter(OptionUtils):
+    """
+    Interface used to write a :class:`DataFrame` to external storage systems
+    (e.g. file systems, key-value stores, etc). Use :attr:`DataFrame.write`
+    to access this.
+
+    .. versionadded:: 3.4.0
+    """
+
+    def __init__(self, plan: "LogicalPlan", session: "RemoteSparkSession"):
+        self._df: "LogicalPlan" = plan
+        self._spark: "RemoteSparkSession" = session
+        self._write: "WriteOperation" = WriteOperation(self._df)
+
+    def mode(self, saveMode: Optional[str]) -> "DataFrameWriter":
+        """Specifies the behavior when data or table already exists.
+
+        Options include:
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Examples
+        --------
+        Raise an error when writing to an existing path.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     spark.createDataFrame(
+        ...         [{"age": 80, "name": "Xinrong Meng"}]
+        ...     ).write.mode("error").format("parquet").save(d)
+        Traceback (most recent call last):
+            ...
+        pyspark.sql.utils.AnalysisException: ...
+
+        Write a Parquet file back with various options, and read it back.
+
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Overwrite the path with a new Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 120, "name": "Takuya Ueshin"}]
+        ...     ).write.mode("append").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 140, "name": "Haejoon Lee"}]
+        ...     ).write.mode("ignore").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |120|Takuya Ueshin|
+        |100| Hyukjin Kwon|
+        +---+-------------+
+        """
+        # At the JVM side, the default value of mode is already set to "error".
+        # So, if the given saveMode is None, we will not call JVM-side's mode method.
+        if saveMode is not None:
+            self._write.mode = saveMode
+        return self
+
+    def format(self, source: str) -> "DataFrameWriter":
+        """Specifies the underlying output data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        source : str
+            string, name of the data source, e.g. 'json', 'parquet'.
+
+        Examples
+        --------
+        >>> spark.range(1).write.format('parquet')
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Write a DataFrame into a Parquet file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.format('parquet').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.source = source
+        return self
+
+    def option(self, key: str, value: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds a output option for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        key : str
+            The key for the option to set.
+        value
+            The value for the option to set.
+
+        Examples
+        --------
+        >>> spark.range(1).write.option("key", "value")
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Specify the option 'nullValue' with writing a CSV file.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file with 'nullValue' option set to 'Hyukjin Kwon'.
+        ...     df = spark.createDataFrame([(100, None)], "age INT, name STRING")
+        ...     df.write.option("nullValue", "Hyukjin Kwon").mode("overwrite").format("csv").save(d)
+        ...
+        ...     # Read the CSV file as a DataFrame.
+        ...     spark.read.schema(df.schema).format('csv').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.options[key] = to_str(value)
+        return self
+
+    def options(self, **options: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds output options for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        **options : dict
+            The dictionary of string keys and prmitive-type values.

Review Comment:
   The dictionary of string keys and primitive-type values.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] grundprinzip commented on a diff in pull request #38801: [SPARK-40539] [CONNECT] [PYTHON] Add basic support for DataFrameWriter

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #38801:
URL: https://github.com/apache/spark/pull/38801#discussion_r1032817007


##########
python/pyspark/sql/connect/writer.py:
##########
@@ -0,0 +1,922 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+
+from typing import Optional, Union, List, overload, Tuple, cast, Any
+from typing import TYPE_CHECKING
+
+from pyspark.sql.connect.plan import WriteOperation, LogicalPlan
+from pyspark.sql.types import StructType
+from pyspark.sql.utils import to_str
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect._typing import OptionalPrimitiveType
+    from pyspark.sql.connect.client import RemoteSparkSession
+
+
+PathOrPaths = Union[str, List[str]]
+TupleOrListOfString = Union[List[str], Tuple[str, ...]]
+
+
+class OptionUtils:
+    def _set_opts(
+        self,
+        schema: Optional[Union[StructType, str]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """
+        Set named options (filter out those the value is None)
+        """
+        if schema is not None:
+            self.schema(schema)  # type: ignore[attr-defined]
+        for k, v in options.items():
+            if v is not None:
+                self.option(k, v)  # type: ignore[attr-defined]
+
+
+class DataFrameWriter(OptionUtils):
+    """
+    Interface used to write a :class:`DataFrame` to external storage systems
+    (e.g. file systems, key-value stores, etc). Use :attr:`DataFrame.write`
+    to access this.
+
+    .. versionadded:: 3.4.0
+    """
+
+    def __init__(self, plan: "LogicalPlan", session: "RemoteSparkSession"):
+        self._df: "LogicalPlan" = plan
+        self._spark: "RemoteSparkSession" = session
+        self._write: "WriteOperation" = WriteOperation(self._df)
+
+    def mode(self, saveMode: Optional[str]) -> "DataFrameWriter":
+        """Specifies the behavior when data or table already exists.
+
+        Options include:
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Examples
+        --------
+        Raise an error when writing to an existing path.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     spark.createDataFrame(
+        ...         [{"age": 80, "name": "Xinrong Meng"}]
+        ...     ).write.mode("error").format("parquet").save(d)
+        Traceback (most recent call last):
+            ...
+        pyspark.sql.utils.AnalysisException: ...
+
+        Write a Parquet file back with various options, and read it back.
+
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Overwrite the path with a new Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 120, "name": "Takuya Ueshin"}]
+        ...     ).write.mode("append").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 140, "name": "Haejoon Lee"}]
+        ...     ).write.mode("ignore").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |120|Takuya Ueshin|
+        |100| Hyukjin Kwon|
+        +---+-------------+
+        """
+        # At the JVM side, the default value of mode is already set to "error".
+        # So, if the given saveMode is None, we will not call JVM-side's mode method.
+        if saveMode is not None:
+            self._write.mode = saveMode
+        return self
+
+    def format(self, source: str) -> "DataFrameWriter":
+        """Specifies the underlying output data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        source : str
+            string, name of the data source, e.g. 'json', 'parquet'.
+
+        Examples
+        --------
+        >>> spark.range(1).write.format('parquet')
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Write a DataFrame into a Parquet file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.format('parquet').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.source = source
+        return self
+
+    def option(self, key: str, value: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds a output option for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        key : str
+            The key for the option to set.
+        value
+            The value for the option to set.
+
+        Examples
+        --------
+        >>> spark.range(1).write.option("key", "value")
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Specify the option 'nullValue' with writing a CSV file.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file with 'nullValue' option set to 'Hyukjin Kwon'.
+        ...     df = spark.createDataFrame([(100, None)], "age INT, name STRING")
+        ...     df.write.option("nullValue", "Hyukjin Kwon").mode("overwrite").format("csv").save(d)
+        ...
+        ...     # Read the CSV file as a DataFrame.
+        ...     spark.read.schema(df.schema).format('csv').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.options[key] = to_str(value)
+        return self
+
+    def options(self, **options: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds output options for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        **options : dict
+            The dictionary of string keys and prmitive-type values.
+
+        Examples
+        --------
+        >>> spark.range(1).write.option("key", "value")
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Specify the option 'nullValue' and 'header' with writing a CSV file.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file with 'nullValue' option set to 'Hyukjin Kwon',
+        ...     # and 'header' option set to `True`.
+        ...     df = spark.createDataFrame([(100, "Hyukjin Kwon")], ["age", "name"])
+        ...     df.write.options(nullValue="Hyukjin Kwon", header=True).mode(
+        ...         "overwrite").format("csv").save(d)
+        ...
+        ...     # Read the CSV file as a DataFrame.
+        ...     spark.read.option("header", True).format('csv').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        for k in options:
+            self._write.options[k] = to_str(options[k])
+        return self
+
+    @overload
+    def partitionBy(self, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def partitionBy(self, *cols: List[str]) -> "DataFrameWriter":
+        ...
+
+    def partitionBy(self, *cols: Union[str, List[str]]) -> "DataFrameWriter":
+        """Partitions the output by the given columns on the file system.
+
+        If specified, the output is laid out on the file system similar
+        to Hive's partitioning scheme.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        cols : str or list
+            name of columns
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a partitioned manner, and read it back.
+
+        >>> import tempfile
+        >>> import os
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file in a partitioned manner.
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}, {"age": 120, "name": "Ruifeng Zheng"}]
+        ...     ).write.partitionBy("name").mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).sort("age").show()
+        ...
+        ...     # Read one partition as a DataFrame.
+        ...     spark.read.parquet(f"{d}{os.path.sep}name=Hyukjin Kwon").show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |100| Hyukjin Kwon|
+        |120|Ruifeng Zheng|
+        +---+-------------+
+        +---+
+        |age|
+        +---+
+        |100|
+        +---+
+        """
+        if len(cols) == 1 and isinstance(cols[0], (list, tuple)):
+            cols = cols[0]  # type: ignore[assignment]
+
+        self._write.partitioning_cols = cast(List[str], cols)
+        return self
+
+    @overload
+    def bucketBy(self, numBuckets: int, col: str, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def bucketBy(self, numBuckets: int, col: TupleOrListOfString) -> "DataFrameWriter":
+        ...
+
+    def bucketBy(
+        self, numBuckets: int, col: Union[str, TupleOrListOfString], *cols: Optional[str]
+    ) -> "DataFrameWriter":
+        """Buckets the output by the given columns. If specified,
+        the output is laid out on the file system similar to Hive's bucketing scheme,
+        but with a different bucket hash function and is not compatible with Hive's bucketing.
+
+        .. versionadded:: 2.3.0
+
+        Parameters
+        ----------
+        numBuckets : int
+            the number of buckets to save
+        col : str, list or tuple
+            a name of a column, or a list of names.
+        cols : str
+            additional names (optional). If `col` is a list it should be empty.
+
+        Notes
+        -----
+        Applicable for file-based data sources in combination with
+        :py:meth:`DataFrameWriter.saveAsTable`.
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a buckted manner, and read it back.
+
+        >>> from pyspark.sql.functions import input_file_name
+        >>> # Write a DataFrame into a Parquet file in a bucketed manner.
+        ... _ = spark.sql("DROP TABLE IF EXISTS bucketed_table")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.bucketBy(2, "name").mode("overwrite").saveAsTable("bucketed_table")
+        >>> # Read the Parquet file as a DataFrame.
+        ... spark.read.table("bucketed_table").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE bucketed_table")
+        """
+        if not isinstance(numBuckets, int):
+            raise TypeError("numBuckets should be an int, got {0}.".format(type(numBuckets)))
+
+        if isinstance(col, (list, tuple)):
+            if cols:
+                raise ValueError("col is a {0} but cols are not empty".format(type(col)))
+
+            col, cols = col[0], col[1:]  # type: ignore[assignment]
+
+        if not all(isinstance(c, str) for c in cols) or not (isinstance(col, str)):
+            raise TypeError("all names should be `str`")
+
+        self._write.num_buckets = numBuckets
+        self._write.bucket_cols = cast(List[str], cols)
+        return self
+
+    @overload
+    def sortBy(self, col: str, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def sortBy(self, col: TupleOrListOfString) -> "DataFrameWriter":
+        ...
+
+    def sortBy(
+        self, col: Union[str, TupleOrListOfString], *cols: Optional[str]
+    ) -> "DataFrameWriter":
+        """Sorts the output in each bucket by the given columns on the file system.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        col : str, tuple or list
+            a name of a column, or a list of names.
+        cols : str
+            additional names (optional). If `col` is a list it should be empty.
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a sorted-buckted manner, and read it back.
+
+        >>> from pyspark.sql.functions import input_file_name
+        >>> # Write a DataFrame into a Parquet file in a sorted-bucketed manner.
+        ... _ = spark.sql("DROP TABLE IF EXISTS sorted_bucketed_table")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.bucketBy(1, "name").sortBy("age").mode(
+        ...     "overwrite").saveAsTable("sorted_bucketed_table")
+        >>> # Read the Parquet file as a DataFrame.
+        ... spark.read.table("sorted_bucketed_table").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE sorted_bucketed_table")
+        """
+        if isinstance(col, (list, tuple)):
+            if cols:
+                raise ValueError("col is a {0} but cols are not empty".format(type(col)))
+
+            col, cols = col[0], col[1:]  # type: ignore[assignment]
+
+        if not all(isinstance(c, str) for c in cols) or not (isinstance(col, str)):
+            raise TypeError("all names should be `str`")
+
+        self._write.sort_cols = cast(List[str], cols)
+        return self
+
+    def save(
+        self,
+        path: Optional[str] = None,
+        format: Optional[str] = None,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """Saves the contents of the :class:`DataFrame` to a data source.
+
+        The data source is specified by the ``format`` and a set of ``options``.
+        If ``format`` is not specified, the default data source configured by
+        ``spark.sql.sources.default`` will be used.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str, optional
+            the path in a Hadoop supported file system
+        format : str, optional
+            the format used to save
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+        partitionBy : list, optional
+            names of partitioning columns
+        **options : dict
+            all other string options
+
+        Examples
+        --------
+        Write a DataFrame into a JSON file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a JSON file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("json").save(d)
+        ...
+        ...     # Read the JSON file as a DataFrame.
+        ...     spark.read.format('json').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode).options(**options)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        if format is not None:
+            self.format(format)
+        self._write.path = path
+        self._spark.execute_command(self._write.command(self._spark))
+
+    def insertInto(self, tableName: str, overwrite: Optional[bool] = None) -> None:
+        """Inserts the content of the :class:`DataFrame` to the specified table.
+
+        It requires that the schema of the :class:`DataFrame` is the same as the
+        schema of the table.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        overwrite : bool, optional
+            If true, overwrites existing data. Disabled by default
+
+        Notes
+        -----
+        Unlike :meth:`DataFrameWriter.saveAsTable`, :meth:`DataFrameWriter.insertInto` ignores
+        the column names and just uses position-based resolution.
+
+        Examples
+        --------
+        >>> _ = spark.sql("DROP TABLE IF EXISTS tblA")
+        >>> df = spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... )
+        >>> df.write.saveAsTable("tblA")
+
+        Insert the data into 'tblA' table but with different column names.
+
+        >>> df.selectExpr("age AS col1", "name AS col2").write.insertInto("tblA")
+        >>> spark.read.table("tblA").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE tblA")
+        """
+        if overwrite is not None:
+            self.mode("overwrite" if overwrite else "append")
+        self.saveAsTable(tableName)
+
+    def saveAsTable(
+        self,
+        name: str,
+        format: Optional[str] = None,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` as the specified table.
+
+        In the case the table already exists, behavior of this function depends on the
+        save mode, specified by the `mode` function (default to throwing an exception).
+        When `mode` is `Overwrite`, the schema of the :class:`DataFrame` does not need to be
+        the same as that of the existing table.
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Notes
+        -----
+        When `mode` is `Append`, if there is an existing table, we will use the format and
+        options of the existing table. The column order in the schema of the :class:`DataFrame`
+        doesn't need to be same as that of the existing table. Unlike

Review Comment:
   Done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] grundprinzip commented on a diff in pull request #38801: [SPARK-40539] [CONNECT] [PYTHON] Add basic support for DataFrameWriter

Posted by GitBox <gi...@apache.org>.
grundprinzip commented on code in PR #38801:
URL: https://github.com/apache/spark/pull/38801#discussion_r1032816983


##########
python/pyspark/sql/connect/writer.py:
##########
@@ -0,0 +1,922 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+
+from typing import Optional, Union, List, overload, Tuple, cast, Any
+from typing import TYPE_CHECKING
+
+from pyspark.sql.connect.plan import WriteOperation, LogicalPlan
+from pyspark.sql.types import StructType
+from pyspark.sql.utils import to_str
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect._typing import OptionalPrimitiveType
+    from pyspark.sql.connect.client import RemoteSparkSession
+
+
+PathOrPaths = Union[str, List[str]]
+TupleOrListOfString = Union[List[str], Tuple[str, ...]]
+
+
+class OptionUtils:
+    def _set_opts(
+        self,
+        schema: Optional[Union[StructType, str]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """
+        Set named options (filter out those the value is None)
+        """
+        if schema is not None:
+            self.schema(schema)  # type: ignore[attr-defined]
+        for k, v in options.items():
+            if v is not None:
+                self.option(k, v)  # type: ignore[attr-defined]
+
+
+class DataFrameWriter(OptionUtils):
+    """
+    Interface used to write a :class:`DataFrame` to external storage systems
+    (e.g. file systems, key-value stores, etc). Use :attr:`DataFrame.write`
+    to access this.
+
+    .. versionadded:: 3.4.0
+    """
+
+    def __init__(self, plan: "LogicalPlan", session: "RemoteSparkSession"):
+        self._df: "LogicalPlan" = plan
+        self._spark: "RemoteSparkSession" = session
+        self._write: "WriteOperation" = WriteOperation(self._df)
+
+    def mode(self, saveMode: Optional[str]) -> "DataFrameWriter":
+        """Specifies the behavior when data or table already exists.
+
+        Options include:
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Examples
+        --------
+        Raise an error when writing to an existing path.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     spark.createDataFrame(
+        ...         [{"age": 80, "name": "Xinrong Meng"}]
+        ...     ).write.mode("error").format("parquet").save(d)
+        Traceback (most recent call last):
+            ...
+        pyspark.sql.utils.AnalysisException: ...
+
+        Write a Parquet file back with various options, and read it back.
+
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Overwrite the path with a new Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 120, "name": "Takuya Ueshin"}]
+        ...     ).write.mode("append").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 140, "name": "Haejoon Lee"}]
+        ...     ).write.mode("ignore").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |120|Takuya Ueshin|
+        |100| Hyukjin Kwon|
+        +---+-------------+
+        """
+        # At the JVM side, the default value of mode is already set to "error".
+        # So, if the given saveMode is None, we will not call JVM-side's mode method.
+        if saveMode is not None:
+            self._write.mode = saveMode
+        return self
+
+    def format(self, source: str) -> "DataFrameWriter":
+        """Specifies the underlying output data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        source : str
+            string, name of the data source, e.g. 'json', 'parquet'.
+
+        Examples
+        --------
+        >>> spark.range(1).write.format('parquet')
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Write a DataFrame into a Parquet file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.format('parquet').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.source = source
+        return self
+
+    def option(self, key: str, value: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds a output option for the underlying data source.

Review Comment:
   Done



##########
python/pyspark/sql/connect/writer.py:
##########
@@ -0,0 +1,922 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+
+from typing import Optional, Union, List, overload, Tuple, cast, Any
+from typing import TYPE_CHECKING
+
+from pyspark.sql.connect.plan import WriteOperation, LogicalPlan
+from pyspark.sql.types import StructType
+from pyspark.sql.utils import to_str
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect._typing import OptionalPrimitiveType
+    from pyspark.sql.connect.client import RemoteSparkSession
+
+
+PathOrPaths = Union[str, List[str]]
+TupleOrListOfString = Union[List[str], Tuple[str, ...]]
+
+
+class OptionUtils:
+    def _set_opts(
+        self,
+        schema: Optional[Union[StructType, str]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """
+        Set named options (filter out those the value is None)
+        """
+        if schema is not None:
+            self.schema(schema)  # type: ignore[attr-defined]
+        for k, v in options.items():
+            if v is not None:
+                self.option(k, v)  # type: ignore[attr-defined]
+
+
+class DataFrameWriter(OptionUtils):
+    """
+    Interface used to write a :class:`DataFrame` to external storage systems
+    (e.g. file systems, key-value stores, etc). Use :attr:`DataFrame.write`
+    to access this.
+
+    .. versionadded:: 3.4.0
+    """
+
+    def __init__(self, plan: "LogicalPlan", session: "RemoteSparkSession"):
+        self._df: "LogicalPlan" = plan
+        self._spark: "RemoteSparkSession" = session
+        self._write: "WriteOperation" = WriteOperation(self._df)
+
+    def mode(self, saveMode: Optional[str]) -> "DataFrameWriter":
+        """Specifies the behavior when data or table already exists.
+
+        Options include:
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Examples
+        --------
+        Raise an error when writing to an existing path.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     spark.createDataFrame(
+        ...         [{"age": 80, "name": "Xinrong Meng"}]
+        ...     ).write.mode("error").format("parquet").save(d)
+        Traceback (most recent call last):
+            ...
+        pyspark.sql.utils.AnalysisException: ...
+
+        Write a Parquet file back with various options, and read it back.
+
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Overwrite the path with a new Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 120, "name": "Takuya Ueshin"}]
+        ...     ).write.mode("append").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 140, "name": "Haejoon Lee"}]
+        ...     ).write.mode("ignore").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |120|Takuya Ueshin|
+        |100| Hyukjin Kwon|
+        +---+-------------+
+        """
+        # At the JVM side, the default value of mode is already set to "error".
+        # So, if the given saveMode is None, we will not call JVM-side's mode method.
+        if saveMode is not None:
+            self._write.mode = saveMode
+        return self
+
+    def format(self, source: str) -> "DataFrameWriter":
+        """Specifies the underlying output data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        source : str
+            string, name of the data source, e.g. 'json', 'parquet'.
+
+        Examples
+        --------
+        >>> spark.range(1).write.format('parquet')
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Write a DataFrame into a Parquet file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.format('parquet').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.source = source
+        return self
+
+    def option(self, key: str, value: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds a output option for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        key : str
+            The key for the option to set.
+        value
+            The value for the option to set.
+
+        Examples
+        --------
+        >>> spark.range(1).write.option("key", "value")
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Specify the option 'nullValue' with writing a CSV file.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file with 'nullValue' option set to 'Hyukjin Kwon'.
+        ...     df = spark.createDataFrame([(100, None)], "age INT, name STRING")
+        ...     df.write.option("nullValue", "Hyukjin Kwon").mode("overwrite").format("csv").save(d)
+        ...
+        ...     # Read the CSV file as a DataFrame.
+        ...     spark.read.schema(df.schema).format('csv').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.options[key] = to_str(value)
+        return self
+
+    def options(self, **options: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds output options for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        **options : dict
+            The dictionary of string keys and prmitive-type values.

Review Comment:
   Done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] bjornjorgensen commented on a diff in pull request #38801: [SPARK-40539] [CONNECT] [PYTHON] Add basic support for DataFrameWriter

Posted by GitBox <gi...@apache.org>.
bjornjorgensen commented on code in PR #38801:
URL: https://github.com/apache/spark/pull/38801#discussion_r1032666193


##########
python/pyspark/sql/connect/writer.py:
##########
@@ -0,0 +1,922 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+
+from typing import Optional, Union, List, overload, Tuple, cast, Any
+from typing import TYPE_CHECKING
+
+from pyspark.sql.connect.plan import WriteOperation, LogicalPlan
+from pyspark.sql.types import StructType
+from pyspark.sql.utils import to_str
+
+if TYPE_CHECKING:
+    from pyspark.sql.connect._typing import OptionalPrimitiveType
+    from pyspark.sql.connect.client import RemoteSparkSession
+
+
+PathOrPaths = Union[str, List[str]]
+TupleOrListOfString = Union[List[str], Tuple[str, ...]]
+
+
+class OptionUtils:
+    def _set_opts(
+        self,
+        schema: Optional[Union[StructType, str]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """
+        Set named options (filter out those the value is None)
+        """
+        if schema is not None:
+            self.schema(schema)  # type: ignore[attr-defined]
+        for k, v in options.items():
+            if v is not None:
+                self.option(k, v)  # type: ignore[attr-defined]
+
+
+class DataFrameWriter(OptionUtils):
+    """
+    Interface used to write a :class:`DataFrame` to external storage systems
+    (e.g. file systems, key-value stores, etc). Use :attr:`DataFrame.write`
+    to access this.
+
+    .. versionadded:: 3.4.0
+    """
+
+    def __init__(self, plan: "LogicalPlan", session: "RemoteSparkSession"):
+        self._df: "LogicalPlan" = plan
+        self._spark: "RemoteSparkSession" = session
+        self._write: "WriteOperation" = WriteOperation(self._df)
+
+    def mode(self, saveMode: Optional[str]) -> "DataFrameWriter":
+        """Specifies the behavior when data or table already exists.
+
+        Options include:
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Examples
+        --------
+        Raise an error when writing to an existing path.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     spark.createDataFrame(
+        ...         [{"age": 80, "name": "Xinrong Meng"}]
+        ...     ).write.mode("error").format("parquet").save(d)
+        Traceback (most recent call last):
+            ...
+        pyspark.sql.utils.AnalysisException: ...
+
+        Write a Parquet file back with various options, and read it back.
+
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Overwrite the path with a new Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 120, "name": "Takuya Ueshin"}]
+        ...     ).write.mode("append").format("parquet").save(d)
+        ...
+        ...     # Append another DataFrame into the Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 140, "name": "Haejoon Lee"}]
+        ...     ).write.mode("ignore").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |120|Takuya Ueshin|
+        |100| Hyukjin Kwon|
+        +---+-------------+
+        """
+        # At the JVM side, the default value of mode is already set to "error".
+        # So, if the given saveMode is None, we will not call JVM-side's mode method.
+        if saveMode is not None:
+            self._write.mode = saveMode
+        return self
+
+    def format(self, source: str) -> "DataFrameWriter":
+        """Specifies the underlying output data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        source : str
+            string, name of the data source, e.g. 'json', 'parquet'.
+
+        Examples
+        --------
+        >>> spark.range(1).write.format('parquet')
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Write a DataFrame into a Parquet file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.format('parquet').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.source = source
+        return self
+
+    def option(self, key: str, value: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds a output option for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        key : str
+            The key for the option to set.
+        value
+            The value for the option to set.
+
+        Examples
+        --------
+        >>> spark.range(1).write.option("key", "value")
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Specify the option 'nullValue' with writing a CSV file.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file with 'nullValue' option set to 'Hyukjin Kwon'.
+        ...     df = spark.createDataFrame([(100, None)], "age INT, name STRING")
+        ...     df.write.option("nullValue", "Hyukjin Kwon").mode("overwrite").format("csv").save(d)
+        ...
+        ...     # Read the CSV file as a DataFrame.
+        ...     spark.read.schema(df.schema).format('csv').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self._write.options[key] = to_str(value)
+        return self
+
+    def options(self, **options: "OptionalPrimitiveType") -> "DataFrameWriter":
+        """
+        Adds output options for the underlying data source.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        **options : dict
+            The dictionary of string keys and prmitive-type values.
+
+        Examples
+        --------
+        >>> spark.range(1).write.option("key", "value")
+        <pyspark.sql.readwriter.DataFrameWriter object ...>
+
+        Specify the option 'nullValue' and 'header' with writing a CSV file.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a CSV file with 'nullValue' option set to 'Hyukjin Kwon',
+        ...     # and 'header' option set to `True`.
+        ...     df = spark.createDataFrame([(100, "Hyukjin Kwon")], ["age", "name"])
+        ...     df.write.options(nullValue="Hyukjin Kwon", header=True).mode(
+        ...         "overwrite").format("csv").save(d)
+        ...
+        ...     # Read the CSV file as a DataFrame.
+        ...     spark.read.option("header", True).format('csv').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        for k in options:
+            self._write.options[k] = to_str(options[k])
+        return self
+
+    @overload
+    def partitionBy(self, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def partitionBy(self, *cols: List[str]) -> "DataFrameWriter":
+        ...
+
+    def partitionBy(self, *cols: Union[str, List[str]]) -> "DataFrameWriter":
+        """Partitions the output by the given columns on the file system.
+
+        If specified, the output is laid out on the file system similar
+        to Hive's partitioning scheme.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        cols : str or list
+            name of columns
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a partitioned manner, and read it back.
+
+        >>> import tempfile
+        >>> import os
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a Parquet file in a partitioned manner.
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}, {"age": 120, "name": "Ruifeng Zheng"}]
+        ...     ).write.partitionBy("name").mode("overwrite").format("parquet").save(d)
+        ...
+        ...     # Read the Parquet file as a DataFrame.
+        ...     spark.read.parquet(d).sort("age").show()
+        ...
+        ...     # Read one partition as a DataFrame.
+        ...     spark.read.parquet(f"{d}{os.path.sep}name=Hyukjin Kwon").show()
+        +---+-------------+
+        |age|         name|
+        +---+-------------+
+        |100| Hyukjin Kwon|
+        |120|Ruifeng Zheng|
+        +---+-------------+
+        +---+
+        |age|
+        +---+
+        |100|
+        +---+
+        """
+        if len(cols) == 1 and isinstance(cols[0], (list, tuple)):
+            cols = cols[0]  # type: ignore[assignment]
+
+        self._write.partitioning_cols = cast(List[str], cols)
+        return self
+
+    @overload
+    def bucketBy(self, numBuckets: int, col: str, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def bucketBy(self, numBuckets: int, col: TupleOrListOfString) -> "DataFrameWriter":
+        ...
+
+    def bucketBy(
+        self, numBuckets: int, col: Union[str, TupleOrListOfString], *cols: Optional[str]
+    ) -> "DataFrameWriter":
+        """Buckets the output by the given columns. If specified,
+        the output is laid out on the file system similar to Hive's bucketing scheme,
+        but with a different bucket hash function and is not compatible with Hive's bucketing.
+
+        .. versionadded:: 2.3.0
+
+        Parameters
+        ----------
+        numBuckets : int
+            the number of buckets to save
+        col : str, list or tuple
+            a name of a column, or a list of names.
+        cols : str
+            additional names (optional). If `col` is a list it should be empty.
+
+        Notes
+        -----
+        Applicable for file-based data sources in combination with
+        :py:meth:`DataFrameWriter.saveAsTable`.
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a buckted manner, and read it back.
+
+        >>> from pyspark.sql.functions import input_file_name
+        >>> # Write a DataFrame into a Parquet file in a bucketed manner.
+        ... _ = spark.sql("DROP TABLE IF EXISTS bucketed_table")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.bucketBy(2, "name").mode("overwrite").saveAsTable("bucketed_table")
+        >>> # Read the Parquet file as a DataFrame.
+        ... spark.read.table("bucketed_table").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE bucketed_table")
+        """
+        if not isinstance(numBuckets, int):
+            raise TypeError("numBuckets should be an int, got {0}.".format(type(numBuckets)))
+
+        if isinstance(col, (list, tuple)):
+            if cols:
+                raise ValueError("col is a {0} but cols are not empty".format(type(col)))
+
+            col, cols = col[0], col[1:]  # type: ignore[assignment]
+
+        if not all(isinstance(c, str) for c in cols) or not (isinstance(col, str)):
+            raise TypeError("all names should be `str`")
+
+        self._write.num_buckets = numBuckets
+        self._write.bucket_cols = cast(List[str], cols)
+        return self
+
+    @overload
+    def sortBy(self, col: str, *cols: str) -> "DataFrameWriter":
+        ...
+
+    @overload
+    def sortBy(self, col: TupleOrListOfString) -> "DataFrameWriter":
+        ...
+
+    def sortBy(
+        self, col: Union[str, TupleOrListOfString], *cols: Optional[str]
+    ) -> "DataFrameWriter":
+        """Sorts the output in each bucket by the given columns on the file system.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        col : str, tuple or list
+            a name of a column, or a list of names.
+        cols : str
+            additional names (optional). If `col` is a list it should be empty.
+
+        Examples
+        --------
+        Write a DataFrame into a Parquet file in a sorted-buckted manner, and read it back.
+
+        >>> from pyspark.sql.functions import input_file_name
+        >>> # Write a DataFrame into a Parquet file in a sorted-bucketed manner.
+        ... _ = spark.sql("DROP TABLE IF EXISTS sorted_bucketed_table")
+        >>> spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... ).write.bucketBy(1, "name").sortBy("age").mode(
+        ...     "overwrite").saveAsTable("sorted_bucketed_table")
+        >>> # Read the Parquet file as a DataFrame.
+        ... spark.read.table("sorted_bucketed_table").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE sorted_bucketed_table")
+        """
+        if isinstance(col, (list, tuple)):
+            if cols:
+                raise ValueError("col is a {0} but cols are not empty".format(type(col)))
+
+            col, cols = col[0], col[1:]  # type: ignore[assignment]
+
+        if not all(isinstance(c, str) for c in cols) or not (isinstance(col, str)):
+            raise TypeError("all names should be `str`")
+
+        self._write.sort_cols = cast(List[str], cols)
+        return self
+
+    def save(
+        self,
+        path: Optional[str] = None,
+        format: Optional[str] = None,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """Saves the contents of the :class:`DataFrame` to a data source.
+
+        The data source is specified by the ``format`` and a set of ``options``.
+        If ``format`` is not specified, the default data source configured by
+        ``spark.sql.sources.default`` will be used.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        path : str, optional
+            the path in a Hadoop supported file system
+        format : str, optional
+            the format used to save
+        mode : str, optional
+            specifies the behavior of the save operation when data already exists.
+
+            * ``append``: Append contents of this :class:`DataFrame` to existing data.
+            * ``overwrite``: Overwrite existing data.
+            * ``ignore``: Silently ignore this operation if data already exists.
+            * ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
+                exists.
+        partitionBy : list, optional
+            names of partitioning columns
+        **options : dict
+            all other string options
+
+        Examples
+        --------
+        Write a DataFrame into a JSON file and read it back.
+
+        >>> import tempfile
+        >>> with tempfile.TemporaryDirectory() as d:
+        ...     # Write a DataFrame into a JSON file
+        ...     spark.createDataFrame(
+        ...         [{"age": 100, "name": "Hyukjin Kwon"}]
+        ...     ).write.mode("overwrite").format("json").save(d)
+        ...
+        ...     # Read the JSON file as a DataFrame.
+        ...     spark.read.format('json').load(d).show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        +---+------------+
+        """
+        self.mode(mode).options(**options)
+        if partitionBy is not None:
+            self.partitionBy(partitionBy)
+        if format is not None:
+            self.format(format)
+        self._write.path = path
+        self._spark.execute_command(self._write.command(self._spark))
+
+    def insertInto(self, tableName: str, overwrite: Optional[bool] = None) -> None:
+        """Inserts the content of the :class:`DataFrame` to the specified table.
+
+        It requires that the schema of the :class:`DataFrame` is the same as the
+        schema of the table.
+
+        .. versionadded:: 3.4.0
+
+        Parameters
+        ----------
+        overwrite : bool, optional
+            If true, overwrites existing data. Disabled by default
+
+        Notes
+        -----
+        Unlike :meth:`DataFrameWriter.saveAsTable`, :meth:`DataFrameWriter.insertInto` ignores
+        the column names and just uses position-based resolution.
+
+        Examples
+        --------
+        >>> _ = spark.sql("DROP TABLE IF EXISTS tblA")
+        >>> df = spark.createDataFrame([
+        ...     (100, "Hyukjin Kwon"), (120, "Hyukjin Kwon"), (140, "Haejoon Lee")],
+        ...     schema=["age", "name"]
+        ... )
+        >>> df.write.saveAsTable("tblA")
+
+        Insert the data into 'tblA' table but with different column names.
+
+        >>> df.selectExpr("age AS col1", "name AS col2").write.insertInto("tblA")
+        >>> spark.read.table("tblA").sort("age").show()
+        +---+------------+
+        |age|        name|
+        +---+------------+
+        |100|Hyukjin Kwon|
+        |100|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |120|Hyukjin Kwon|
+        |140| Haejoon Lee|
+        |140| Haejoon Lee|
+        +---+------------+
+        >>> _ = spark.sql("DROP TABLE tblA")
+        """
+        if overwrite is not None:
+            self.mode("overwrite" if overwrite else "append")
+        self.saveAsTable(tableName)
+
+    def saveAsTable(
+        self,
+        name: str,
+        format: Optional[str] = None,
+        mode: Optional[str] = None,
+        partitionBy: Optional[Union[str, List[str]]] = None,
+        **options: "OptionalPrimitiveType",
+    ) -> None:
+        """Saves the content of the :class:`DataFrame` as the specified table.
+
+        In the case the table already exists, behavior of this function depends on the
+        save mode, specified by the `mode` function (default to throwing an exception).
+        When `mode` is `Overwrite`, the schema of the :class:`DataFrame` does not need to be
+        the same as that of the existing table.
+
+        * `append`: Append contents of this :class:`DataFrame` to existing data.
+        * `overwrite`: Overwrite existing data.
+        * `error` or `errorifexists`: Throw an exception if data already exists.
+        * `ignore`: Silently ignore this operation if data already exists.
+
+        .. versionadded:: 3.4.0
+
+        Notes
+        -----
+        When `mode` is `Append`, if there is an existing table, we will use the format and
+        options of the existing table. The column order in the schema of the :class:`DataFrame`
+        doesn't need to be same as that of the existing table. Unlike

Review Comment:
   doesn't need to be the same as 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhengruifeng closed pull request #38801: [SPARK-41317][CONNECT][PYTHON] Add basic support for DataFrameWriter

Posted by GitBox <gi...@apache.org>.
zhengruifeng closed pull request #38801: [SPARK-41317][CONNECT][PYTHON] Add basic support for DataFrameWriter
URL: https://github.com/apache/spark/pull/38801


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org