This is an automated email from the ASF dual-hosted git repository. gurwls223 pushed a commit to branch branch-3.0 in repository https://gitbox.apache.org/repos/asf/spark.git
The following commit(s) were added to refs/heads/branch-3.0 by this push: new d586549 [SPARK-30510][SQL][DOCS] Publicly document Spark SQL configuration options d586549 is described below commit d5865493ae71e6369e9f3350dd7e694afcf57298 Author: Nicholas Chammas <nicholas.cham...@liveramp.com> AuthorDate: Sun Feb 9 19:20:47 2020 +0900 [SPARK-30510][SQL][DOCS] Publicly document Spark SQL configuration options ### What changes were proposed in this pull request? This PR adds a doc builder for Spark SQL's configuration options. Here's what the new Spark SQL config docs look like ([configuration.html.zip](https://github.com/apache/spark/files/4172109/configuration.html.zip)): ![Screen Shot 2020-02-07 at 12 13 23 PM](https://user-images.githubusercontent.com/1039369/74050007-425b5480-49a3-11ea-818c-42700c54d1fb.png) Compare this to the [current docs](http://spark.apache.org/docs/3.0.0-preview2/configuration.html#spark-sql): ![Screen Shot 2020-02-04 at 4 55 10 PM](https://user-images.githubusercontent.com/1039369/73790828-24a5a980-476f-11ea-998c-12cd613883e8.png) ### Why are the changes needed? There is no visibility into the various Spark SQL configs on [the config docs page](http://spark.apache.org/docs/3.0.0-preview2/configuration.html#spark-sql). ### Does this PR introduce any user-facing change? No, apart from new documentation. ### How was this patch tested? I tested this manually by building the docs and reviewing them in my browser. Closes #27459 from nchammas/SPARK-30510-spark-sql-options. Authored-by: Nicholas Chammas <nicholas.cham...@liveramp.com> Signed-off-by: HyukjinKwon <gurwls...@apache.org> (cherry picked from commit 339c0f9a623521acd4d66292b3fe3e6c4ec3b108) Signed-off-by: HyukjinKwon <gurwls...@apache.org> --- docs/.gitignore | 1 + docs/configuration.md | 46 ++------ sql/README.md | 2 +- .../org/apache/spark/sql/internal/SQLConf.scala | 35 +++--- .../spark/sql/api/python/PythonSQLUtils.scala | 7 ++ sql/create-docs.sh | 14 +-- sql/{gen-sql-markdown.py => gen-sql-api-docs.py} | 8 +- sql/gen-sql-config-docs.py | 117 +++++++++++++++++++++ 8 files changed, 163 insertions(+), 67 deletions(-) diff --git a/docs/.gitignore b/docs/.gitignore new file mode 100644 index 0000000..2260493 --- /dev/null +++ b/docs/.gitignore @@ -0,0 +1 @@ +sql-configs.html diff --git a/docs/configuration.md b/docs/configuration.md index 5bd3f3e..1343755 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -2399,47 +2399,15 @@ the driver or executor, or, in the absence of that value, the number of cores av Please refer to the [Security](security.html) page for available options on how to secure different Spark subsystems. -### Spark SQL - -Running the <code>SET -v</code> command will show the entire list of the SQL configuration. - -<div class="codetabs"> -<div data-lang="scala" markdown="1"> -{% highlight scala %} -// spark is an existing SparkSession -spark.sql("SET -v").show(numRows = 200, truncate = false) -{% endhighlight %} - -</div> - -<div data-lang="java" markdown="1"> - -{% highlight java %} -// spark is an existing SparkSession -spark.sql("SET -v").show(200, false); -{% endhighlight %} -</div> - -<div data-lang="python" markdown="1"> - -{% highlight python %} -# spark is an existing SparkSession -spark.sql("SET -v").show(n=200, truncate=False) -{% endhighlight %} - -</div> - -<div data-lang="r" markdown="1"> - -{% highlight r %} -sparkR.session() -properties <- sql("SET -v") -showDF(properties, numRows = 200, truncate = FALSE) -{% endhighlight %} +{% for static_file in site.static_files %} + {% if static_file.name == 'sql-configs.html' %} +### Spark SQL -</div> -</div> + {% include_relative sql-configs.html %} + {% break %} + {% endif %} +{% endfor %} ### Spark Streaming diff --git a/sql/README.md b/sql/README.md index 67e3225..ae5ebd1 100644 --- a/sql/README.md +++ b/sql/README.md @@ -9,4 +9,4 @@ Spark SQL is broken up into four subprojects: - Hive Support (sql/hive) - Includes extensions that allow users to write queries using a subset of HiveQL and access data from a Hive Metastore using Hive SerDes. There are also wrappers that allow 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`. +Running `./sql/create-docs.sh` generates SQL documentation for built-in functions under `sql/site`, and SQL configuration documentation that gets included as part of `configuration.md` in the main `docs` directory. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index a72bd53..e38fe76 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -324,11 +324,11 @@ object SQLConf { .doc("Configures the maximum size in bytes for a table that will be broadcast to all worker " + "nodes when performing a join. By setting this value to -1 broadcasting can be disabled. " + "Note that currently statistics are only supported for Hive Metastore tables where the " + - "command <code>ANALYZE TABLE <tableName> COMPUTE STATISTICS noscan</code> has been " + + "command `ANALYZE TABLE <tableName> COMPUTE STATISTICS noscan` has been " + "run, and file-based data source tables where the statistics are computed directly on " + "the files of data.") .bytesConf(ByteUnit.BYTE) - .createWithDefault(10L * 1024 * 1024) + .createWithDefaultString("10MB") val LIMIT_SCALE_UP_FACTOR = buildConf("spark.sql.limit.scaleUpFactor") .internal() @@ -402,7 +402,7 @@ object SQLConf { s"an effect when '${ADAPTIVE_EXECUTION_ENABLED.key}' and " + s"'${REDUCE_POST_SHUFFLE_PARTITIONS_ENABLED.key}' is enabled.") .bytesConf(ByteUnit.BYTE) - .createWithDefault(64 * 1024 * 1024) + .createWithDefaultString("64MB") val SHUFFLE_MAX_NUM_POSTSHUFFLE_PARTITIONS = buildConf("spark.sql.adaptive.shuffle.maxNumPostShufflePartitions") @@ -436,7 +436,7 @@ object SQLConf { .doc("Configures the minimum size in bytes for a partition that is considered as a skewed " + "partition in adaptive skewed join.") .bytesConf(ByteUnit.BYTE) - .createWithDefault(64 * 1024 * 1024) + .createWithDefaultString("64MB") val ADAPTIVE_EXECUTION_SKEWED_PARTITION_FACTOR = buildConf("spark.sql.adaptive.optimizeSkewedJoin.skewedPartitionFactor") @@ -770,7 +770,7 @@ object SQLConf { val BROADCAST_TIMEOUT = buildConf("spark.sql.broadcastTimeout") .doc("Timeout in seconds for the broadcast wait time in broadcast joins.") .timeConf(TimeUnit.SECONDS) - .createWithDefault(5 * 60) + .createWithDefaultString(s"${5 * 60}") // This is only used for the thriftserver val THRIFTSERVER_POOL = buildConf("spark.sql.thriftserver.scheduler.pool") @@ -830,7 +830,7 @@ object SQLConf { .createWithDefault(true) val BUCKETING_MAX_BUCKETS = buildConf("spark.sql.sources.bucketing.maxBuckets") - .doc("The maximum number of buckets allowed. Defaults to 100000") + .doc("The maximum number of buckets allowed.") .intConf .checkValue(_ > 0, "the value of spark.sql.sources.bucketing.maxBuckets must be greater than 0") .createWithDefault(100000) @@ -1022,7 +1022,7 @@ object SQLConf { "This configuration is effective only when using file-based sources such as Parquet, JSON " + "and ORC.") .bytesConf(ByteUnit.BYTE) - .createWithDefault(128 * 1024 * 1024) // parquet.block.size + .createWithDefaultString("128MB") // parquet.block.size val FILES_OPEN_COST_IN_BYTES = buildConf("spark.sql.files.openCostInBytes") .internal() @@ -1161,7 +1161,8 @@ object SQLConf { val VARIABLE_SUBSTITUTE_ENABLED = buildConf("spark.sql.variable.substitute") - .doc("This enables substitution using syntax like ${var} ${system:var} and ${env:var}.") + .doc("This enables substitution using syntax like `${var}`, `${system:var}`, " + + "and `${env:var}`.") .booleanConf .createWithDefault(true) @@ -1171,7 +1172,7 @@ object SQLConf { .doc("Enable two-level aggregate hash map. When enabled, records will first be " + "inserted/looked-up at a 1st-level, small, fast map, and then fallback to a " + "2nd-level, larger, slower map when 1st level is full or keys cannot be found. " + - "When disabled, records go directly to the 2nd level. Defaults to true.") + "When disabled, records go directly to the 2nd level.") .booleanConf .createWithDefault(true) @@ -1325,10 +1326,10 @@ object SQLConf { val STREAMING_STOP_TIMEOUT = buildConf("spark.sql.streaming.stopTimeout") - .doc("How long to wait for the streaming execution thread to stop when calling the " + - "streaming query's stop() method in milliseconds. 0 or negative values wait indefinitely.") + .doc("How long to wait in milliseconds for the streaming execution thread to stop when " + + "calling the streaming query's stop() method. 0 or negative values wait indefinitely.") .timeConf(TimeUnit.MILLISECONDS) - .createWithDefault(0L) + .createWithDefaultString("0") val STREAMING_NO_DATA_PROGRESS_EVENT_INTERVAL = buildConf("spark.sql.streaming.noDataProgressEventInterval") @@ -1611,10 +1612,10 @@ object SQLConf { val PANDAS_UDF_BUFFER_SIZE = buildConf("spark.sql.pandas.udf.buffer.size") .doc( - s"Same as ${BUFFER_SIZE} but only applies to Pandas UDF executions. If it is not set, " + - s"the fallback is ${BUFFER_SIZE}. Note that Pandas execution requires more than 4 bytes. " + - "Lowering this value could make small Pandas UDF batch iterated and pipelined; however, " + - "it might degrade performance. See SPARK-27870.") + s"Same as `${BUFFER_SIZE.key}` but only applies to Pandas UDF executions. If it is not " + + s"set, the fallback is `${BUFFER_SIZE.key}`. Note that Pandas execution requires more " + + "than 4 bytes. Lowering this value could make small Pandas UDF batch iterated and " + + "pipelined; however, it might degrade performance. See SPARK-27870.") .fallbackConf(BUFFER_SIZE) val PANDAS_GROUPED_MAP_ASSIGN_COLUMNS_BY_NAME = @@ -2039,7 +2040,7 @@ object SQLConf { .checkValue(i => i >= 0 && i <= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH, "Invalid " + "value for 'spark.sql.maxPlanStringLength'. Length must be a valid string length " + "(nonnegative and shorter than the maximum size).") - .createWithDefault(ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) + .createWithDefaultString(s"${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}") val SET_COMMAND_REJECTS_SPARK_CORE_CONFS = buildConf("spark.sql.legacy.setCommandRejectsSparkCoreConfs") 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 b232aa1..bf3055d 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 @@ -29,6 +29,7 @@ import org.apache.spark.sql.catalyst.expressions.ExpressionInfo import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.execution.{ExplainMode, QueryExecution} import org.apache.spark.sql.execution.arrow.ArrowConverters +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.DataType private[sql] object PythonSQLUtils { @@ -39,6 +40,12 @@ private[sql] object PythonSQLUtils { FunctionRegistry.functionSet.flatMap(f => FunctionRegistry.builtin.lookupFunction(f)).toArray } + def listSQLConfigs(): Array[(String, String, String)] = { + val conf = new SQLConf() + // Py4J doesn't seem to translate Seq well, so we convert to an Array. + conf.getAllDefinedConfs.toArray + } + /** * Python callable function to read a file in Arrow stream format and create a [[RDD]] * using each serialized ArrowRecordBatch as a partition. diff --git a/sql/create-docs.sh b/sql/create-docs.sh index 4353708..44aa877 100755 --- a/sql/create-docs.sh +++ b/sql/create-docs.sh @@ -17,7 +17,7 @@ # limitations under the License. # -# Script to create SQL API docs. This requires `mkdocs` and to build +# Script to create SQL API and config docs. This requires `mkdocs` and to build # Spark first. After running this script the html docs can be found in # $SPARK_HOME/sql/site @@ -39,14 +39,16 @@ fi pushd "$FWDIR" > /dev/null -# 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." +echo "Generating SQL API Markdown files." +"$SPARK_HOME/bin/spark-submit" gen-sql-api-docs.py + +echo "Generating SQL configuration table HTML file." +"$SPARK_HOME/bin/spark-submit" gen-sql-config-docs.py + +echo "Generating HTML files for SQL API documentation." mkdocs build --clean rm -fr docs diff --git a/sql/gen-sql-markdown.py b/sql/gen-sql-api-docs.py similarity index 96% rename from sql/gen-sql-markdown.py rename to sql/gen-sql-api-docs.py index e0529f8..4feee7a 100644 --- a/sql/gen-sql-markdown.py +++ b/sql/gen-sql-api-docs.py @@ -15,10 +15,11 @@ # limitations under the License. # -import sys import os from collections import namedtuple +from pyspark.java_gateway import launch_gateway + ExpressionInfo = namedtuple( "ExpressionInfo", "className name usage arguments examples note since deprecated") @@ -219,8 +220,7 @@ def generate_sql_markdown(jvm, path): 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]) + spark_root_dir = os.path.dirname(os.path.dirname(__file__)) + markdown_file_path = os.path.join(spark_root_dir, "sql/docs/index.md") generate_sql_markdown(jvm, markdown_file_path) diff --git a/sql/gen-sql-config-docs.py b/sql/gen-sql-config-docs.py new file mode 100644 index 0000000..04f5a85 --- /dev/null +++ b/sql/gen-sql-config-docs.py @@ -0,0 +1,117 @@ +# +# 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 re +from collections import namedtuple +from textwrap import dedent + +# To avoid adding a new direct dependency, we import markdown from within mkdocs. +from mkdocs.structure.pages import markdown +from pyspark.java_gateway import launch_gateway + +SQLConfEntry = namedtuple( + "SQLConfEntry", ["name", "default", "description"]) + + +def get_public_sql_configs(jvm): + sql_configs = [ + SQLConfEntry( + name=_sql_config._1(), + default=_sql_config._2(), + description=_sql_config._3(), + ) + for _sql_config in jvm.org.apache.spark.sql.api.python.PythonSQLUtils.listSQLConfigs() + ] + return sql_configs + + +def generate_sql_configs_table(sql_configs, path): + """ + Generates an HTML table at `path` that lists all public SQL + configuration options. + + The table will look something like this: + + ```html + <table class="table"> + <tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr> + + <tr> + <td><code>spark.sql.adaptive.enabled</code></td> + <td>false</td> + <td><p>When true, enable adaptive query execution.</p></td> + </tr> + + ... + + </table> + ``` + """ + value_reference_pattern = re.compile(r"^<value of (\S*)>$") + + with open(path, 'w') as f: + f.write(dedent( + """ + <table class="table"> + <tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr> + """ + )) + for config in sorted(sql_configs, key=lambda x: x.name): + if config.default == "<undefined>": + default = "(none)" + elif config.default.startswith("<value of "): + referenced_config_name = value_reference_pattern.match(config.default).group(1) + default = "(value of <code>{}</code>)".format(referenced_config_name) + else: + default = config.default + + if default.startswith("<"): + raise Exception( + "Unhandled reference in SQL config docs. Config '{name}' " + "has default '{default}' that looks like an HTML tag." + .format( + name=config.name, + default=config.default, + ) + ) + + f.write(dedent( + """ + <tr> + <td><code>{name}</code></td> + <td>{default}</td> + <td>{description}</td> + </tr> + """ + .format( + name=config.name, + default=default, + description=markdown.markdown(config.description), + ) + )) + f.write("</table>\n") + + +if __name__ == "__main__": + jvm = launch_gateway().jvm + sql_configs = get_public_sql_configs(jvm) + + spark_root_dir = os.path.dirname(os.path.dirname(__file__)) + sql_configs_table_path = os.path.join(spark_root_dir, "docs/sql-configs.html") + + generate_sql_configs_table(sql_configs, path=sql_configs_table_path) --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org