1919import os
2020from collections import namedtuple
2121
22- from pyspark .sql import SparkSession
23-
2422ExpressionInfo = namedtuple ("ExpressionInfo" , "className usage name extended" )
2523
2624
27- def _list_function_infos (spark ):
25+ def _list_function_infos (jvm ):
2826 """
2927 Returns a list of function information via JVM. Sorts wrapped expression infos by name
3028 and returns them.
3129 """
3230
33- jinfos = spark .sparkContext ._jvm \
34- .org .apache .spark .sql .api .python .PythonSQLUtils .listBuiltinFunctionInfos ()
31+ jinfos = jvm .org .apache .spark .sql .api .python .PythonSQLUtils .listBuiltinFunctionInfos ()
3532 infos = []
3633 for jinfo in jinfos :
3734 name = jinfo .getName ()
@@ -69,14 +66,14 @@ def _make_pretty_extended(extended):
6966 return "```%s```\n \n " % extended
7067
7168
72- def generate_sql_markdown (spark , path ):
69+ def generate_sql_markdown (jvm , path ):
7370 """
7471 Generates a markdown file after listing the function information. The output file
7572 is created in `path`.
7673 """
7774
7875 with open (path , 'w' ) as mdfile :
79- for info in _list_function_infos (spark ):
76+ for info in _list_function_infos (jvm ):
8077 mdfile .write ("### %s\n \n " % info .name )
8178 usage = _make_pretty_usage (info .usage )
8279 extended = _make_pretty_extended (info .extended )
@@ -87,10 +84,8 @@ def generate_sql_markdown(spark, path):
8784
8885
8986if __name__ == "__main__" :
90- spark = SparkSession \
91- .builder \
92- .appName ("GenSQLDocs" ) \
93- .getOrCreate ()
87+ from pyspark .java_gateway import launch_gateway
88+
89+ jvm = launch_gateway ().jvm
9490 markdown_file_path = "%s/docs/index.md" % os .path .dirname (sys .argv [0 ])
95- generate_sql_markdown (spark , markdown_file_path )
96- spark .stop ()
91+ generate_sql_markdown (jvm , markdown_file_path )
0 commit comments