Skip to content

[SPARK-52010] Do not generate API docs for internal classes #50797

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Closed
wants to merge 1 commit into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -16,11 +16,11 @@
*/
package org.apache.spark.util

trait SparkStringUtils {
private[spark] trait SparkStringUtils {
Copy link
Contributor Author

@cloud-fan cloud-fan May 6, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

added in #49062

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
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@ import javax.tools.{JavaFileObject, SimpleJavaFileObject, ToolProvider}

import scala.jdk.CollectionConverters._

trait SparkTestUtils {
private[spark] trait SparkTestUtils {
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

added in #43354

// Adapted from the JavaCompiler.java doc examples
private val SOURCE = JavaFileObject.Kind.SOURCE

Expand Down Expand Up @@ -103,4 +103,4 @@ trait SparkTestUtils {

}

object SparkTestUtils extends SparkTestUtils
private[spark] object SparkTestUtils extends SparkTestUtils
Original file line number Diff line number Diff line change
Expand Up @@ -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(
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

4.0 added two new internal classes in this package: AvroExpressionEvalUtils, AvroCompressionCodec. Given that this package contains only internal classes and deprecated APIs since 3.0, I marked the entire org.apache.spark.sql.avro package as private, so that we no longer need private[sql] in these existing classes.

case class AvroDataToCatalyst(
child: Expression,
jsonFormatSchema: String,
options: Map[String, String])
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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 {

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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])
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

4.0 added one new internal class in this package: KafkaTokenProviderExceptions. Given that this package contains only internal classes, I marked the entire org.apache.spark.kafka010 package as private, so that we no longer need private[spark] in these existing classes.

case class KafkaConfigUpdater(module: String, kafkaParams: Map[String, Object])
extends Logging {
private val map = new ju.HashMap[String, Object](kafkaParams.asJava)

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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) =>
Expand All @@ -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 {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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,
Expand Down Expand Up @@ -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"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,25 +44,25 @@ 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 =
DateTimeFormatter
.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()
Expand All @@ -81,15 +81,15 @@ 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
require(!SparkHadoopUtil.get.isProxyUser(currentUser), "Obtaining delegation token for proxy " +
"user is not yet supported.")
}

private[kafka010] def createAdminClientProperties(
def createAdminClientProperties(
sparkConf: SparkConf,
clusterConf: KafkaTokenClusterConf): ju.Properties = {
val adminClientProperties = new ju.Properties
Expand Down
14 changes: 11 additions & 3 deletions project/SparkBuild.scala
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

"org.apache.spark.network",
"org.apache.spark.sql.avro",
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

"org.apache.spark.sql.scripting",
"org.apache.spark.types.variant",
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The new variant project adds this new package, but it only contain internal classes.

"org.apache.spark.ui.flamegraph",
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

added by #42988 , should be intrenal

"org.apache.spark.util.collection"
).mkString(":"),
"-doc-title", "Spark " + version.value.replaceAll("-SNAPSHOT", "") + " ScalaDoc"
),
Expand Down Expand Up @@ -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/")))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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 (
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

added by the new scripting feature.

errorClass: String,
cause: Throwable,
val origin: Origin,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

added by #46937


override def classifyException(
e: Throwable,
Expand Down