xinrong-databricks commented on a change in pull request #34677:
URL: https://github.com/apache/spark/pull/34677#discussion_r756328097
##
File path: python/pyspark/pandas/sql_formatter.py
##
@@ -0,0 +1,273 @@
+#
+# 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.
+#
+
+import os
+import string
+from typing import Any, Optional, Union, List, Sequence, Mapping, Tuple
+import uuid
+import warnings
+
+import pandas as pd
+
+from pyspark.pandas.internal import InternalFrame
+from pyspark.pandas.namespace import _get_index_map
+from pyspark.sql.functions import lit
+from pyspark import pandas as ps
+from pyspark.sql import SparkSession
+from pyspark.pandas.utils import default_session
+from pyspark.pandas.frame import DataFrame
+from pyspark.pandas.series import Series
+
+
+__all__ = ["sql"]
+
+
+# This is not used in this file. It's for legacy sql_processor.
+_CAPTURE_SCOPES = 3
+
+
+def sql(
+query: str,
+index_col: Optional[Union[str, List[str]]] = None,
+**kwargs: Any,
+) -> DataFrame:
+"""
+Execute a SQL query and return the result as a pandas-on-Spark DataFrame.
+
+This function acts as a standard Python string formatter with understanding
+the following variable types:
+
+* pandas-on-Spark DataFrame
+* pandas-on-Spark Series
+* pandas DataFrame
+* pandas Series
+* string
+
+Parameters
+--
+query : str
+the SQL query
+index_col : str or list of str, optional
+Column names to be used in Spark to represent pandas-on-Spark's index.
The index name
+in pandas-on-Spark is ignored. By default, the index is always lost.
+
+.. note:: If you want to preserve the index, explicitly use
:func:`DataFrame.reset_index`,
+and pass it to the sql statement with `index_col` parameter.
+
+For example,
+
+>>> psdf = ps.DataFrame({"A": [1, 2, 3], "B":[4, 5, 6]},
index=['a', 'b', 'c'])
+>>> new_psdf = psdf.reset_index()
+>>> ps.sql("SELECT * FROM {new_psdf}", index_col="index",
new_psdf=new_psdf)
+... # doctest: +NORMALIZE_WHITESPACE
+ A B
+index
+a 1 4
+b 2 5
+c 3 6
+
+For MultiIndex,
+
+>>> psdf = ps.DataFrame(
+... {"A": [1, 2, 3], "B": [4, 5, 6]},
+... index=pd.MultiIndex.from_tuples(
+... [("a", "b"), ("c", "d"), ("e", "f")], names=["index1",
"index2"]
+... ),
+... )
+>>> new_psdf = psdf.reset_index()
+>>> ps.sql("SELECT * FROM {new_psdf}", index_col=["index1",
"index2"], new_psdf=new_psdf)
+... # doctest: +NORMALIZE_WHITESPACE
+ A B
+index1 index2
+a b 1 4
+c d 2 5
+e f 3 6
+
+Also note that the index name(s) should be matched to the existing
name.
+kwargs
+other variables that the user want to set that can be referenced in
the query
+
+Returns
+---
+pandas-on-Spark DataFrame
+
+Examples
+
+
+Calling a built-in SQL function.
+
+>>> ps.sql("SELECT * FROM range(10) where id > 7")
+ id
+0 8
+1 9
+
+>>> ps.sql("SELECT * FROM range(10) WHERE id > {bound1} AND id <
{bound2}", bound1=7, bound2=9)
+ id
+0 8
+
+>>> mydf = ps.range(10)
+>>> x = tuple(range(4))
+>>> ps.sql("SELECT {ser} FROM {mydf} WHERE id IN {x}", ser=mydf.id,
mydf=mydf, x=x)
+ id
+0 0
+1 1
+2 2
+3 3
+
+Mixing pandas-on-Spark and pandas DataFrames in a join operation. Note
that the index is
+dropped.
+
+>>> ps.sql('''
+... SELECT m1.a, m2.b
+... FROM {table1} m1 INNER JOIN {table2} m2
+... ON m1.key = m2.key
+... ORDER BY m1.a, m2.b''',
+... table1=ps.DataFrame({"a": [1,2], "key": ["a", "b"]}),
+... table2=pd.DataFrame({"b": [3,4,5], "key": ["a", "b", "b"]}))
+ a b
+0 1 3
+1 2 4
+2 2 5
+
+Also, it is possible to query using Series.
+
+>>> psdf = ps.DataFrame({"A": [1,