[SPARK-36457][DOCS][3.2] Review and fix issues in Scala/Java API docs
### What changes were proposed in this pull request? Compare the 3.2.0 API doc with the latest release version 3.1.2. Fix the following issues: - Add missing `Since` annotation for new APIs - Remove the leaking class/object in API doc ### Why are the changes needed? Improve API docs ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing UT Closes #33845 from gengliangwang/SPARK-36457-3.2. Authored-by: Gengliang Wang <gengliang@apache.org> Signed-off-by: Gengliang Wang <gengliang@apache.org>
This commit is contained in:
parent
f50f2d474c
commit
eca81cc0ae
|
@ -72,7 +72,7 @@ private[spark] class SparkUpgradeException(version: String, message: String, cau
|
|||
/**
|
||||
* Arithmetic exception thrown from Spark with an error class.
|
||||
*/
|
||||
class SparkArithmeticException(errorClass: String, messageParameters: Array[String])
|
||||
private[spark] class SparkArithmeticException(errorClass: String, messageParameters: Array[String])
|
||||
extends ArithmeticException(SparkThrowableHelper.getMessage(errorClass, messageParameters))
|
||||
with SparkThrowable {
|
||||
|
||||
|
|
|
@ -24,7 +24,7 @@ import java.util.zip.Checksum
|
|||
* A variant of [[java.util.zip.CheckedOutputStream]] which can
|
||||
* change the checksum calculator at runtime.
|
||||
*/
|
||||
class MutableCheckedOutputStream(out: OutputStream) extends OutputStream {
|
||||
private[spark] class MutableCheckedOutputStream(out: OutputStream) extends OutputStream {
|
||||
private var checksum: Checksum = _
|
||||
|
||||
def setChecksum(c: Checksum): Unit = {
|
||||
|
|
|
@ -16,7 +16,7 @@
|
|||
*/
|
||||
package org.apache.spark.scheduler
|
||||
|
||||
import org.apache.spark.annotation.DeveloperApi
|
||||
import org.apache.spark.annotation.{DeveloperApi, Since}
|
||||
|
||||
/**
|
||||
* :: DeveloperApi ::
|
||||
|
@ -24,6 +24,7 @@ import org.apache.spark.annotation.DeveloperApi
|
|||
*/
|
||||
|
||||
@DeveloperApi
|
||||
@Since("3.2.0")
|
||||
class MiscellaneousProcessDetails(
|
||||
val hostPort: String,
|
||||
val cores: Int,
|
||||
|
|
|
@ -228,6 +228,7 @@ case class SparkListenerUnschedulableTaskSetRemoved(
|
|||
case class SparkListenerBlockUpdated(blockUpdatedInfo: BlockUpdatedInfo) extends SparkListenerEvent
|
||||
|
||||
@DeveloperApi
|
||||
@Since("3.2.0")
|
||||
case class SparkListenerMiscellaneousProcessAdded(time: Long, processId: String,
|
||||
info: MiscellaneousProcessDetails) extends SparkListenerEvent
|
||||
|
||||
|
|
|
@ -29,7 +29,7 @@ import org.apache.spark.deploy.SparkSubmitUtils
|
|||
import org.apache.spark.internal.Logging
|
||||
import org.apache.spark.internal.config._
|
||||
|
||||
case class IvyProperties(
|
||||
private[spark] case class IvyProperties(
|
||||
packagesExclusions: String,
|
||||
packages: String,
|
||||
repositories: String,
|
||||
|
|
|
@ -967,6 +967,7 @@ object Unidoc {
|
|||
.map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/sql/internal")))
|
||||
.map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/sql/hive")))
|
||||
.map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/sql/catalog/v2/utils")))
|
||||
.map(_.filterNot(_.getCanonicalPath.contains("org.apache.spark.sql.errors")))
|
||||
.map(_.filterNot(_.getCanonicalPath.contains("org/apache/hive")))
|
||||
.map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/sql/v2/avro")))
|
||||
.map(_.filterNot(_.getCanonicalPath.contains("SSLOptions")))
|
||||
|
|
|
@ -17,13 +17,17 @@
|
|||
|
||||
package org.apache.spark.sql.connector.catalog;
|
||||
|
||||
import org.apache.spark.annotation.Evolving;
|
||||
import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
|
||||
import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
|
||||
import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
|
||||
|
||||
/**
|
||||
* Catalog methods for working with Functions.
|
||||
*
|
||||
* @since 3.2.0
|
||||
*/
|
||||
@Evolving
|
||||
public interface FunctionCatalog extends CatalogPlugin {
|
||||
|
||||
/**
|
||||
|
|
|
@ -21,6 +21,8 @@ import org.apache.spark.annotation.Evolving;
|
|||
|
||||
/**
|
||||
* Represents a table which can be atomically truncated.
|
||||
*
|
||||
* @since 3.2.0
|
||||
*/
|
||||
@Evolving
|
||||
public interface TruncatableTable extends Table {
|
||||
|
|
|
@ -17,6 +17,7 @@
|
|||
|
||||
package org.apache.spark.sql.connector.catalog.functions;
|
||||
|
||||
import org.apache.spark.annotation.Evolving;
|
||||
import org.apache.spark.sql.catalyst.InternalRow;
|
||||
import org.apache.spark.sql.types.DataType;
|
||||
|
||||
|
@ -42,7 +43,10 @@ import java.io.Serializable;
|
|||
*
|
||||
* @param <S> the JVM type for the aggregation's intermediate state; must be {@link Serializable}
|
||||
* @param <R> the JVM type of result values
|
||||
*
|
||||
* @since 3.2.0
|
||||
*/
|
||||
@Evolving
|
||||
public interface AggregateFunction<S extends Serializable, R> extends BoundFunction {
|
||||
|
||||
/**
|
||||
|
|
|
@ -17,6 +17,7 @@
|
|||
|
||||
package org.apache.spark.sql.connector.catalog.functions;
|
||||
|
||||
import org.apache.spark.annotation.Evolving;
|
||||
import org.apache.spark.sql.types.DataType;
|
||||
import org.apache.spark.sql.types.IntegerType;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
|
@ -25,7 +26,10 @@ import java.util.UUID;
|
|||
|
||||
/**
|
||||
* Represents a function that is bound to an input type.
|
||||
*
|
||||
* @since 3.2.0
|
||||
*/
|
||||
@Evolving
|
||||
public interface BoundFunction extends Function {
|
||||
|
||||
/**
|
||||
|
|
|
@ -17,11 +17,16 @@
|
|||
|
||||
package org.apache.spark.sql.connector.catalog.functions;
|
||||
|
||||
import org.apache.spark.annotation.Evolving;
|
||||
|
||||
import java.io.Serializable;
|
||||
|
||||
/**
|
||||
* Base class for user-defined functions.
|
||||
*
|
||||
* @since 3.2.0
|
||||
*/
|
||||
@Evolving
|
||||
public interface Function extends Serializable {
|
||||
|
||||
/**
|
||||
|
|
|
@ -17,6 +17,7 @@
|
|||
|
||||
package org.apache.spark.sql.connector.catalog.functions;
|
||||
|
||||
import org.apache.spark.annotation.Evolving;
|
||||
import org.apache.spark.sql.catalyst.InternalRow;
|
||||
import org.apache.spark.sql.types.DataType;
|
||||
|
||||
|
@ -133,7 +134,10 @@ import org.apache.spark.sql.types.DataType;
|
|||
*
|
||||
* @param <R> the JVM type of result values, MUST be consistent with the {@link DataType}
|
||||
* returned via {@link #resultType()}, according to the mapping above.
|
||||
*
|
||||
* @since 3.2.0
|
||||
*/
|
||||
@Evolving
|
||||
public interface ScalarFunction<R> extends BoundFunction {
|
||||
String MAGIC_METHOD_NAME = "invoke";
|
||||
|
||||
|
|
|
@ -17,11 +17,15 @@
|
|||
|
||||
package org.apache.spark.sql.connector.catalog.functions;
|
||||
|
||||
import org.apache.spark.annotation.Evolving;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
|
||||
/**
|
||||
* Represents a user-defined function that is not bound to input types.
|
||||
*
|
||||
* @since 3.2.0
|
||||
*/
|
||||
@Evolving
|
||||
public interface UnboundFunction extends Function {
|
||||
|
||||
/**
|
||||
|
|
|
@ -25,6 +25,8 @@ import org.apache.spark.annotation.Evolving;
|
|||
/**
|
||||
* A mix-in interface for {@link SparkDataStream} streaming sources to signal that they can report
|
||||
* metrics.
|
||||
*
|
||||
* @since 3.2.0
|
||||
*/
|
||||
@Evolving
|
||||
public interface ReportsSourceMetrics extends SparkDataStream {
|
||||
|
|
|
@ -46,7 +46,7 @@ import org.apache.spark.sql.types._
|
|||
* As commands are executed eagerly, this also includes errors thrown during the execution of
|
||||
* commands, which users can see immediately.
|
||||
*/
|
||||
private[spark] object QueryCompilationErrors {
|
||||
object QueryCompilationErrors {
|
||||
|
||||
def groupingIDMismatchError(groupingID: GroupingID, groupByExprs: Seq[Expression]): Throwable = {
|
||||
new AnalysisException(
|
||||
|
|
|
@ -19,7 +19,7 @@ package org.apache.spark.sql.types
|
|||
|
||||
import scala.collection.mutable
|
||||
|
||||
import org.apache.spark.annotation.DeveloperApi
|
||||
import org.apache.spark.annotation.{DeveloperApi, Since}
|
||||
import org.apache.spark.internal.Logging
|
||||
import org.apache.spark.sql.errors.QueryExecutionErrors
|
||||
import org.apache.spark.util.Utils
|
||||
|
@ -31,6 +31,7 @@ import org.apache.spark.util.Utils
|
|||
* alternative approach to register UDTs for user classes.
|
||||
*/
|
||||
@DeveloperApi
|
||||
@Since("3.2.0")
|
||||
object UDTRegistration extends Serializable with Logging {
|
||||
|
||||
/** The mapping between the Class between UserDefinedType and user classes. */
|
||||
|
|
|
@ -22,7 +22,7 @@ import java.util.Objects
|
|||
import org.json4s.JsonAST.JValue
|
||||
import org.json4s.JsonDSL._
|
||||
|
||||
import org.apache.spark.annotation.DeveloperApi
|
||||
import org.apache.spark.annotation.{DeveloperApi, Since}
|
||||
|
||||
/**
|
||||
* The data type for User Defined Types (UDTs).
|
||||
|
@ -38,6 +38,7 @@ import org.apache.spark.annotation.DeveloperApi
|
|||
* The conversion via `deserialize` occurs when reading from a `DataFrame`.
|
||||
*/
|
||||
@DeveloperApi
|
||||
@Since("3.2.0")
|
||||
abstract class UserDefinedType[UserType >: Null] extends DataType with Serializable {
|
||||
|
||||
/** Underlying storage type for this UDT */
|
||||
|
|
|
@ -26,6 +26,8 @@ import org.apache.spark.sql.sources.InsertableRelation;
|
|||
* <p>
|
||||
* Tables that have {@link TableCapability#V1_BATCH_WRITE} in the list of their capabilities
|
||||
* must build {@link V1Write}.
|
||||
*
|
||||
* @since 3.2.0
|
||||
*/
|
||||
@Unstable
|
||||
public interface V1Write extends Write {
|
||||
|
|
Loading…
Reference in a new issue