diff --git a/.gitignore b/.gitignore index cf9780db37ad7..903297db96901 100644 --- a/.gitignore +++ b/.gitignore @@ -47,6 +47,8 @@ dev/pr-deps/ dist/ docs/_site docs/api +sql/docs +sql/site lib_managed/ lint-r-report.log log/ diff --git a/docs/README.md b/docs/README.md index 90e10a104b517..0090dd071e15f 100644 --- a/docs/README.md +++ b/docs/README.md @@ -68,6 +68,6 @@ jekyll plugin to run `build/sbt unidoc` before building the site so if you haven may take some time as it generates all of the scaladoc. The jekyll plugin also generates the PySpark docs using [Sphinx](http://sphinx-doc.org/). -NOTE: To skip the step of building and copying over the Scala, Python, R API docs, run `SKIP_API=1 -jekyll`. In addition, `SKIP_SCALADOC=1`, `SKIP_PYTHONDOC=1`, and `SKIP_RDOC=1` can be used to skip a single -step of the corresponding language. +NOTE: To skip the step of building and copying over the Scala, Python, R and SQL API docs, run `SKIP_API=1 +jekyll`. In addition, `SKIP_SCALADOC=1`, `SKIP_PYTHONDOC=1`, `SKIP_RDOC=1` and `SKIP_SQLDOC=1` can be used +to skip a single step of the corresponding language. diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index 570483c0b04ea..67b05ecf7a858 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -86,6 +86,7 @@
  • Java
  • Python
  • R
  • +
  • SQL, Built-in Functions
  • diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb index 95e3ba35e9027..00366f803c2ad 100644 --- a/docs/_plugins/copy_api_dirs.rb +++ b/docs/_plugins/copy_api_dirs.rb @@ -150,4 +150,31 @@ cp("../R/pkg/DESCRIPTION", "api") end + if not (ENV['SKIP_SQLDOC'] == '1') + # Build SQL API docs + + puts "Moving to project root and building API docs." + curr_dir = pwd + cd("..") + + puts "Running 'build/sbt clean package' from " + pwd + "; this may take a few minutes..." + system("build/sbt clean package") || raise("SQL doc generation failed") + + puts "Moving back into docs dir." + cd("docs") + + puts "Moving to SQL directory and building docs." + cd("../sql") + system("./create-docs.sh") || raise("SQL doc generation failed") + + puts "Moving back into docs dir." + cd("../docs") + + puts "Making directory api/sql" + mkdir_p "api/sql" + + puts "cp -r ../sql/site/. api/sql" + cp_r("../sql/site/.", "api/sql") + end + end diff --git a/docs/api.md b/docs/api.md index ae7d51c2aefbf..70484f02de78d 100644 --- a/docs/api.md +++ b/docs/api.md @@ -9,3 +9,4 @@ Here you can read API docs for Spark and its submodules. - [Spark Java API (Javadoc)](api/java/index.html) - [Spark Python API (Sphinx)](api/python/index.html) - [Spark R API (Roxygen2)](api/R/index.html) +- [Spark SQL, Built-in Functions (MkDocs)](api/sql/index.html) diff --git a/docs/index.md b/docs/index.md index 2d4607b3119bd..b867c972b4b48 100644 --- a/docs/index.md +++ b/docs/index.md @@ -100,6 +100,7 @@ options for deployment: * [Spark Java API (Javadoc)](api/java/index.html) * [Spark Python API (Sphinx)](api/python/index.html) * [Spark R API (Roxygen2)](api/R/index.html) +* [Spark SQL, Built-in Functions (MkDocs)](api/sql/index.html) **Deployment Guides:** diff --git a/sql/README.md b/sql/README.md index 58e9097ed4db1..fe1d352050c09 100644 --- a/sql/README.md +++ b/sql/README.md @@ -8,3 +8,5 @@ Spark SQL is broken up into four subprojects: - Execution (sql/core) - A query planner / execution engine for translating Catalyst's logical query plans into Spark RDDs. This component also includes a new public interface, SQLContext, that allows users to execute SQL or LINQ statements against existing RDDs and Parquet files. - Hive Support (sql/hive) - Includes an extension of SQLContext called HiveContext that allows users to write queries using a subset of HiveQL and access data from a Hive Metastore using Hive SerDes. There are also wrappers that allows users to run queries that include Hive UDFs, UDAFs, and UDTFs. - HiveServer and CLI support (sql/hive-thriftserver) - Includes support for the SQL CLI (bin/spark-sql) and a HiveServer2 (for JDBC/ODBC) compatible server. + +Running `sql/create-docs.sh` generates SQL documentation for built-in functions under `sql/site`. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala index 731feb914d251..4d5ce0bb60c0b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala @@ -17,9 +17,16 @@ package org.apache.spark.sql.api.python +import org.apache.spark.sql.catalyst.analysis.FunctionRegistry +import org.apache.spark.sql.catalyst.expressions.ExpressionInfo import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.types.DataType private[sql] object PythonSQLUtils { def parseDataType(typeText: String): DataType = CatalystSqlParser.parseDataType(typeText) + + // This is needed when generating SQL documentation for built-in functions. + def listBuiltinFunctionInfos(): Array[ExpressionInfo] = { + FunctionRegistry.functionSet.flatMap(f => FunctionRegistry.builtin.lookupFunction(f)).toArray + } } diff --git a/sql/create-docs.sh b/sql/create-docs.sh new file mode 100755 index 0000000000000..275e4c391a388 --- /dev/null +++ b/sql/create-docs.sh @@ -0,0 +1,49 @@ +#!/bin/bash + +# +# 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. +# + +# Script to create SQL API docs. This requires `mkdocs` and to build +# Spark first. After running this script the html docs can be found in +# $SPARK_HOME/sql/site + +set -o pipefail +set -e + +FWDIR="$(cd "`dirname "${BASH_SOURCE[0]}"`"; pwd)" +SPARK_HOME="$(cd "`dirname "${BASH_SOURCE[0]}"`"/..; pwd)" + +if ! hash python 2>/dev/null; then + echo "Missing python in your path, skipping SQL documentation generation." + exit 0 +fi + +if ! hash mkdocs 2>/dev/null; then + echo "Missing mkdocs in your path, skipping SQL documentation generation." + exit 0 +fi + +# Now create the markdown file +rm -fr docs +mkdir docs +echo "Generating markdown files for SQL documentation." +"$SPARK_HOME/bin/spark-submit" gen-sql-markdown.py + +# Now create the HTML files +echo "Generating HTML files for SQL documentation." +mkdocs build --clean +rm -fr docs diff --git a/sql/gen-sql-markdown.py b/sql/gen-sql-markdown.py new file mode 100644 index 0000000000000..8132af2708aea --- /dev/null +++ b/sql/gen-sql-markdown.py @@ -0,0 +1,91 @@ +# +# 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 sys +import os +from collections import namedtuple + +ExpressionInfo = namedtuple("ExpressionInfo", "className usage name extended") + + +def _list_function_infos(jvm): + """ + Returns a list of function information via JVM. Sorts wrapped expression infos by name + and returns them. + """ + + jinfos = jvm.org.apache.spark.sql.api.python.PythonSQLUtils.listBuiltinFunctionInfos() + infos = [] + for jinfo in jinfos: + name = jinfo.getName() + usage = jinfo.getUsage() + usage = usage.replace("_FUNC_", name) if usage is not None else usage + extended = jinfo.getExtended() + extended = extended.replace("_FUNC_", name) if extended is not None else extended + infos.append(ExpressionInfo( + className=jinfo.getClassName(), + usage=usage, + name=name, + extended=extended)) + return sorted(infos, key=lambda i: i.name) + + +def _make_pretty_usage(usage): + """ + Makes the usage description pretty and returns a formatted string. + Otherwise, returns None. + """ + + if usage is not None and usage.strip() != "": + usage = "\n".join(map(lambda u: u.strip(), usage.split("\n"))) + return "%s\n\n" % usage + + +def _make_pretty_extended(extended): + """ + Makes the extended description pretty and returns a formatted string. + Otherwise, returns None. + """ + + if extended is not None and extended.strip() != "": + extended = "\n".join(map(lambda u: u.strip(), extended.split("\n"))) + return "```%s```\n\n" % extended + + +def generate_sql_markdown(jvm, path): + """ + Generates a markdown file after listing the function information. The output file + is created in `path`. + """ + + with open(path, 'w') as mdfile: + for info in _list_function_infos(jvm): + mdfile.write("### %s\n\n" % info.name) + usage = _make_pretty_usage(info.usage) + extended = _make_pretty_extended(info.extended) + if usage is not None: + mdfile.write(usage) + if extended is not None: + mdfile.write(extended) + + +if __name__ == "__main__": + from pyspark.java_gateway import launch_gateway + + jvm = launch_gateway().jvm + markdown_file_path = "%s/docs/index.md" % os.path.dirname(sys.argv[0]) + generate_sql_markdown(jvm, markdown_file_path) diff --git a/sql/mkdocs.yml b/sql/mkdocs.yml new file mode 100644 index 0000000000000..c34c891bb9e42 --- /dev/null +++ b/sql/mkdocs.yml @@ -0,0 +1,19 @@ +# 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. + +site_name: Spark SQL, Built-in Functions +theme: readthedocs +pages: + - 'Functions': 'index.md'