From 9452d5d0d77ab4de1b3760cd0cbac0628ce3f731 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 6 May 2025 09:49:35 +0800 Subject: [PATCH] Do not generate API docs for private classes --- .../org/apache/spark/util/SparkStringUtils.scala | 4 ++-- .../org/apache/spark/util/SparkTestUtils.scala | 4 ++-- .../apache/spark/sql/avro/AvroDataToCatalyst.scala | 2 +- .../apache/spark/sql/avro/CatalystDataToAvro.scala | 2 +- .../org/apache/spark/sql/avro/SchemaOfAvro.scala | 2 +- .../apache/spark/kafka010/KafkaConfigUpdater.scala | 2 +- .../kafka010/KafkaDelegationTokenProvider.scala | 2 +- .../apache/spark/kafka010/KafkaRedactionUtil.scala | 6 +++--- .../spark/kafka010/KafkaTokenSparkConf.scala | 4 ++-- .../org/apache/spark/kafka010/KafkaTokenUtil.scala | 12 ++++++------ project/SparkBuild.scala | 14 +++++++++++--- .../sql/exceptions/SqlScriptingException.scala | 2 +- .../org/apache/spark/sql/jdbc/JdbcDialects.scala | 2 +- 13 files changed, 33 insertions(+), 25 deletions(-) diff --git a/common/utils/src/main/scala/org/apache/spark/util/SparkStringUtils.scala b/common/utils/src/main/scala/org/apache/spark/util/SparkStringUtils.scala index 6915f373b84e5..9f58288ee266c 100644 --- a/common/utils/src/main/scala/org/apache/spark/util/SparkStringUtils.scala +++ b/common/utils/src/main/scala/org/apache/spark/util/SparkStringUtils.scala @@ -16,11 +16,11 @@ */ package org.apache.spark.util -trait SparkStringUtils { +private[spark] trait SparkStringUtils { def stringToSeq(str: String): Seq[String] = { import org.apache.spark.util.ArrayImplicits._ str.split(",").map(_.trim()).filter(_.nonEmpty).toImmutableArraySeq } } -object SparkStringUtils extends SparkStringUtils +private[spark] object SparkStringUtils extends SparkStringUtils diff --git a/common/utils/src/main/scala/org/apache/spark/util/SparkTestUtils.scala b/common/utils/src/main/scala/org/apache/spark/util/SparkTestUtils.scala index bcb2668d31e65..34b40a5a8890c 100644 --- a/common/utils/src/main/scala/org/apache/spark/util/SparkTestUtils.scala +++ b/common/utils/src/main/scala/org/apache/spark/util/SparkTestUtils.scala @@ -25,7 +25,7 @@ import javax.tools.{JavaFileObject, SimpleJavaFileObject, ToolProvider} import scala.jdk.CollectionConverters._ -trait SparkTestUtils { +private[spark] trait SparkTestUtils { // Adapted from the JavaCompiler.java doc examples private val SOURCE = JavaFileObject.Kind.SOURCE @@ -103,4 +103,4 @@ trait SparkTestUtils { } -object SparkTestUtils extends SparkTestUtils +private[spark] object SparkTestUtils extends SparkTestUtils diff --git a/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala b/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala index 62c185a91d85c..c0b50e5cba480 100644 --- a/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala +++ b/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.util.{FailFastMode, ParseMode, PermissiveMo import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.types._ -private[sql] case class AvroDataToCatalyst( +case class AvroDataToCatalyst( child: Expression, jsonFormatSchema: String, options: Map[String, String]) diff --git a/connector/avro/src/main/scala/org/apache/spark/sql/avro/CatalystDataToAvro.scala b/connector/avro/src/main/scala/org/apache/spark/sql/avro/CatalystDataToAvro.scala index 56ed117aef580..01daac1cfa6b4 100644 --- a/connector/avro/src/main/scala/org/apache/spark/sql/avro/CatalystDataToAvro.scala +++ b/connector/avro/src/main/scala/org/apache/spark/sql/avro/CatalystDataToAvro.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.expressions.{Expression, UnaryExpression} import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.types.{BinaryType, DataType} -private[sql] case class CatalystDataToAvro( +case class CatalystDataToAvro( child: Expression, jsonFormatSchema: Option[String]) extends UnaryExpression { diff --git a/connector/avro/src/main/scala/org/apache/spark/sql/avro/SchemaOfAvro.scala b/connector/avro/src/main/scala/org/apache/spark/sql/avro/SchemaOfAvro.scala index e062ad7781064..362fc79a0f1b5 100644 --- a/connector/avro/src/main/scala/org/apache/spark/sql/avro/SchemaOfAvro.scala +++ b/connector/avro/src/main/scala/org/apache/spark/sql/avro/SchemaOfAvro.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.util.{FailFastMode, ParseMode, PermissiveMo import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.types.{DataType, ObjectType, StringType} -private[sql] case class SchemaOfAvro( +case class SchemaOfAvro( jsonFormatSchema: String, options: Map[String, String]) extends LeafExpression with RuntimeReplaceable { diff --git a/connector/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaConfigUpdater.scala b/connector/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaConfigUpdater.scala index 8251c0c43d9e8..17392f5a231db 100644 --- a/connector/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaConfigUpdater.scala +++ b/connector/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaConfigUpdater.scala @@ -30,7 +30,7 @@ import org.apache.spark.internal.Logging /** * Class to conveniently update Kafka config params, while logging the changes */ -private[spark] case class KafkaConfigUpdater(module: String, kafkaParams: Map[String, Object]) +case class KafkaConfigUpdater(module: String, kafkaParams: Map[String, Object]) extends Logging { private val map = new ju.HashMap[String, Object](kafkaParams.asJava) diff --git a/connector/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaDelegationTokenProvider.scala b/connector/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaDelegationTokenProvider.scala index 3616f93659fbb..8acdd01a15bab 100644 --- a/connector/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaDelegationTokenProvider.scala +++ b/connector/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaDelegationTokenProvider.scala @@ -28,7 +28,7 @@ import org.apache.spark.internal.{Logging, MDC} import org.apache.spark.internal.LogKeys.{CLUSTER_ID, SERVICE_NAME} import org.apache.spark.security.HadoopDelegationTokenProvider -private[spark] class KafkaDelegationTokenProvider +class KafkaDelegationTokenProvider extends HadoopDelegationTokenProvider with Logging { override def serviceName: String = "kafka" diff --git a/connector/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaRedactionUtil.scala b/connector/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaRedactionUtil.scala index c552e469ab837..44c0b1cb32c99 100644 --- a/connector/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaRedactionUtil.scala +++ b/connector/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaRedactionUtil.scala @@ -24,8 +24,8 @@ import org.apache.spark.internal.Logging import org.apache.spark.internal.config.SECRET_REDACTION_PATTERN import org.apache.spark.util.Utils.{redact, REDACTION_REPLACEMENT_TEXT} -private[spark] object KafkaRedactionUtil extends Logging { - private[spark] def redactParams(params: Seq[(String, Object)]): Seq[(String, String)] = { +object KafkaRedactionUtil extends Logging { + def redactParams(params: Seq[(String, Object)]): Seq[(String, String)] = { val redactionPattern = Some(Option(SparkEnv.get).map(_.conf) .getOrElse(new SparkConf()).get(SECRET_REDACTION_PATTERN)) params.map { case (key, value) => @@ -42,7 +42,7 @@ private[spark] object KafkaRedactionUtil extends Logging { } } - private[kafka010] def redactJaasParam(param: String): String = { + def redactJaasParam(param: String): String = { if (param != null && !param.isEmpty) { param.replaceAll("password=\".*\"", s"""password="$REDACTION_REPLACEMENT_TEXT"""") } else { diff --git a/connector/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaTokenSparkConf.scala b/connector/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaTokenSparkConf.scala index 21ba7b21ed9d6..64cc2cba6b30e 100644 --- a/connector/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaTokenSparkConf.scala +++ b/connector/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaTokenSparkConf.scala @@ -25,7 +25,7 @@ import org.apache.spark.SparkConf import org.apache.spark.internal.Logging import org.apache.spark.util.Utils.REDACTION_REPLACEMENT_TEXT -private[spark] case class KafkaTokenClusterConf( +case class KafkaTokenClusterConf( identifier: String, authBootstrapServers: String, targetServersRegex: String, @@ -57,7 +57,7 @@ private[spark] case class KafkaTokenClusterConf( s"specifiedKafkaParams=${KafkaRedactionUtil.redactParams(specifiedKafkaParams.toSeq)}}" } -private [kafka010] object KafkaTokenSparkConf extends Logging { +object KafkaTokenSparkConf extends Logging { val CLUSTERS_CONFIG_PREFIX = "spark.kafka.clusters." val DEFAULT_TARGET_SERVERS_REGEX = ".*" val DEFAULT_SASL_KERBEROS_SERVICE_NAME = "kafka" diff --git a/connector/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaTokenUtil.scala b/connector/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaTokenUtil.scala index 669491f142059..54014859bc5e5 100644 --- a/connector/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaTokenUtil.scala +++ b/connector/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaTokenUtil.scala @@ -44,7 +44,7 @@ import org.apache.spark.internal.config._ import org.apache.spark.util.{SecurityUtils, Utils} import org.apache.spark.util.Utils.REDACTION_REPLACEMENT_TEXT -private[spark] object KafkaTokenUtil extends Logging { +object KafkaTokenUtil extends Logging { val TOKEN_KIND = new Text("KAFKA_DELEGATION_TOKEN") private val TOKEN_SERVICE_PREFIX = "kafka.server.delegation.token" private val DATE_TIME_FORMATTER = @@ -52,17 +52,17 @@ private[spark] object KafkaTokenUtil extends Logging { .ofPattern("yyyy-MM-dd'T'HH:mm") .withZone(ZoneId.systemDefault()) - private[kafka010] def getTokenService(identifier: String): Text = + def getTokenService(identifier: String): Text = new Text(s"$TOKEN_SERVICE_PREFIX.$identifier") private def getClusterIdentifier(service: Text): String = service.toString().replace(s"$TOKEN_SERVICE_PREFIX.", "") - private[spark] class KafkaDelegationTokenIdentifier extends AbstractDelegationTokenIdentifier { + class KafkaDelegationTokenIdentifier extends AbstractDelegationTokenIdentifier { override def getKind: Text = TOKEN_KIND } - private[kafka010] def obtainToken( + def obtainToken( sparkConf: SparkConf, clusterConf: KafkaTokenClusterConf): (Token[KafkaDelegationTokenIdentifier], Long) = { checkProxyUser() @@ -81,7 +81,7 @@ private[spark] object KafkaTokenUtil extends Logging { ), token.tokenInfo.expiryTimestamp) } - private[kafka010] def checkProxyUser(): Unit = { + def checkProxyUser(): Unit = { val currentUser = UserGroupInformation.getCurrentUser() // Obtaining delegation token for proxy user is planned but not yet implemented // See https://issues.apache.org/jira/browse/KAFKA-6945 @@ -89,7 +89,7 @@ private[spark] object KafkaTokenUtil extends Logging { "user is not yet supported.") } - private[kafka010] def createAdminClientProperties( + def createAdminClientProperties( sparkConf: SparkConf, clusterConf: KafkaTokenClusterConf): ju.Properties = { val adminClientProperties = new ju.Properties diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index d282fa2611c2f..6b67f253e9a1c 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -334,10 +334,14 @@ object SparkBuild extends PomBuild { "-groups", "-skip-packages", Seq( "org.apache.spark.api.python", - "org.apache.spark.network", "org.apache.spark.deploy", - "org.apache.spark.util.collection", - "org.apache.spark.sql.scripting" + "org.apache.spark.kafka010", + "org.apache.spark.network", + "org.apache.spark.sql.avro", + "org.apache.spark.sql.scripting", + "org.apache.spark.types.variant", + "org.apache.spark.ui.flamegraph", + "org.apache.spark.util.collection" ).mkString(":"), "-doc-title", "Spark " + version.value.replaceAll("-SNAPSHOT", "") + " ScalaDoc" ), @@ -1380,10 +1384,14 @@ object Unidoc { f.getCanonicalPath.contains("org/apache/spark/unsafe") && !f.getCanonicalPath.contains("org/apache/spark/unsafe/types/CalendarInterval"))) .map(_.filterNot(_.getCanonicalPath.contains("python"))) + .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/kafka010"))) + .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/types/variant"))) + .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/ui/flamegraph"))) .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/util/collection"))) .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/util/io"))) .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/util/kvstore"))) .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/sql/artifact"))) + .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/sql/avro"))) .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/sql/catalyst"))) .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/sql/connect/"))) .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/sql/classic/"))) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/exceptions/SqlScriptingException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/exceptions/SqlScriptingException.scala index 7602366c71a65..28d8177dbb236 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/exceptions/SqlScriptingException.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/exceptions/SqlScriptingException.scala @@ -23,7 +23,7 @@ import org.apache.spark.{SparkThrowable, SparkThrowableHelper} import org.apache.spark.sql.catalyst.trees.Origin import org.apache.spark.sql.exceptions.SqlScriptingException.errorMessageWithLineNumber -class SqlScriptingException ( +private[sql] class SqlScriptingException ( errorClass: String, cause: Throwable, val origin: Origin, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala index 0d10b8e04484e..7719d6a67053c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala @@ -871,7 +871,7 @@ abstract class JdbcDialect extends Serializable with Logging { /** * Make the `classifyException` method throw out the original exception */ -trait NoLegacyJDBCError extends JdbcDialect { +private[sql] trait NoLegacyJDBCError extends JdbcDialect { override def classifyException( e: Throwable,