diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb index f95e4e2f97..8e2a06e4bc 100644 --- a/docs/_plugins/copy_api_dirs.rb +++ b/docs/_plugins/copy_api_dirs.rb @@ -157,8 +157,8 @@ if not (ENV['SKIP_API'] == '1') 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 "Running 'build/sbt clean package -Phive' from " + pwd + "; this may take a few minutes..." + system("build/sbt clean package -Phive") || raise("SQL doc generation failed") puts "Moving back into docs dir." cd("docs") 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 89bb4f362d..38254609e8 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 @@ -20,10 +20,14 @@ package org.apache.spark.sql.api.python import java.io.InputStream import java.nio.channels.Channels +import scala.util.control.NonFatal + import org.apache.spark.api.java.JavaRDD import org.apache.spark.api.python.PythonRDDServer +import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, SQLContext} +import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.analysis.FunctionRegistry import org.apache.spark.sql.catalyst.expressions.ExpressionInfo import org.apache.spark.sql.catalyst.parser.CatalystSqlParser @@ -32,7 +36,7 @@ import org.apache.spark.sql.execution.arrow.ArrowConverters import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} import org.apache.spark.sql.types.DataType -private[sql] object PythonSQLUtils { +private[sql] object PythonSQLUtils extends Logging { def parseDataType(typeText: String): DataType = CatalystSqlParser.parseDataType(typeText) // This is needed when generating SQL documentation for built-in functions. @@ -44,6 +48,14 @@ private[sql] object PythonSQLUtils { val conf = new SQLConf() // Force to build static SQL configurations StaticSQLConf + // Force to build SQL configurations from Hive module + try { + val symbol = ScalaReflection.mirror.staticModule("org.apache.spark.sql.hive.HiveUtils") + ScalaReflection.mirror.reflectModule(symbol).instance + } catch { + case NonFatal(e) => + logWarning("Cannot generated sql configurations from hive module", e) + } conf.getAllDefinedConfs }