From fdeb288f955b68698298f62438f481bb6df202cc Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Fri, 6 Sep 2024 16:26:14 +0900 Subject: [PATCH] [SPARK-49511][SQL] Apply formatting rules to sql/api ### What changes were proposed in this pull request? This PR proposes to apply formatting rules to sql/api ### Why are the changes needed? The package is new, and we can start autoformatting. It won't have downside of backporting difficulty and etc. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Yes, by `dev/lint-scala` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #47989 from hvanhovell/SPARK-49511. Lead-authored-by: Herman van Hovell Co-authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon --- dev/lint-scala | 3 +- .../apache/spark/sql/AnalysisException.scala | 36 +- .../scala/org/apache/spark/sql/Artifact.scala | 3 +- .../scala/org/apache/spark/sql/Column.scala | 204 +- .../apache/spark/sql/DataFrameWriter.scala | 216 +- .../apache/spark/sql/DataFrameWriterV2.scala | 62 +- .../scala/org/apache/spark/sql/Encoder.scala | 8 +- .../apache/spark/sql/MergeIntoWriter.scala | 157 +- .../org/apache/spark/sql/Observation.scala | 35 +- .../main/scala/org/apache/spark/sql/Row.scala | 200 +- .../spark/sql/api/DataFrameNaFunctions.scala | 116 +- .../spark/sql/api/DataFrameReader.scala | 240 +- .../sql/api/DataFrameStatFunctions.scala | 358 +- .../org/apache/spark/sql/api/Dataset.scala | 1258 ++++--- .../sql/api/KeyValueGroupedDataset.scala | 984 ++--- .../sql/api/RelationalGroupedDataset.scala | 148 +- .../apache/spark/sql/api/SparkSession.scala | 173 +- .../spark/sql/api/UDFRegistration.scala | 978 ++++- .../catalyst/DefinedByConstructorParams.scala | 5 +- .../sql/catalyst/JavaTypeInference.scala | 32 +- .../spark/sql/catalyst/ScalaReflection.scala | 194 +- .../spark/sql/catalyst/WalkedTypePath.scala | 9 +- .../catalyst/analysis/NonEmptyException.scala | 13 +- .../catalyst/analysis/SqlApiAnalysis.scala | 5 +- .../analysis/alreadyExistException.scala | 106 +- .../analysis/noSuchItemsExceptions.scala | 114 +- .../catalyst/encoders/AgnosticEncoder.scala | 89 +- .../sql/catalyst/encoders/OuterScopes.scala | 15 +- .../sql/catalyst/encoders/RowEncoder.scala | 124 +- .../spark/sql/catalyst/encoders/codecs.scala | 6 +- .../sql/catalyst/expressions/OrderUtils.scala | 1 + .../spark/sql/catalyst/expressions/rows.scala | 5 +- .../catalyst/parser/DataTypeAstBuilder.scala | 2 +- .../parser/DataTypeParserInterface.scala | 5 +- .../parser/LegacyTypeStringParser.scala | 63 +- .../parser/SparkParserErrorStrategy.scala | 35 +- .../spark/sql/catalyst/parser/parsers.scala | 54 +- .../sql/catalyst/plans/logical/TimeMode.scala | 7 +- .../streaming/InternalOutputModes.scala | 20 +- .../sql/catalyst/trees/QueryContexts.scala | 35 +- .../spark/sql/catalyst/trees/origin.scala | 38 +- .../catalyst/util/AttributeNameParser.scala | 8 +- .../catalyst/util/CaseInsensitiveMap.scala | 11 +- .../sql/catalyst/util/DataTypeJsonUtils.scala | 11 +- .../sql/catalyst/util/DateFormatter.scala | 47 +- .../util/DateTimeFormatterHelper.scala | 78 +- .../spark/sql/catalyst/util/MathUtils.scala | 5 +- .../sql/catalyst/util/RebaseDateTime.scala | 301 +- .../catalyst/util/SparkCharVarcharUtils.scala | 7 +- .../catalyst/util/SparkDateTimeUtils.scala | 210 +- .../catalyst/util/SparkIntervalUtils.scala | 229 +- .../sql/catalyst/util/SparkParserUtils.scala | 24 +- .../spark/sql/catalyst/util/StringUtils.scala | 32 +- .../catalyst/util/TimestampFormatter.scala | 193 +- .../spark/sql/catalyst/util/UDTUtils.scala | 22 +- .../spark/sql/errors/CompilationErrors.scala | 15 +- .../spark/sql/errors/DataTypeErrors.scala | 89 +- .../spark/sql/errors/ExecutionErrors.scala | 45 +- .../spark/sql/errors/QueryParsingErrors.scala | 137 +- .../sql/execution/streaming/Triggers.scala | 4 +- .../spark/sql/expressions/Aggregator.scala | 15 +- .../sql/expressions/UserDefinedFunction.scala | 10 +- .../apache/spark/sql/expressions/Window.scala | 88 +- .../spark/sql/expressions/WindowSpec.scala | 63 +- .../apache/spark/sql/expressions/udaf.scala | 44 +- .../org/apache/spark/sql/functions.scala | 3280 +++++++++-------- .../spark/sql/internal/SqlApiConf.scala | 4 +- .../spark/sql/internal/SqlApiConfHelper.scala | 6 +- .../spark/sql/internal/ToScalaUDF.scala | 610 ++- .../spark/sql/internal/columnNodes.scala | 180 +- .../internal/types/AbstractArrayType.scala | 3 +- .../sql/internal/types/AbstractMapType.scala | 15 +- .../sql/streaming/ExpiredTimerInfo.scala | 5 +- .../spark/sql/streaming/GroupState.scala | 248 +- .../spark/sql/streaming/ListState.scala | 3 +- .../apache/spark/sql/streaming/MapState.scala | 6 +- .../spark/sql/streaming/QueryInfo.scala | 4 +- .../sql/streaming/StatefulProcessor.scala | 68 +- .../streaming/StatefulProcessorHandle.scala | 169 +- .../spark/sql/streaming/TTLConfig.scala | 10 +- .../spark/sql/streaming/TimerValues.scala | 18 +- .../spark/sql/streaming/ValueState.scala | 9 +- .../spark/sql/types/AbstractDataType.scala | 32 +- .../apache/spark/sql/types/ArrayType.scala | 29 +- .../apache/spark/sql/types/BinaryType.scala | 7 +- .../apache/spark/sql/types/BooleanType.scala | 3 +- .../org/apache/spark/sql/types/ByteType.scala | 4 +- .../sql/types/CalendarIntervalType.scala | 12 +- .../org/apache/spark/sql/types/DataType.scala | 273 +- .../org/apache/spark/sql/types/DateType.scala | 15 +- .../spark/sql/types/DayTimeIntervalType.scala | 19 +- .../org/apache/spark/sql/types/Decimal.scala | 122 +- .../apache/spark/sql/types/DecimalType.scala | 26 +- .../apache/spark/sql/types/DoubleType.scala | 3 +- .../apache/spark/sql/types/FloatType.scala | 4 +- .../apache/spark/sql/types/IntegerType.scala | 3 +- .../org/apache/spark/sql/types/LongType.scala | 3 +- .../org/apache/spark/sql/types/MapType.scala | 34 +- .../org/apache/spark/sql/types/Metadata.scala | 9 +- .../org/apache/spark/sql/types/NullType.scala | 2 +- .../apache/spark/sql/types/ShortType.scala | 3 +- .../apache/spark/sql/types/StringType.scala | 28 +- .../apache/spark/sql/types/StructField.scala | 19 +- .../apache/spark/sql/types/StructType.scala | 238 +- .../spark/sql/types/TimestampNTZType.scala | 17 +- .../spark/sql/types/TimestampType.scala | 14 +- .../spark/sql/types/UDTRegistration.scala | 27 +- .../apache/spark/sql/types/UpCastRule.scala | 16 +- .../spark/sql/types/UserDefinedType.scala | 19 +- .../apache/spark/sql/types/VariantType.scala | 6 +- .../sql/types/YearMonthIntervalType.scala | 13 +- .../apache/spark/sql/util/ArrowUtils.scala | 154 +- 112 files changed, 7952 insertions(+), 5684 deletions(-) diff --git a/dev/lint-scala b/dev/lint-scala index 98b850da68838..23df146a8d1b4 100755 --- a/dev/lint-scala +++ b/dev/lint-scala @@ -29,6 +29,7 @@ ERRORS=$(./build/mvn \ -Dscalafmt.skip=false \ -Dscalafmt.validateOnly=true \ -Dscalafmt.changedOnly=false \ + -pl sql/api \ -pl sql/connect/common \ -pl sql/connect/server \ -pl connector/connect/client/jvm \ @@ -38,7 +39,7 @@ ERRORS=$(./build/mvn \ if test ! -z "$ERRORS"; then echo -e "The scalafmt check failed on sql/connect or connector/connect at following occurrences:\n\n$ERRORS\n" echo "Before submitting your change, please make sure to format your code using the following command:" - echo "./build/mvn scalafmt:format -Dscalafmt.skip=false -Dscalafmt.validateOnly=false -Dscalafmt.changedOnly=false -pl sql/connect/common -pl sql/connect/server -pl connector/connect/client/jvm" + echo "./build/mvn scalafmt:format -Dscalafmt.skip=false -Dscalafmt.validateOnly=false -Dscalafmt.changedOnly=false -pl sql/api -pl sql/connect/common -pl sql/connect/server -pl connector/connect/client/jvm" exit 1 else echo -e "Scalafmt checks passed." diff --git a/sql/api/src/main/scala/org/apache/spark/sql/AnalysisException.scala b/sql/api/src/main/scala/org/apache/spark/sql/AnalysisException.scala index 7a428f6cc3288..a2c1f2cc41f8f 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/AnalysisException.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/AnalysisException.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.trees.{Origin, WithOrigin} * @since 1.3.0 */ @Stable -class AnalysisException protected( +class AnalysisException protected ( val message: String, val line: Option[Int] = None, val startPosition: Option[Int] = None, @@ -37,12 +37,12 @@ class AnalysisException protected( val errorClass: Option[String] = None, val messageParameters: Map[String, String] = Map.empty, val context: Array[QueryContext] = Array.empty) - extends Exception(message, cause.orNull) with SparkThrowable with Serializable with WithOrigin { + extends Exception(message, cause.orNull) + with SparkThrowable + with Serializable + with WithOrigin { - def this( - errorClass: String, - messageParameters: Map[String, String], - cause: Option[Throwable]) = + def this(errorClass: String, messageParameters: Map[String, String], cause: Option[Throwable]) = this( SparkThrowableHelper.getMessage(errorClass, messageParameters), errorClass = Some(errorClass), @@ -73,18 +73,10 @@ class AnalysisException protected( cause = null, context = context) - def this( - errorClass: String, - messageParameters: Map[String, String]) = - this( - errorClass = errorClass, - messageParameters = messageParameters, - cause = None) + def this(errorClass: String, messageParameters: Map[String, String]) = + this(errorClass = errorClass, messageParameters = messageParameters, cause = None) - def this( - errorClass: String, - messageParameters: Map[String, String], - origin: Origin) = + def this(errorClass: String, messageParameters: Map[String, String], origin: Origin) = this( SparkThrowableHelper.getMessage(errorClass, messageParameters), line = origin.line, @@ -115,8 +107,14 @@ class AnalysisException protected( errorClass: Option[String] = this.errorClass, messageParameters: Map[String, String] = this.messageParameters, context: Array[QueryContext] = this.context): AnalysisException = - new AnalysisException(message, line, startPosition, cause, errorClass, - messageParameters, context) + new AnalysisException( + message, + line, + startPosition, + cause, + errorClass, + messageParameters, + context) def withPosition(origin: Origin): AnalysisException = { val newException = this.copy( diff --git a/sql/api/src/main/scala/org/apache/spark/sql/Artifact.scala b/sql/api/src/main/scala/org/apache/spark/sql/Artifact.scala index c78280af6e021..7e020df06fe47 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/Artifact.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/Artifact.scala @@ -28,8 +28,7 @@ import org.apache.spark.sql.util.ArtifactUtils import org.apache.spark.util.ArrayImplicits._ import org.apache.spark.util.MavenUtils - -private[sql] class Artifact private(val path: Path, val storage: LocalData) { +private[sql] class Artifact private (val path: Path, val storage: LocalData) { require(!path.isAbsolute, s"Bad path: $path") lazy val size: Long = storage match { diff --git a/sql/api/src/main/scala/org/apache/spark/sql/Column.scala b/sql/api/src/main/scala/org/apache/spark/sql/Column.scala index cd6a04b2a0562..31ce44eca1684 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/Column.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/Column.scala @@ -72,34 +72,34 @@ private[spark] object Column { isDistinct: Boolean, isInternal: Boolean, inputs: Seq[Column]): Column = withOrigin { - Column(internal.UnresolvedFunction( - name, - inputs.map(_.node), - isDistinct = isDistinct, - isInternal = isInternal)) + Column( + internal.UnresolvedFunction( + name, + inputs.map(_.node), + isDistinct = isDistinct, + isInternal = isInternal)) } } /** - * A [[Column]] where an [[Encoder]] has been given for the expected input and return type. - * To create a [[TypedColumn]], use the `as` function on a [[Column]]. + * A [[Column]] where an [[Encoder]] has been given for the expected input and return type. To + * create a [[TypedColumn]], use the `as` function on a [[Column]]. * - * @tparam T The input type expected for this expression. Can be `Any` if the expression is type - * checked by the analyzer instead of the compiler (i.e. `expr("sum(...)")`). - * @tparam U The output type of this column. + * @tparam T + * The input type expected for this expression. Can be `Any` if the expression is type checked + * by the analyzer instead of the compiler (i.e. `expr("sum(...)")`). + * @tparam U + * The output type of this column. * * @since 1.6.0 */ @Stable -class TypedColumn[-T, U]( - node: ColumnNode, - private[sql] val encoder: Encoder[U]) - extends Column(node) { +class TypedColumn[-T, U](node: ColumnNode, private[sql] val encoder: Encoder[U]) + extends Column(node) { /** - * Gives the [[TypedColumn]] a name (alias). - * If the current `TypedColumn` has metadata associated with it, this metadata will be propagated - * to the new column. + * Gives the [[TypedColumn]] a name (alias). If the current `TypedColumn` has metadata + * associated with it, this metadata will be propagated to the new column. * * @group expr_ops * @since 2.0.0 @@ -168,23 +168,20 @@ class Column(val node: ColumnNode) extends Logging { override def hashCode: Int = this.node.normalized.hashCode() /** - * Provides a type hint about the expected return value of this column. This information can - * be used by operations such as `select` on a [[Dataset]] to automatically convert the - * results into the correct JVM types. + * Provides a type hint about the expected return value of this column. This information can be + * used by operations such as `select` on a [[Dataset]] to automatically convert the results + * into the correct JVM types. * @since 1.6.0 */ - def as[U : Encoder]: TypedColumn[Any, U] = new TypedColumn[Any, U](node, implicitly[Encoder[U]]) + def as[U: Encoder]: TypedColumn[Any, U] = new TypedColumn[Any, U](node, implicitly[Encoder[U]]) /** - * Extracts a value or values from a complex type. - * The following types of extraction are supported: - *
    - *
  • Given an Array, an integer ordinal can be used to retrieve a single value.
  • - *
  • Given a Map, a key of the correct type can be used to retrieve an individual value.
  • - *
  • Given a Struct, a string fieldName can be used to extract that field.
  • - *
  • Given an Array of Structs, a string fieldName can be used to extract filed - * of every struct in that array, and return an Array of fields.
  • - *
+ * Extracts a value or values from a complex type. The following types of extraction are + * supported:
  • Given an Array, an integer ordinal can be used to retrieve a single + * value.
  • Given a Map, a key of the correct type can be used to retrieve an individual + * value.
  • Given a Struct, a string fieldName can be used to extract that field.
  • + *
  • Given an Array of Structs, a string fieldName can be used to extract filed of every + * struct in that array, and return an Array of fields.
* @group expr_ops * @since 1.4.0 */ @@ -283,8 +280,8 @@ class Column(val node: ColumnNode) extends Logging { * * @group expr_ops * @since 2.0.0 - */ - def =!= (other: Any): Column = !(this === other) + */ + def =!=(other: Any): Column = !(this === other) /** * Inequality test. @@ -300,9 +297,9 @@ class Column(val node: ColumnNode) extends Logging { * * @group expr_ops * @since 1.3.0 - */ + */ @deprecated("!== does not have the same precedence as ===, use =!= instead", "2.0.0") - def !== (other: Any): Column = this =!= other + def !==(other: Any): Column = this =!= other /** * Inequality test. @@ -464,8 +461,8 @@ class Column(val node: ColumnNode) extends Logging { def eqNullSafe(other: Any): Column = this <=> other /** - * Evaluates a list of conditions and returns one of multiple possible result expressions. - * If otherwise is not defined at the end, null is returned for unmatched conditions. + * Evaluates a list of conditions and returns one of multiple possible result expressions. If + * otherwise is not defined at the end, null is returned for unmatched conditions. * * {{{ * // Example: encoding gender string column into integer. @@ -489,8 +486,7 @@ class Column(val node: ColumnNode) extends Logging { case internal.CaseWhenOtherwise(branches, None, _) => internal.CaseWhenOtherwise(branches :+ ((condition.node, lit(value).node)), None) case internal.CaseWhenOtherwise(_, Some(_), _) => - throw new IllegalArgumentException( - "when() cannot be applied once otherwise() is applied") + throw new IllegalArgumentException("when() cannot be applied once otherwise() is applied") case _ => throw new IllegalArgumentException( "when() can only be applied on a Column previously generated by when() function") @@ -498,8 +494,8 @@ class Column(val node: ColumnNode) extends Logging { } /** - * Evaluates a list of conditions and returns one of multiple possible result expressions. - * If otherwise is not defined at the end, null is returned for unmatched conditions. + * Evaluates a list of conditions and returns one of multiple possible result expressions. If + * otherwise is not defined at the end, null is returned for unmatched conditions. * * {{{ * // Example: encoding gender string column into integer. @@ -765,13 +761,11 @@ class Column(val node: ColumnNode) extends Logging { * A boolean expression that is evaluated to true if the value of this expression is contained * by the evaluated values of the arguments. * - * Note: Since the type of the elements in the list are inferred only during the run time, - * the elements will be "up-casted" to the most common type for comparison. - * For eg: - * 1) In the case of "Int vs String", the "Int" will be up-casted to "String" and the - * comparison will look like "String vs String". - * 2) In the case of "Float vs Double", the "Float" will be up-casted to "Double" and the - * comparison will look like "Double vs Double" + * Note: Since the type of the elements in the list are inferred only during the run time, the + * elements will be "up-casted" to the most common type for comparison. For eg: 1) In the case + * of "Int vs String", the "Int" will be up-casted to "String" and the comparison will look like + * "String vs String". 2) In the case of "Float vs Double", the "Float" will be up-casted to + * "Double" and the comparison will look like "Double vs Double" * * @group expr_ops * @since 1.5.0 @@ -784,12 +778,10 @@ class Column(val node: ColumnNode) extends Logging { * by the provided collection. * * Note: Since the type of the elements in the collection are inferred only during the run time, - * the elements will be "up-casted" to the most common type for comparison. - * For eg: - * 1) In the case of "Int vs String", the "Int" will be up-casted to "String" and the - * comparison will look like "String vs String". - * 2) In the case of "Float vs Double", the "Float" will be up-casted to "Double" and the - * comparison will look like "Double vs Double" + * the elements will be "up-casted" to the most common type for comparison. For eg: 1) In the + * case of "Int vs String", the "Int" will be up-casted to "String" and the comparison will look + * like "String vs String". 2) In the case of "Float vs Double", the "Float" will be up-casted + * to "Double" and the comparison will look like "Double vs Double" * * @group expr_ops * @since 2.4.0 @@ -801,12 +793,10 @@ class Column(val node: ColumnNode) extends Logging { * by the provided collection. * * Note: Since the type of the elements in the collection are inferred only during the run time, - * the elements will be "up-casted" to the most common type for comparison. - * For eg: - * 1) In the case of "Int vs String", the "Int" will be up-casted to "String" and the - * comparison will look like "String vs String". - * 2) In the case of "Float vs Double", the "Float" will be up-casted to "Double" and the - * comparison will look like "Double vs Double" + * the elements will be "up-casted" to the most common type for comparison. For eg: 1) In the + * case of "Int vs String", the "Int" will be up-casted to "String" and the comparison will look + * like "String vs String". 2) In the case of "Float vs Double", the "Float" will be up-casted + * to "Double" and the comparison will look like "Double vs Double" * * @group java_expr_ops * @since 2.4.0 @@ -822,8 +812,7 @@ class Column(val node: ColumnNode) extends Logging { def like(literal: String): Column = fn("like", literal) /** - * SQL RLIKE expression (LIKE with Regex). Returns a boolean column based on a regex - * match. + * SQL RLIKE expression (LIKE with Regex). Returns a boolean column based on a regex match. * * @group expr_ops * @since 1.3.0 @@ -839,8 +828,8 @@ class Column(val node: ColumnNode) extends Logging { def ilike(literal: String): Column = fn("ilike", literal) /** - * An expression that gets an item at position `ordinal` out of an array, - * or gets a value by key `key` in a `MapType`. + * An expression that gets an item at position `ordinal` out of an array, or gets a value by key + * `key` in a `MapType`. * * @group expr_ops * @since 1.3.0 @@ -885,8 +874,8 @@ class Column(val node: ColumnNode) extends Logging { * // result: {"a":{"a":1,"b":2,"c":3,"d":4}} * }}} * - * However, if you are going to add/replace multiple nested fields, it is more optimal to extract - * out the nested struct before adding/replacing multiple fields e.g. + * However, if you are going to add/replace multiple nested fields, it is more optimal to + * extract out the nested struct before adding/replacing multiple fields e.g. * * {{{ * val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col") @@ -906,8 +895,8 @@ class Column(val node: ColumnNode) extends Logging { // scalastyle:off line.size.limit /** - * An expression that drops fields in `StructType` by name. - * This is a no-op if schema doesn't contain field name(s). + * An expression that drops fields in `StructType` by name. This is a no-op if schema doesn't + * contain field name(s). * * {{{ * val df = sql("SELECT named_struct('a', 1, 'b', 2) struct_col") @@ -951,8 +940,8 @@ class Column(val node: ColumnNode) extends Logging { * // result: {"a":{"a":1}} * }}} * - * However, if you are going to drop multiple nested fields, it is more optimal to extract - * out the nested struct before dropping multiple fields from it e.g. + * However, if you are going to drop multiple nested fields, it is more optimal to extract out + * the nested struct before dropping multiple fields from it e.g. * * {{{ * val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col") @@ -980,8 +969,10 @@ class Column(val node: ColumnNode) extends Logging { /** * An expression that returns a substring. - * @param startPos expression for the starting position. - * @param len expression for the length of the substring. + * @param startPos + * expression for the starting position. + * @param len + * expression for the length of the substring. * * @group expr_ops * @since 1.3.0 @@ -990,8 +981,10 @@ class Column(val node: ColumnNode) extends Logging { /** * An expression that returns a substring. - * @param startPos starting position. - * @param len length of the substring. + * @param startPos + * starting position. + * @param len + * length of the substring. * * @group expr_ops * @since 1.3.0 @@ -1057,9 +1050,9 @@ class Column(val node: ColumnNode) extends Logging { * df.select($"colA".as("colB")) * }}} * - * If the current column has metadata associated with it, this metadata will be propagated - * to the new column. If this not desired, use the API `as(alias: String, metadata: Metadata)` - * with explicit metadata. + * If the current column has metadata associated with it, this metadata will be propagated to + * the new column. If this not desired, use the API `as(alias: String, metadata: Metadata)` with + * explicit metadata. * * @group expr_ops * @since 1.3.0 @@ -1097,9 +1090,9 @@ class Column(val node: ColumnNode) extends Logging { * df.select($"colA".as("colB")) * }}} * - * If the current column has metadata associated with it, this metadata will be propagated - * to the new column. If this not desired, use the API `as(alias: String, metadata: Metadata)` - * with explicit metadata. + * If the current column has metadata associated with it, this metadata will be propagated to + * the new column. If this not desired, use the API `as(alias: String, metadata: Metadata)` with + * explicit metadata. * * @group expr_ops * @since 1.3.0 @@ -1126,9 +1119,9 @@ class Column(val node: ColumnNode) extends Logging { * df.select($"colA".name("colB")) * }}} * - * If the current column has metadata associated with it, this metadata will be propagated - * to the new column. If this not desired, use the API `as(alias: String, metadata: Metadata)` - * with explicit metadata. + * If the current column has metadata associated with it, this metadata will be propagated to + * the new column. If this not desired, use the API `as(alias: String, metadata: Metadata)` with + * explicit metadata. * * @group expr_ops * @since 2.0.0 @@ -1152,9 +1145,9 @@ class Column(val node: ColumnNode) extends Logging { def cast(to: DataType): Column = Column(internal.Cast(node, to)) /** - * Casts the column to a different data type, using the canonical string representation - * of the type. The supported types are: `string`, `boolean`, `byte`, `short`, `int`, `long`, - * `float`, `double`, `decimal`, `date`, `timestamp`. + * Casts the column to a different data type, using the canonical string representation of the + * type. The supported types are: `string`, `boolean`, `byte`, `short`, `int`, `long`, `float`, + * `double`, `decimal`, `date`, `timestamp`. * {{{ * // Casts colA to integer. * df.select(df("colA").cast("int")) @@ -1224,8 +1217,8 @@ class Column(val node: ColumnNode) extends Logging { def desc: Column = desc_nulls_last /** - * Returns a sort expression based on the descending order of the column, - * and null values appear before non-null values. + * Returns a sort expression based on the descending order of the column, and null values appear + * before non-null values. * {{{ * // Scala: sort a DataFrame by age column in descending order and null values appearing first. * df.sort(df("age").desc_nulls_first) @@ -1237,13 +1230,12 @@ class Column(val node: ColumnNode) extends Logging { * @group expr_ops * @since 2.1.0 */ - def desc_nulls_first: Column = sortOrder( - internal.SortOrder.Descending, - internal.SortOrder.NullsFirst) + def desc_nulls_first: Column = + sortOrder(internal.SortOrder.Descending, internal.SortOrder.NullsFirst) /** - * Returns a sort expression based on the descending order of the column, - * and null values appear after non-null values. + * Returns a sort expression based on the descending order of the column, and null values appear + * after non-null values. * {{{ * // Scala: sort a DataFrame by age column in descending order and null values appearing last. * df.sort(df("age").desc_nulls_last) @@ -1255,9 +1247,8 @@ class Column(val node: ColumnNode) extends Logging { * @group expr_ops * @since 2.1.0 */ - def desc_nulls_last: Column = sortOrder( - internal.SortOrder.Descending, - internal.SortOrder.NullsLast) + def desc_nulls_last: Column = + sortOrder(internal.SortOrder.Descending, internal.SortOrder.NullsLast) /** * Returns a sort expression based on ascending order of the column. @@ -1275,8 +1266,8 @@ class Column(val node: ColumnNode) extends Logging { def asc: Column = asc_nulls_first /** - * Returns a sort expression based on ascending order of the column, - * and null values return before non-null values. + * Returns a sort expression based on ascending order of the column, and null values return + * before non-null values. * {{{ * // Scala: sort a DataFrame by age column in ascending order and null values appearing first. * df.sort(df("age").asc_nulls_first) @@ -1288,13 +1279,12 @@ class Column(val node: ColumnNode) extends Logging { * @group expr_ops * @since 2.1.0 */ - def asc_nulls_first: Column = sortOrder( - internal.SortOrder.Ascending, - internal.SortOrder.NullsFirst) + def asc_nulls_first: Column = + sortOrder(internal.SortOrder.Ascending, internal.SortOrder.NullsFirst) /** - * Returns a sort expression based on ascending order of the column, - * and null values appear after non-null values. + * Returns a sort expression based on ascending order of the column, and null values appear + * after non-null values. * {{{ * // Scala: sort a DataFrame by age column in ascending order and null values appearing last. * df.sort(df("age").asc_nulls_last) @@ -1306,9 +1296,8 @@ class Column(val node: ColumnNode) extends Logging { * @group expr_ops * @since 2.1.0 */ - def asc_nulls_last: Column = sortOrder( - internal.SortOrder.Ascending, - internal.SortOrder.NullsLast) + def asc_nulls_last: Column = + sortOrder(internal.SortOrder.Ascending, internal.SortOrder.NullsLast) /** * Prints the expression to the console for debugging purposes. @@ -1378,8 +1367,8 @@ class Column(val node: ColumnNode) extends Logging { } /** - * Defines an empty analytic clause. In this case the analytic function is applied - * and presented for all rows in the result set. + * Defines an empty analytic clause. In this case the analytic function is applied and presented + * for all rows in the result set. * * {{{ * df.select( @@ -1395,7 +1384,6 @@ class Column(val node: ColumnNode) extends Logging { } - /** * A convenient class used for constructing schema. * diff --git a/sql/api/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/api/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 96855ee5ad164..1838c6bc8468f 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -34,15 +34,11 @@ import org.apache.spark.sql.errors.CompilationErrors abstract class DataFrameWriter[T] { /** - * Specifies the behavior when data or table already exists. Options include: - *
    - *
  • `SaveMode.Overwrite`: overwrite the existing data.
  • - *
  • `SaveMode.Append`: append the data.
  • - *
  • `SaveMode.Ignore`: ignore the operation (i.e. no-op).
  • - *
  • `SaveMode.ErrorIfExists`: throw an exception at runtime.
  • - *
- *

- * The default option is `ErrorIfExists`. + * Specifies the behavior when data or table already exists. Options include:

    + *
  • `SaveMode.Overwrite`: overwrite the existing data.
  • `SaveMode.Append`: append the + * data.
  • `SaveMode.Ignore`: ignore the operation (i.e. no-op).
  • + *
  • `SaveMode.ErrorIfExists`: throw an exception at runtime.

The default + * option is `ErrorIfExists`. * * @since 1.4.0 */ @@ -52,13 +48,10 @@ abstract class DataFrameWriter[T] { } /** - * Specifies the behavior when data or table already exists. Options include: - *

    - *
  • `overwrite`: overwrite the existing data.
  • - *
  • `append`: append the data.
  • - *
  • `ignore`: ignore the operation (i.e. no-op).
  • - *
  • `error` or `errorifexists`: default option, throw an exception at runtime.
  • - *
+ * Specifies the behavior when data or table already exists. Options include:
    + *
  • `overwrite`: overwrite the existing data.
  • `append`: append the data.
  • + *
  • `ignore`: ignore the operation (i.e. no-op).
  • `error` or `errorifexists`: default + * option, throw an exception at runtime.
* * @since 1.4.0 */ @@ -85,8 +78,8 @@ abstract class DataFrameWriter[T] { /** * Adds an output option for the underlying data source. * - * All options are maintained in a case-insensitive way in terms of key names. - * If a new option has the same key case-insensitively, it will override the existing option. + * All options are maintained in a case-insensitive way in terms of key names. If a new option + * has the same key case-insensitively, it will override the existing option. * * @since 1.4.0 */ @@ -98,8 +91,8 @@ abstract class DataFrameWriter[T] { /** * Adds an output option for the underlying data source. * - * All options are maintained in a case-insensitive way in terms of key names. - * If a new option has the same key case-insensitively, it will override the existing option. + * All options are maintained in a case-insensitive way in terms of key names. If a new option + * has the same key case-insensitively, it will override the existing option. * * @since 2.0.0 */ @@ -108,8 +101,8 @@ abstract class DataFrameWriter[T] { /** * Adds an output option for the underlying data source. * - * All options are maintained in a case-insensitive way in terms of key names. - * If a new option has the same key case-insensitively, it will override the existing option. + * All options are maintained in a case-insensitive way in terms of key names. If a new option + * has the same key case-insensitively, it will override the existing option. * * @since 2.0.0 */ @@ -118,8 +111,8 @@ abstract class DataFrameWriter[T] { /** * Adds an output option for the underlying data source. * - * All options are maintained in a case-insensitive way in terms of key names. - * If a new option has the same key case-insensitively, it will override the existing option. + * All options are maintained in a case-insensitive way in terms of key names. If a new option + * has the same key case-insensitively, it will override the existing option. * * @since 2.0.0 */ @@ -128,8 +121,8 @@ abstract class DataFrameWriter[T] { /** * (Scala-specific) Adds output options for the underlying data source. * - * All options are maintained in a case-insensitive way in terms of key names. - * If a new option has the same key case-insensitively, it will override the existing option. + * All options are maintained in a case-insensitive way in terms of key names. If a new option + * has the same key case-insensitively, it will override the existing option. * * @since 1.4.0 */ @@ -141,8 +134,8 @@ abstract class DataFrameWriter[T] { /** * Adds output options for the underlying data source. * - * All options are maintained in a case-insensitive way in terms of key names. - * If a new option has the same key case-insensitively, it will override the existing option. + * All options are maintained in a case-insensitive way in terms of key names. If a new option + * has the same key case-insensitively, it will override the existing option. * * @since 1.4.0 */ @@ -154,16 +147,13 @@ abstract class DataFrameWriter[T] { /** * Partitions the output by the given columns on the file system. If specified, the output is * laid out on the file system similar to Hive's partitioning scheme. As an example, when we - * partition a dataset by year and then month, the directory layout would look like: - *
    - *
  • year=2016/month=01/
  • - *
  • year=2016/month=02/
  • - *
+ * partition a dataset by year and then month, the directory layout would look like:
    + *
  • year=2016/month=01/
  • year=2016/month=02/
* - * Partitioning is one of the most widely used techniques to optimize physical data layout. - * It provides a coarse-grained index for skipping unnecessary data reads when queries have - * predicates on the partitioned columns. In order for partitioning to work well, the number - * of distinct values in each column should typically be less than tens of thousands. + * Partitioning is one of the most widely used techniques to optimize physical data layout. It + * provides a coarse-grained index for skipping unnecessary data reads when queries have + * predicates on the partitioned columns. In order for partitioning to work well, the number of + * distinct values in each column should typically be less than tens of thousands. * * This is applicable for all file-based data sources (e.g. Parquet, JSON) starting with Spark * 2.1.0. @@ -179,8 +169,8 @@ abstract class DataFrameWriter[T] { /** * Buckets the output by the given columns. If specified, the output is laid out on the file - * system similar to Hive's bucketing scheme, but with a different bucket hash function - * and is not compatible with Hive's bucketing. + * system similar to Hive's bucketing scheme, but with a different bucket hash function and is + * not compatible with Hive's bucketing. * * This is applicable for all file-based data sources (e.g. Parquet, JSON) starting with Spark * 2.1.0. @@ -241,13 +231,15 @@ abstract class DataFrameWriter[T] { def save(): Unit /** - * Inserts the content of the `DataFrame` to the specified table. It requires that - * the schema of the `DataFrame` is the same as the schema of the table. + * Inserts the content of the `DataFrame` to the specified table. It requires that the schema of + * the `DataFrame` is the same as the schema of the table. * - * @note Unlike `saveAsTable`, `insertInto` ignores the column names and just uses position-based - * resolution. For example: - * @note SaveMode.ErrorIfExists and SaveMode.Ignore behave as SaveMode.Append in `insertInto` as - * `insertInto` is not a table creating operation. + * @note + * Unlike `saveAsTable`, `insertInto` ignores the column names and just uses position-based + * resolution. For example: + * @note + * SaveMode.ErrorIfExists and SaveMode.Ignore behave as SaveMode.Append in `insertInto` as + * `insertInto` is not a table creating operation. * * {{{ * scala> Seq((1, 2)).toDF("i", "j").write.mode("overwrite").saveAsTable("t1") @@ -263,7 +255,7 @@ abstract class DataFrameWriter[T] { * +---+---+ * }}} * - * Because it inserts data to an existing table, format or options will be ignored. + * Because it inserts data to an existing table, format or options will be ignored. * @since 1.4.0 */ def insertInto(tableName: String): Unit @@ -271,15 +263,15 @@ abstract class DataFrameWriter[T] { /** * Saves the content of the `DataFrame` as the specified table. * - * In the case the table already exists, behavior of this function depends on the - * save mode, specified by the `mode` function (default to throwing an exception). - * When `mode` is `Overwrite`, the schema of the `DataFrame` does not need to be - * the same as that of the existing table. + * In the case the table already exists, behavior of this function depends on the save mode, + * specified by the `mode` function (default to throwing an exception). When `mode` is + * `Overwrite`, the schema of the `DataFrame` does not need to be the same as that of the + * existing table. * * When `mode` is `Append`, if there is an existing table, we will use the format and options of * the existing table. The column order in the schema of the `DataFrame` doesn't need to be same - * as that of the existing table. Unlike `insertInto`, `saveAsTable` will use the column names to - * find the correct column positions. For example: + * as that of the existing table. Unlike `insertInto`, `saveAsTable` will use the column names + * to find the correct column positions. For example: * * {{{ * scala> Seq((1, 2)).toDF("i", "j").write.mode("overwrite").saveAsTable("t1") @@ -293,10 +285,10 @@ abstract class DataFrameWriter[T] { * +---+---+ * }}} * - * In this method, save mode is used to determine the behavior if the data source table exists in - * Spark catalog. We will always overwrite the underlying data of data source (e.g. a table in - * JDBC data source) if the table doesn't exist in Spark catalog, and will always append to the - * underlying data of data source if the table already exists. + * In this method, save mode is used to determine the behavior if the data source table exists + * in Spark catalog. We will always overwrite the underlying data of data source (e.g. a table + * in JDBC data source) if the table doesn't exist in Spark catalog, and will always append to + * the underlying data of data source if the table already exists. * * When the DataFrame is created from a non-partitioned `HadoopFsRelation` with a single input * path, and the data source provider can be mapped to an existing Hive builtin SerDe (i.e. ORC @@ -310,25 +302,25 @@ abstract class DataFrameWriter[T] { /** * Saves the content of the `DataFrame` to an external database table via JDBC. In the case the - * table already exists in the external database, behavior of this function depends on the - * save mode, specified by the `mode` function (default to throwing an exception). + * table already exists in the external database, behavior of this function depends on the save + * mode, specified by the `mode` function (default to throwing an exception). * * Don't create too many partitions in parallel on a large cluster; otherwise Spark might crash * your external database systems. * - * JDBC-specific option and parameter documentation for storing tables via JDBC in - * - * Data Source Option in the version you use. - * - * @param table Name of the table in the external database. - * @param connectionProperties JDBC database connection arguments, a list of arbitrary string - * tag/value. Normally at least a "user" and "password" property - * should be included. "batchsize" can be used to control the - * number of rows per insert. "isolationLevel" can be one of - * "NONE", "READ_COMMITTED", "READ_UNCOMMITTED", "REPEATABLE_READ", - * or "SERIALIZABLE", corresponding to standard transaction - * isolation levels defined by JDBC's Connection object, with default - * of "READ_UNCOMMITTED". + * JDBC-specific option and parameter documentation for storing tables via JDBC in + * Data Source Option in the version you use. + * + * @param table + * Name of the table in the external database. + * @param connectionProperties + * JDBC database connection arguments, a list of arbitrary string tag/value. Normally at least + * a "user" and "password" property should be included. "batchsize" can be used to control the + * number of rows per insert. "isolationLevel" can be one of "NONE", "READ_COMMITTED", + * "READ_UNCOMMITTED", "REPEATABLE_READ", or "SERIALIZABLE", corresponding to standard + * transaction isolation levels defined by JDBC's Connection object, with default of + * "READ_UNCOMMITTED". * @since 1.4.0 */ def jdbc(url: String, table: String, connectionProperties: util.Properties): Unit = { @@ -343,16 +335,16 @@ abstract class DataFrameWriter[T] { } /** - * Saves the content of the `DataFrame` in JSON format ( - * JSON Lines text format or newline-delimited JSON) at the specified path. - * This is equivalent to: + * Saves the content of the `DataFrame` in JSON format ( JSON + * Lines text format or newline-delimited JSON) at the specified path. This is equivalent + * to: * {{{ * format("json").save(path) * }}} * - * You can find the JSON-specific options for writing JSON files in - * - * Data Source Option in the version you use. + * You can find the JSON-specific options for writing JSON files in + * Data Source Option in the version you use. * * @since 1.4.0 */ @@ -361,16 +353,15 @@ abstract class DataFrameWriter[T] { } /** - * Saves the content of the `DataFrame` in Parquet format at the specified path. - * This is equivalent to: + * Saves the content of the `DataFrame` in Parquet format at the specified path. This is + * equivalent to: * {{{ * format("parquet").save(path) * }}} * - * Parquet-specific option(s) for writing Parquet files can be found in - * - * Data Source Option in the version you use. + * Parquet-specific option(s) for writing Parquet files can be found in Data + * Source Option in the version you use. * * @since 1.4.0 */ @@ -379,16 +370,15 @@ abstract class DataFrameWriter[T] { } /** - * Saves the content of the `DataFrame` in ORC format at the specified path. - * This is equivalent to: + * Saves the content of the `DataFrame` in ORC format at the specified path. This is equivalent + * to: * {{{ * format("orc").save(path) * }}} * - * ORC-specific option(s) for writing ORC files can be found in - * - * Data Source Option in the version you use. + * ORC-specific option(s) for writing ORC files can be found in Data + * Source Option in the version you use. * * @since 1.5.0 */ @@ -397,9 +387,9 @@ abstract class DataFrameWriter[T] { } /** - * Saves the content of the `DataFrame` in a text file at the specified path. - * The DataFrame must have only one column that is of string type. - * Each row becomes a new line in the output file. For example: + * Saves the content of the `DataFrame` in a text file at the specified path. The DataFrame must + * have only one column that is of string type. Each row becomes a new line in the output file. + * For example: * {{{ * // Scala: * df.write.text("/path/to/output") @@ -409,9 +399,9 @@ abstract class DataFrameWriter[T] { * }}} * The text files will be encoded as UTF-8. * - * You can find the text-specific options for writing text files in - * - * Data Source Option in the version you use. + * You can find the text-specific options for writing text files in + * Data Source Option in the version you use. * * @since 1.6.0 */ @@ -420,15 +410,15 @@ abstract class DataFrameWriter[T] { } /** - * Saves the content of the `DataFrame` in CSV format at the specified path. - * This is equivalent to: + * Saves the content of the `DataFrame` in CSV format at the specified path. This is equivalent + * to: * {{{ * format("csv").save(path) * }}} * - * You can find the CSV-specific options for writing CSV files in - * - * Data Source Option in the version you use. + * You can find the CSV-specific options for writing CSV files in + * Data Source Option in the version you use. * * @since 2.0.0 */ @@ -437,31 +427,25 @@ abstract class DataFrameWriter[T] { } /** - * Saves the content of the `DataFrame` in XML format at the specified path. - * This is equivalent to: + * Saves the content of the `DataFrame` in XML format at the specified path. This is equivalent + * to: * {{{ * format("xml").save(path) * }}} * - * Note that writing a XML file from `DataFrame` having a field `ArrayType` with - * its element as `ArrayType` would have an additional nested field for the element. - * For example, the `DataFrame` having a field below, + * Note that writing a XML file from `DataFrame` having a field `ArrayType` with its element as + * `ArrayType` would have an additional nested field for the element. For example, the + * `DataFrame` having a field below, * - * {@code fieldA [[data1], [data2]]} + * {@code fieldA [[data1], [data2]]} * - * would produce a XML file below. - * {@code - * - * data1 - * - * - * data2 - * } + * would produce a XML file below. {@code data1 + * data2 } * * Namely, roundtrip in writing and reading can end up in different schema structure. * - * You can find the XML-specific options for writing XML files in - * + * You can find the XML-specific options for writing XML files in * Data Source Option in the version you use. */ def xml(path: String): Unit = { diff --git a/sql/api/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala b/sql/api/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala index ddc89178cd835..37a29c2e4b66d 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala @@ -22,13 +22,14 @@ import org.apache.spark.annotation.Experimental import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NoSuchTableException, TableAlreadyExistsException} /** - * Interface used to write a [[org.apache.spark.sql.api.Dataset]] to external storage - * using the v2 API. + * Interface used to write a [[org.apache.spark.sql.api.Dataset]] to external storage using the v2 + * API. * * @since 3.0.0 */ @Experimental abstract class DataFrameWriterV2[T] extends CreateTableWriter[T] { + /** @inheritdoc */ override def using(provider: String): this.type @@ -68,20 +69,22 @@ abstract class DataFrameWriterV2[T] extends CreateTableWriter[T] { * [[org.apache.spark.sql.catalyst.analysis.NoSuchTableException]]. The data frame will be * validated to ensure it is compatible with the existing table. * - * @throws org.apache.spark.sql.catalyst.analysis.NoSuchTableException If the table does not exist + * @throws org.apache.spark.sql.catalyst.analysis.NoSuchTableException + * If the table does not exist */ @throws(classOf[NoSuchTableException]) def append(): Unit /** - * Overwrite rows matching the given filter condition with the contents of the data frame in - * the output table. + * Overwrite rows matching the given filter condition with the contents of the data frame in the + * output table. * * If the output table does not exist, this operation will fail with - * [[org.apache.spark.sql.catalyst.analysis.NoSuchTableException]]. - * The data frame will be validated to ensure it is compatible with the existing table. + * [[org.apache.spark.sql.catalyst.analysis.NoSuchTableException]]. The data frame will be + * validated to ensure it is compatible with the existing table. * - * @throws org.apache.spark.sql.catalyst.analysis.NoSuchTableException If the table does not exist + * @throws org.apache.spark.sql.catalyst.analysis.NoSuchTableException + * If the table does not exist */ @throws(classOf[NoSuchTableException]) def overwrite(condition: Column): Unit @@ -97,7 +100,8 @@ abstract class DataFrameWriterV2[T] extends CreateTableWriter[T] { * [[org.apache.spark.sql.catalyst.analysis.NoSuchTableException]]. The data frame will be * validated to ensure it is compatible with the existing table. * - * @throws org.apache.spark.sql.catalyst.analysis.NoSuchTableException If the table does not exist + * @throws org.apache.spark.sql.catalyst.analysis.NoSuchTableException + * If the table does not exist */ @throws(classOf[NoSuchTableException]) def overwritePartitions(): Unit @@ -105,10 +109,12 @@ abstract class DataFrameWriterV2[T] extends CreateTableWriter[T] { /** * Configuration methods common to create/replace operations and insert/overwrite operations. - * @tparam R builder type to return + * @tparam R + * builder type to return * @since 3.0.0 */ trait WriteConfigMethods[R] { + /** * Add a write option. * @@ -158,17 +164,18 @@ trait WriteConfigMethods[R] { * @since 3.0.0 */ trait CreateTableWriter[T] extends WriteConfigMethods[CreateTableWriter[T]] { + /** * Create a new table from the contents of the data frame. * - * The new table's schema, partition layout, properties, and other configuration will be - * based on the configuration set on this writer. + * The new table's schema, partition layout, properties, and other configuration will be based + * on the configuration set on this writer. * * If the output table exists, this operation will fail with * [[org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException]]. * * @throws org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException - * If the table already exists + * If the table already exists */ @throws(classOf[TableAlreadyExistsException]) def create(): Unit @@ -183,7 +190,7 @@ trait CreateTableWriter[T] extends WriteConfigMethods[CreateTableWriter[T]] { * [[org.apache.spark.sql.catalyst.analysis.CannotReplaceMissingTableException]]. * * @throws org.apache.spark.sql.catalyst.analysis.CannotReplaceMissingTableException - * If the table does not exist + * If the table does not exist */ @throws(classOf[CannotReplaceMissingTableException]) def replace(): Unit @@ -191,28 +198,25 @@ trait CreateTableWriter[T] extends WriteConfigMethods[CreateTableWriter[T]] { /** * Create a new table or replace an existing table with the contents of the data frame. * - * The output table's schema, partition layout, properties, and other configuration will be based - * on the contents of the data frame and the configuration set on this writer. If the table - * exists, its configuration and data will be replaced. + * The output table's schema, partition layout, properties, and other configuration will be + * based on the contents of the data frame and the configuration set on this writer. If the + * table exists, its configuration and data will be replaced. */ def createOrReplace(): Unit /** - * Partition the output table created by `create`, `createOrReplace`, or `replace` using - * the given columns or transforms. + * Partition the output table created by `create`, `createOrReplace`, or `replace` using the + * given columns or transforms. * * When specified, the table data will be stored by these values for efficient reads. * * For example, when a table is partitioned by day, it may be stored in a directory layout like: - *
    - *
  • `table/day=2019-06-01/`
  • - *
  • `table/day=2019-06-02/`
  • - *
+ *
  • `table/day=2019-06-01/`
  • `table/day=2019-06-02/`
* - * Partitioning is one of the most widely used techniques to optimize physical data layout. - * It provides a coarse-grained index for skipping unnecessary data reads when queries have - * predicates on the partitioned columns. In order for partitioning to work well, the number - * of distinct values in each column should typically be less than tens of thousands. + * Partitioning is one of the most widely used techniques to optimize physical data layout. It + * provides a coarse-grained index for skipping unnecessary data reads when queries have + * predicates on the partitioned columns. In order for partitioning to work well, the number of + * distinct values in each column should typically be less than tens of thousands. * * @since 3.0.0 */ @@ -220,8 +224,8 @@ trait CreateTableWriter[T] extends WriteConfigMethods[CreateTableWriter[T]] { def partitionedBy(column: Column, columns: Column*): CreateTableWriter[T] /** - * Clusters the output by the given columns on the storage. The rows with matching values in - * the specified clustering columns will be consolidated within the same group. + * Clusters the output by the given columns on the storage. The rows with matching values in the + * specified clustering columns will be consolidated within the same group. * * For instance, if you cluster a dataset by date, the data sharing the same date will be stored * together in a file. This arrangement improves query efficiency when you apply selective diff --git a/sql/api/src/main/scala/org/apache/spark/sql/Encoder.scala b/sql/api/src/main/scala/org/apache/spark/sql/Encoder.scala index ea760d80541c8..d125e89b8c410 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/Encoder.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/Encoder.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.types._ /** * Used to convert a JVM object of type `T` to and from the internal Spark SQL representation. * - * == Scala == + * ==Scala== * Encoders are generally created automatically through implicits from a `SparkSession`, or can be * explicitly created by calling static methods on [[Encoders]]. * @@ -35,7 +35,7 @@ import org.apache.spark.sql.types._ * val ds = Seq(1, 2, 3).toDS() // implicitly provided (spark.implicits.newIntEncoder) * }}} * - * == Java == + * ==Java== * Encoders are specified by calling static methods on [[Encoders]]. * * {{{ @@ -57,8 +57,8 @@ import org.apache.spark.sql.types._ * Encoders.bean(MyClass.class); * }}} * - * == Implementation == - * - Encoders should be thread-safe. + * ==Implementation== + * - Encoders should be thread-safe. * * @since 1.6.0 */ diff --git a/sql/api/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala b/sql/api/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala index dabd900917e3b..db56b39e28aeb 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala @@ -19,12 +19,13 @@ package org.apache.spark.sql import org.apache.spark.annotation.Experimental /** - * `MergeIntoWriter` provides methods to define and execute merge actions based - * on specified conditions. + * `MergeIntoWriter` provides methods to define and execute merge actions based on specified + * conditions. * * Please note that schema evolution is disabled by default. * - * @tparam T the type of data in the Dataset. + * @tparam T + * the type of data in the Dataset. * @since 4.0.0 */ @Experimental @@ -41,11 +42,12 @@ abstract class MergeIntoWriter[T] { * * This `WhenMatched` can be followed by one of the following merge actions: * - `updateAll`: Update all the matched target table rows with source dataset rows. - * - `update(Map)`: Update all the matched target table rows while changing only - * a subset of columns based on the provided assignment. + * - `update(Map)`: Update all the matched target table rows while changing only a subset of + * columns based on the provided assignment. * - `delete`: Delete all target rows that have a match in the source table. * - * @return a new `WhenMatched` object. + * @return + * a new `WhenMatched` object. */ def whenMatched(): WhenMatched[T] = { new WhenMatched[T](this, None) @@ -59,12 +61,14 @@ abstract class MergeIntoWriter[T] { * * This `WhenMatched` can be followed by one of the following merge actions: * - `updateAll`: Update all the matched target table rows with source dataset rows. - * - `update(Map)`: Update all the matched target table rows while changing only - * a subset of columns based on the provided assignment. + * - `update(Map)`: Update all the matched target table rows while changing only a subset of + * columns based on the provided assignment. * - `delete`: Delete all target rows that have a match in the source table. * - * @param condition a `Column` representing the condition to be evaluated for the action. - * @return a new `WhenMatched` object configured with the specified condition. + * @param condition + * a `Column` representing the condition to be evaluated for the action. + * @return + * a new `WhenMatched` object configured with the specified condition. */ def whenMatched(condition: Column): WhenMatched[T] = { new WhenMatched[T](this, Some(condition)) @@ -79,9 +83,10 @@ abstract class MergeIntoWriter[T] { * This `WhenNotMatched` can be followed by one of the following merge actions: * - `insertAll`: Insert all rows from the source that are not already in the target table. * - `insert(Map)`: Insert all rows from the source that are not already in the target table, - * with the specified columns based on the provided assignment. + * with the specified columns based on the provided assignment. * - * @return a new `WhenNotMatched` object. + * @return + * a new `WhenNotMatched` object. */ def whenNotMatched(): WhenNotMatched[T] = { new WhenNotMatched[T](this, None) @@ -98,8 +103,10 @@ abstract class MergeIntoWriter[T] { * - `insert(Map)`: Insert all rows from the source that are not already in the target table, * with the specified columns based on the provided assignment. * - * @param condition a `Column` representing the condition to be evaluated for the action. - * @return a new `WhenNotMatched` object configured with the specified condition. + * @param condition + * a `Column` representing the condition to be evaluated for the action. + * @return + * a new `WhenNotMatched` object configured with the specified condition. */ def whenNotMatched(condition: Column): WhenNotMatched[T] = { new WhenNotMatched[T](this, Some(condition)) @@ -113,11 +120,12 @@ abstract class MergeIntoWriter[T] { * * This `WhenNotMatchedBySource` can be followed by one of the following merge actions: * - `updateAll`: Update all the not matched target table rows with source dataset rows. - * - `update(Map)`: Update all the not matched target table rows while changing only - * the specified columns based on the provided assignment. + * - `update(Map)`: Update all the not matched target table rows while changing only the + * specified columns based on the provided assignment. * - `delete`: Delete all target rows that have no matches in the source table. * - * @return a new `WhenNotMatchedBySource` object. + * @return + * a new `WhenNotMatchedBySource` object. */ def whenNotMatchedBySource(): WhenNotMatchedBySource[T] = { new WhenNotMatchedBySource[T](this, None) @@ -127,17 +135,19 @@ abstract class MergeIntoWriter[T] { * Initialize a `WhenNotMatchedBySource` action with a condition. * * This `WhenNotMatchedBySource` action will be executed when a target row does not match any - * rows in the source table based on the merge condition and the specified `condition` - * is satisfied. + * rows in the source table based on the merge condition and the specified `condition` is + * satisfied. * * This `WhenNotMatchedBySource` can be followed by one of the following merge actions: * - `updateAll`: Update all the not matched target table rows with source dataset rows. - * - `update(Map)`: Update all the not matched target table rows while changing only - * the specified columns based on the provided assignment. + * - `update(Map)`: Update all the not matched target table rows while changing only the + * specified columns based on the provided assignment. * - `delete`: Delete all target rows that have no matches in the source table. * - * @param condition a `Column` representing the condition to be evaluated for the action. - * @return a new `WhenNotMatchedBySource` object configured with the specified condition. + * @param condition + * a `Column` representing the condition to be evaluated for the action. + * @return + * a new `WhenNotMatchedBySource` object configured with the specified condition. */ def whenNotMatchedBySource(condition: Column): WhenNotMatchedBySource[T] = { new WhenNotMatchedBySource[T](this, Some(condition)) @@ -146,7 +156,8 @@ abstract class MergeIntoWriter[T] { /** * Enable automatic schema evolution for this merge operation. * - * @return A `MergeIntoWriter` instance with schema evolution enabled. + * @return + * A `MergeIntoWriter` instance with schema evolution enabled. */ def withSchemaEvolution(): MergeIntoWriter[T] = { schemaEvolution = true @@ -180,25 +191,26 @@ abstract class MergeIntoWriter[T] { } /** - * A class for defining actions to be taken when matching rows in a DataFrame during - * a merge operation. + * A class for defining actions to be taken when matching rows in a DataFrame during a merge + * operation. * - * @param mergeIntoWriter The MergeIntoWriter instance responsible for writing data to a - * target DataFrame. - * @param condition An optional condition Expression that specifies when the actions - * should be applied. - * If the condition is None, the actions will be applied to all matched - * rows. - * @tparam T The type of data in the MergeIntoWriter. + * @param mergeIntoWriter + * The MergeIntoWriter instance responsible for writing data to a target DataFrame. + * @param condition + * An optional condition Expression that specifies when the actions should be applied. If the + * condition is None, the actions will be applied to all matched rows. + * @tparam T + * The type of data in the MergeIntoWriter. */ -case class WhenMatched[T] private[sql]( +case class WhenMatched[T] private[sql] ( mergeIntoWriter: MergeIntoWriter[T], condition: Option[Column]) { /** * Specifies an action to update all matched rows in the DataFrame. * - * @return The MergeIntoWriter instance with the update all action configured. + * @return + * The MergeIntoWriter instance with the update all action configured. */ def updateAll(): MergeIntoWriter[T] = mergeIntoWriter.updateAll(condition, notMatchedBySource = false) @@ -207,8 +219,10 @@ case class WhenMatched[T] private[sql]( * Specifies an action to update matched rows in the DataFrame with the provided column * assignments. * - * @param map A Map of column names to Column expressions representing the updates to be applied. - * @return The MergeIntoWriter instance with the update action configured. + * @param map + * A Map of column names to Column expressions representing the updates to be applied. + * @return + * The MergeIntoWriter instance with the update action configured. */ def update(map: Map[String, Column]): MergeIntoWriter[T] = mergeIntoWriter.update(condition, map, notMatchedBySource = false) @@ -216,65 +230,73 @@ case class WhenMatched[T] private[sql]( /** * Specifies an action to delete matched rows from the DataFrame. * - * @return The MergeIntoWriter instance with the delete action configured. + * @return + * The MergeIntoWriter instance with the delete action configured. */ def delete(): MergeIntoWriter[T] = mergeIntoWriter.delete(condition, notMatchedBySource = false) } /** - * A class for defining actions to be taken when no matching rows are found in a DataFrame - * during a merge operation. + * A class for defining actions to be taken when no matching rows are found in a DataFrame during + * a merge operation. * - * @param mergeIntoWriter The MergeIntoWriter instance responsible for writing data to a - * target DataFrame. - * @param condition An optional condition Expression that specifies when the actions - * defined in this configuration should be applied. - * If the condition is None, the actions will be applied when there - * are no matching rows. - * @tparam T The type of data in the MergeIntoWriter. + * @param mergeIntoWriter + * The MergeIntoWriter instance responsible for writing data to a target DataFrame. + * @param condition + * An optional condition Expression that specifies when the actions defined in this + * configuration should be applied. If the condition is None, the actions will be applied when + * there are no matching rows. + * @tparam T + * The type of data in the MergeIntoWriter. */ -case class WhenNotMatched[T] private[sql]( +case class WhenNotMatched[T] private[sql] ( mergeIntoWriter: MergeIntoWriter[T], condition: Option[Column]) { /** * Specifies an action to insert all non-matched rows into the DataFrame. * - * @return The MergeIntoWriter instance with the insert all action configured. + * @return + * The MergeIntoWriter instance with the insert all action configured. */ def insertAll(): MergeIntoWriter[T] = mergeIntoWriter.insertAll(condition) /** - * Specifies an action to insert non-matched rows into the DataFrame with the provided - * column assignments. + * Specifies an action to insert non-matched rows into the DataFrame with the provided column + * assignments. * - * @param map A Map of column names to Column expressions representing the values to be inserted. - * @return The MergeIntoWriter instance with the insert action configured. + * @param map + * A Map of column names to Column expressions representing the values to be inserted. + * @return + * The MergeIntoWriter instance with the insert action configured. */ def insert(map: Map[String, Column]): MergeIntoWriter[T] = mergeIntoWriter.insert(condition, map) } - /** - * A class for defining actions to be performed when there is no match by source - * during a merge operation in a MergeIntoWriter. + * A class for defining actions to be performed when there is no match by source during a merge + * operation in a MergeIntoWriter. * - * @param mergeIntoWriter the MergeIntoWriter instance to which the merge actions will be applied. - * @param condition an optional condition to be used with the merge actions. - * @tparam T the type parameter for the MergeIntoWriter. + * @param mergeIntoWriter + * the MergeIntoWriter instance to which the merge actions will be applied. + * @param condition + * an optional condition to be used with the merge actions. + * @tparam T + * the type parameter for the MergeIntoWriter. */ -case class WhenNotMatchedBySource[T] private[sql]( +case class WhenNotMatchedBySource[T] private[sql] ( mergeIntoWriter: MergeIntoWriter[T], condition: Option[Column]) { /** - * Specifies an action to update all non-matched rows in the target DataFrame when - * not matched by the source. + * Specifies an action to update all non-matched rows in the target DataFrame when not matched + * by the source. * - * @return The MergeIntoWriter instance with the update all action configured. + * @return + * The MergeIntoWriter instance with the update all action configured. */ def updateAll(): MergeIntoWriter[T] = mergeIntoWriter.updateAll(condition, notMatchedBySource = true) @@ -283,8 +305,10 @@ case class WhenNotMatchedBySource[T] private[sql]( * Specifies an action to update non-matched rows in the target DataFrame with the provided * column assignments when not matched by the source. * - * @param map A Map of column names to Column expressions representing the updates to be applied. - * @return The MergeIntoWriter instance with the update action configured. + * @param map + * A Map of column names to Column expressions representing the updates to be applied. + * @return + * The MergeIntoWriter instance with the update action configured. */ def update(map: Map[String, Column]): MergeIntoWriter[T] = mergeIntoWriter.update(condition, map, notMatchedBySource = true) @@ -293,7 +317,8 @@ case class WhenNotMatchedBySource[T] private[sql]( * Specifies an action to delete non-matched rows from the target DataFrame when not matched by * the source. * - * @return The MergeIntoWriter instance with the delete action configured. + * @return + * The MergeIntoWriter instance with the delete action configured. */ def delete(): MergeIntoWriter[T] = mergeIntoWriter.delete(condition, notMatchedBySource = true) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/Observation.scala b/sql/api/src/main/scala/org/apache/spark/sql/Observation.scala index 02f5a8de1e3f6..fa427fe651907 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/Observation.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/Observation.scala @@ -38,13 +38,14 @@ import org.apache.spark.util.SparkThreadUtils * val metrics = observation.get * }}} * - * This collects the metrics while the first action is executed on the observed dataset. Subsequent - * actions do not modify the metrics returned by [[get]]. Retrieval of the metric via [[get]] - * blocks until the first action has finished and metrics become available. + * This collects the metrics while the first action is executed on the observed dataset. + * Subsequent actions do not modify the metrics returned by [[get]]. Retrieval of the metric via + * [[get]] blocks until the first action has finished and metrics become available. * * This class does not support streaming datasets. * - * @param name name of the metric + * @param name + * name of the metric * @since 3.3.0 */ class Observation(val name: String) { @@ -65,23 +66,27 @@ class Observation(val name: String) { val future: Future[Map[String, Any]] = promise.future /** - * (Scala-specific) Get the observed metrics. This waits for the observed dataset to finish - * its first action. Only the result of the first action is available. Subsequent actions do not + * (Scala-specific) Get the observed metrics. This waits for the observed dataset to finish its + * first action. Only the result of the first action is available. Subsequent actions do not * modify the result. * - * @return the observed metrics as a `Map[String, Any]` - * @throws InterruptedException interrupted while waiting + * @return + * the observed metrics as a `Map[String, Any]` + * @throws InterruptedException + * interrupted while waiting */ @throws[InterruptedException] def get: Map[String, Any] = SparkThreadUtils.awaitResult(future, Duration.Inf) /** - * (Java-specific) Get the observed metrics. This waits for the observed dataset to finish - * its first action. Only the result of the first action is available. Subsequent actions do not + * (Java-specific) Get the observed metrics. This waits for the observed dataset to finish its + * first action. Only the result of the first action is available. Subsequent actions do not * modify the result. * - * @return the observed metrics as a `java.util.Map[String, Object]` - * @throws InterruptedException interrupted while waiting + * @return + * the observed metrics as a `java.util.Map[String, Object]` + * @throws InterruptedException + * interrupted while waiting */ @throws[InterruptedException] def getAsJava: java.util.Map[String, Any] = get.asJava @@ -89,7 +94,8 @@ class Observation(val name: String) { /** * Get the observed metrics. This returns the metrics if they are available, otherwise an empty. * - * @return the observed metrics as a `Map[String, Any]` + * @return + * the observed metrics as a `Map[String, Any]` */ @throws[InterruptedException] private[sql] def getOrEmpty: Map[String, Any] = { @@ -108,7 +114,8 @@ class Observation(val name: String) { /** * Set the observed metrics and notify all waiting threads to resume. * - * @return `true` if all waiting threads were notified, `false` if otherwise. + * @return + * `true` if all waiting threads were notified, `false` if otherwise. */ private[sql] def setMetricsAndNotify(metrics: Row): Boolean = { val metricsMap = metrics.getValuesMap(metrics.schema.map(_.name)) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/Row.scala b/sql/api/src/main/scala/org/apache/spark/sql/Row.scala index fb4b4a6f37c8d..aa14115453aea 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/Row.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/Row.scala @@ -45,6 +45,7 @@ import org.apache.spark.util.ArrayImplicits._ */ @Stable object Row { + /** * This method can be used to extract fields from a [[Row]] object in a pattern match. Example: * {{{ @@ -83,9 +84,8 @@ object Row { val empty = apply() } - /** - * Represents one row of output from a relational operator. Allows both generic access by ordinal, + * Represents one row of output from a relational operator. Allows both generic access by ordinal, * which will incur boxing overhead for primitives, as well as native primitive access. * * It is invalid to use the native primitive interface to retrieve a value that is null, instead a @@ -103,9 +103,9 @@ object Row { * Row.fromSeq(Seq(value1, value2, ...)) * }}} * - * A value of a row can be accessed through both generic access by ordinal, - * which will incur boxing overhead for primitives, as well as native primitive access. - * An example of generic access by ordinal: + * A value of a row can be accessed through both generic access by ordinal, which will incur + * boxing overhead for primitives, as well as native primitive access. An example of generic + * access by ordinal: * {{{ * import org.apache.spark.sql._ * @@ -117,10 +117,9 @@ object Row { * // fourthValue: Any = null * }}} * - * For native primitive access, it is invalid to use the native primitive interface to retrieve - * a value that is null, instead a user must check `isNullAt` before attempting to retrieve a - * value that might be null. - * An example of native primitive access: + * For native primitive access, it is invalid to use the native primitive interface to retrieve a + * value that is null, instead a user must check `isNullAt` before attempting to retrieve a value + * that might be null. An example of native primitive access: * {{{ * // using the row from the previous example. * val firstValue = row.getInt(0) @@ -143,6 +142,7 @@ object Row { */ @Stable trait Row extends Serializable { + /** Number of elements in the Row. */ def size: Int = length @@ -155,8 +155,8 @@ trait Row extends Serializable { def schema: StructType = null /** - * Returns the value at position i. If the value is null, null is returned. The following - * is a mapping between Spark SQL types and return types: + * Returns the value at position i. If the value is null, null is returned. The following is a + * mapping between Spark SQL types and return types: * * {{{ * BooleanType -> java.lang.Boolean @@ -184,8 +184,8 @@ trait Row extends Serializable { def apply(i: Int): Any = get(i) /** - * Returns the value at position i. If the value is null, null is returned. The following - * is a mapping between Spark SQL types and return types: + * Returns the value at position i. If the value is null, null is returned. The following is a + * mapping between Spark SQL types and return types: * * {{{ * BooleanType -> java.lang.Boolean @@ -218,106 +218,127 @@ trait Row extends Serializable { /** * Returns the value at position i as a primitive boolean. * - * @throws ClassCastException when data type does not match. - * @throws org.apache.spark.SparkRuntimeException when value is null. + * @throws ClassCastException + * when data type does not match. + * @throws org.apache.spark.SparkRuntimeException + * when value is null. */ def getBoolean(i: Int): Boolean = getAnyValAs[Boolean](i) /** * Returns the value at position i as a primitive byte. * - * @throws ClassCastException when data type does not match. - * @throws org.apache.spark.SparkRuntimeException when value is null. + * @throws ClassCastException + * when data type does not match. + * @throws org.apache.spark.SparkRuntimeException + * when value is null. */ def getByte(i: Int): Byte = getAnyValAs[Byte](i) /** * Returns the value at position i as a primitive short. * - * @throws ClassCastException when data type does not match. - * @throws org.apache.spark.SparkRuntimeException when value is null. + * @throws ClassCastException + * when data type does not match. + * @throws org.apache.spark.SparkRuntimeException + * when value is null. */ def getShort(i: Int): Short = getAnyValAs[Short](i) /** * Returns the value at position i as a primitive int. * - * @throws ClassCastException when data type does not match. - * @throws org.apache.spark.SparkRuntimeException when value is null. + * @throws ClassCastException + * when data type does not match. + * @throws org.apache.spark.SparkRuntimeException + * when value is null. */ def getInt(i: Int): Int = getAnyValAs[Int](i) /** * Returns the value at position i as a primitive long. * - * @throws ClassCastException when data type does not match. - * @throws org.apache.spark.SparkRuntimeException when value is null. + * @throws ClassCastException + * when data type does not match. + * @throws org.apache.spark.SparkRuntimeException + * when value is null. */ def getLong(i: Int): Long = getAnyValAs[Long](i) /** - * Returns the value at position i as a primitive float. - * Throws an exception if the type mismatches or if the value is null. + * Returns the value at position i as a primitive float. Throws an exception if the type + * mismatches or if the value is null. * - * @throws ClassCastException when data type does not match. - * @throws org.apache.spark.SparkRuntimeException when value is null. + * @throws ClassCastException + * when data type does not match. + * @throws org.apache.spark.SparkRuntimeException + * when value is null. */ def getFloat(i: Int): Float = getAnyValAs[Float](i) /** * Returns the value at position i as a primitive double. * - * @throws ClassCastException when data type does not match. - * @throws org.apache.spark.SparkRuntimeException when value is null. + * @throws ClassCastException + * when data type does not match. + * @throws org.apache.spark.SparkRuntimeException + * when value is null. */ def getDouble(i: Int): Double = getAnyValAs[Double](i) /** * Returns the value at position i as a String object. * - * @throws ClassCastException when data type does not match. + * @throws ClassCastException + * when data type does not match. */ def getString(i: Int): String = getAs[String](i) /** * Returns the value at position i of decimal type as java.math.BigDecimal. * - * @throws ClassCastException when data type does not match. + * @throws ClassCastException + * when data type does not match. */ def getDecimal(i: Int): java.math.BigDecimal = getAs[java.math.BigDecimal](i) /** * Returns the value at position i of date type as java.sql.Date. * - * @throws ClassCastException when data type does not match. + * @throws ClassCastException + * when data type does not match. */ def getDate(i: Int): java.sql.Date = getAs[java.sql.Date](i) /** * Returns the value at position i of date type as java.time.LocalDate. * - * @throws ClassCastException when data type does not match. + * @throws ClassCastException + * when data type does not match. */ def getLocalDate(i: Int): java.time.LocalDate = getAs[java.time.LocalDate](i) /** * Returns the value at position i of date type as java.sql.Timestamp. * - * @throws ClassCastException when data type does not match. + * @throws ClassCastException + * when data type does not match. */ def getTimestamp(i: Int): java.sql.Timestamp = getAs[java.sql.Timestamp](i) /** * Returns the value at position i of date type as java.time.Instant. * - * @throws ClassCastException when data type does not match. + * @throws ClassCastException + * when data type does not match. */ def getInstant(i: Int): java.time.Instant = getAs[java.time.Instant](i) /** * Returns the value at position i of array type as a Scala Seq. * - * @throws ClassCastException when data type does not match. + * @throws ClassCastException + * when data type does not match. */ def getSeq[T](i: Int): Seq[T] = { getAs[scala.collection.Seq[T]](i) match { @@ -334,7 +355,8 @@ trait Row extends Serializable { /** * Returns the value at position i of array type as `java.util.List`. * - * @throws ClassCastException when data type does not match. + * @throws ClassCastException + * when data type does not match. */ def getList[T](i: Int): java.util.List[T] = getSeq[T](i).asJava @@ -342,14 +364,16 @@ trait Row extends Serializable { /** * Returns the value at position i of map type as a Scala Map. * - * @throws ClassCastException when data type does not match. + * @throws ClassCastException + * when data type does not match. */ def getMap[K, V](i: Int): scala.collection.Map[K, V] = getAs[Map[K, V]](i) /** * Returns the value at position i of array type as a `java.util.Map`. * - * @throws ClassCastException when data type does not match. + * @throws ClassCastException + * when data type does not match. */ def getJavaMap[K, V](i: Int): java.util.Map[K, V] = getMap[K, V](i).asJava @@ -357,48 +381,56 @@ trait Row extends Serializable { /** * Returns the value at position i of struct type as a [[Row]] object. * - * @throws ClassCastException when data type does not match. + * @throws ClassCastException + * when data type does not match. */ def getStruct(i: Int): Row = getAs[Row](i) /** - * Returns the value at position i. - * For primitive types if value is null it returns 'zero value' specific for primitive - * i.e. 0 for Int - use isNullAt to ensure that value is not null + * Returns the value at position i. For primitive types if value is null it returns 'zero value' + * specific for primitive i.e. 0 for Int - use isNullAt to ensure that value is not null * - * @throws ClassCastException when data type does not match. + * @throws ClassCastException + * when data type does not match. */ def getAs[T](i: Int): T = get(i).asInstanceOf[T] /** - * Returns the value of a given fieldName. - * For primitive types if value is null it returns 'zero value' specific for primitive - * i.e. 0 for Int - use isNullAt to ensure that value is not null + * Returns the value of a given fieldName. For primitive types if value is null it returns 'zero + * value' specific for primitive i.e. 0 for Int - use isNullAt to ensure that value is not null * - * @throws UnsupportedOperationException when schema is not defined. - * @throws IllegalArgumentException when fieldName do not exist. - * @throws ClassCastException when data type does not match. + * @throws UnsupportedOperationException + * when schema is not defined. + * @throws IllegalArgumentException + * when fieldName do not exist. + * @throws ClassCastException + * when data type does not match. */ def getAs[T](fieldName: String): T = getAs[T](fieldIndex(fieldName)) /** * Returns the index of a given field name. * - * @throws UnsupportedOperationException when schema is not defined. - * @throws IllegalArgumentException when a field `name` does not exist. + * @throws UnsupportedOperationException + * when schema is not defined. + * @throws IllegalArgumentException + * when a field `name` does not exist. */ def fieldIndex(name: String): Int = { throw DataTypeErrors.fieldIndexOnRowWithoutSchemaError(fieldName = name) } /** - * Returns a Map consisting of names and values for the requested fieldNames - * For primitive types if value is null it returns 'zero value' specific for primitive - * i.e. 0 for Int - use isNullAt to ensure that value is not null + * Returns a Map consisting of names and values for the requested fieldNames For primitive types + * if value is null it returns 'zero value' specific for primitive i.e. 0 for Int - use isNullAt + * to ensure that value is not null * - * @throws UnsupportedOperationException when schema is not defined. - * @throws IllegalArgumentException when fieldName do not exist. - * @throws ClassCastException when data type does not match. + * @throws UnsupportedOperationException + * when schema is not defined. + * @throws IllegalArgumentException + * when fieldName do not exist. + * @throws ClassCastException + * when data type does not match. */ def getValuesMap[T](fieldNames: Seq[String]): Map[String, T] = { fieldNames.map { name => @@ -445,24 +477,25 @@ trait Row extends Serializable { o1 match { case b1: Array[Byte] => if (!o2.isInstanceOf[Array[Byte]] || - !java.util.Arrays.equals(b1, o2.asInstanceOf[Array[Byte]])) { + !java.util.Arrays.equals(b1, o2.asInstanceOf[Array[Byte]])) { return false } case f1: Float if java.lang.Float.isNaN(f1) => - if (!o2.isInstanceOf[Float] || ! java.lang.Float.isNaN(o2.asInstanceOf[Float])) { + if (!o2.isInstanceOf[Float] || !java.lang.Float.isNaN(o2.asInstanceOf[Float])) { return false } case d1: Double if java.lang.Double.isNaN(d1) => - if (!o2.isInstanceOf[Double] || ! java.lang.Double.isNaN(o2.asInstanceOf[Double])) { + if (!o2.isInstanceOf[Double] || !java.lang.Double.isNaN(o2.asInstanceOf[Double])) { return false } case d1: java.math.BigDecimal if o2.isInstanceOf[java.math.BigDecimal] => if (d1.compareTo(o2.asInstanceOf[java.math.BigDecimal]) != 0) { return false } - case _ => if (o1 != o2) { - return false - } + case _ => + if (o1 != o2) { + return false + } } } i += 1 @@ -505,8 +538,8 @@ trait Row extends Serializable { def mkString(sep: String): String = mkString("", sep, "") /** - * Displays all elements of this traversable or iterator in a string using - * start, end, and separator strings. + * Displays all elements of this traversable or iterator in a string using start, end, and + * separator strings. */ def mkString(start: String, sep: String, end: String): String = { val n = length @@ -528,9 +561,12 @@ trait Row extends Serializable { /** * Returns the value at position i. * - * @throws UnsupportedOperationException when schema is not defined. - * @throws ClassCastException when data type does not match. - * @throws org.apache.spark.SparkRuntimeException when value is null. + * @throws UnsupportedOperationException + * when schema is not defined. + * @throws ClassCastException + * when data type does not match. + * @throws org.apache.spark.SparkRuntimeException + * when value is null. */ private def getAnyValAs[T <: AnyVal](i: Int): T = if (isNullAt(i)) throw DataTypeErrors.valueIsNullError(i) @@ -556,7 +592,8 @@ trait Row extends Serializable { * Note that this only supports the data types that are also supported by * [[org.apache.spark.sql.catalyst.encoders.RowEncoder]]. * - * @return the JSON representation of the row. + * @return + * the JSON representation of the row. */ private[sql] def jsonValue: JValue = { require(schema != null, "JSON serialization requires a non-null schema.") @@ -598,13 +635,12 @@ trait Row extends Serializable { case (s: Seq[_], ArrayType(elementType, _)) => iteratorToJsonArray(s.iterator, elementType) case (m: Map[String @unchecked, _], MapType(StringType, valueType, _)) => - new JObject(m.toList.sortBy(_._1).map { - case (k, v) => k -> toJson(v, valueType) + new JObject(m.toList.sortBy(_._1).map { case (k, v) => + k -> toJson(v, valueType) }) case (m: Map[_, _], MapType(keyType, valueType, _)) => - new JArray(m.iterator.map { - case (k, v) => - new JObject("key" -> toJson(k, keyType) :: "value" -> toJson(v, valueType) :: Nil) + new JArray(m.iterator.map { case (k, v) => + new JObject("key" -> toJson(k, keyType) :: "value" -> toJson(v, valueType) :: Nil) }.toList) case (row: Row, schema: StructType) => var n = 0 @@ -618,13 +654,13 @@ trait Row extends Serializable { new JObject(elements.toList) case (v: Any, udt: UserDefinedType[Any @unchecked]) => toJson(UDTUtils.toRow(v, udt), udt.sqlType) - case _ => throw new SparkIllegalArgumentException( - errorClass = "FAILED_ROW_TO_JSON", - messageParameters = Map( - "value" -> toSQLValue(value.toString), - "class" -> value.getClass.toString, - "sqlType" -> toSQLType(dataType.toString)) - ) + case _ => + throw new SparkIllegalArgumentException( + errorClass = "FAILED_ROW_TO_JSON", + messageParameters = Map( + "value" -> toSQLValue(value.toString), + "class" -> value.getClass.toString, + "sqlType" -> toSQLType(dataType.toString))) } toJson(this, schema) } diff --git a/sql/api/src/main/scala/org/apache/spark/sql/api/DataFrameNaFunctions.scala b/sql/api/src/main/scala/org/apache/spark/sql/api/DataFrameNaFunctions.scala index 7400f90992d8f..12d3d41aa5546 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/api/DataFrameNaFunctions.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/api/DataFrameNaFunctions.scala @@ -42,16 +42,16 @@ abstract class DataFrameNaFunctions[DS[U] <: Dataset[U, DS]] { /** * Returns a new `DataFrame` that drops rows containing null or NaN values. * - * If `how` is "any", then drop rows containing any null or NaN values. - * If `how` is "all", then drop rows only if every column is null or NaN for that row. + * If `how` is "any", then drop rows containing any null or NaN values. If `how` is "all", then + * drop rows only if every column is null or NaN for that row. * * @since 1.3.1 */ def drop(how: String): DS[Row] = drop(toMinNonNulls(how)) /** - * Returns a new `DataFrame` that drops rows containing any null or NaN values - * in the specified columns. + * Returns a new `DataFrame` that drops rows containing any null or NaN values in the specified + * columns. * * @since 1.3.1 */ @@ -66,8 +66,8 @@ abstract class DataFrameNaFunctions[DS[U] <: Dataset[U, DS]] { def drop(cols: Seq[String]): DS[Row] = drop(cols.size, cols) /** - * Returns a new `DataFrame` that drops rows containing null or NaN values - * in the specified columns. + * Returns a new `DataFrame` that drops rows containing null or NaN values in the specified + * columns. * * If `how` is "any", then drop rows containing any null or NaN values in the specified columns. * If `how` is "all", then drop rows only if every specified column is null or NaN for that row. @@ -77,8 +77,8 @@ abstract class DataFrameNaFunctions[DS[U] <: Dataset[U, DS]] { def drop(how: String, cols: Array[String]): DS[Row] = drop(how, cols.toImmutableArraySeq) /** - * (Scala-specific) Returns a new `DataFrame` that drops rows containing null or NaN values - * in the specified columns. + * (Scala-specific) Returns a new `DataFrame` that drops rows containing null or NaN values in + * the specified columns. * * If `how` is "any", then drop rows containing any null or NaN values in the specified columns. * If `how` is "all", then drop rows only if every specified column is null or NaN for that row. @@ -88,16 +88,16 @@ abstract class DataFrameNaFunctions[DS[U] <: Dataset[U, DS]] { def drop(how: String, cols: Seq[String]): DS[Row] = drop(toMinNonNulls(how), cols) /** - * Returns a new `DataFrame` that drops rows containing - * less than `minNonNulls` non-null and non-NaN values. + * Returns a new `DataFrame` that drops rows containing less than `minNonNulls` non-null and + * non-NaN values. * * @since 1.3.1 */ def drop(minNonNulls: Int): DS[Row] = drop(Option(minNonNulls)) /** - * Returns a new `DataFrame` that drops rows containing - * less than `minNonNulls` non-null and non-NaN values in the specified columns. + * Returns a new `DataFrame` that drops rows containing less than `minNonNulls` non-null and + * non-NaN values in the specified columns. * * @since 1.3.1 */ @@ -105,8 +105,8 @@ abstract class DataFrameNaFunctions[DS[U] <: Dataset[U, DS]] { drop(minNonNulls, cols.toImmutableArraySeq) /** - * (Scala-specific) Returns a new `DataFrame` that drops rows containing less than - * `minNonNulls` non-null and non-NaN values in the specified columns. + * (Scala-specific) Returns a new `DataFrame` that drops rows containing less than `minNonNulls` + * non-null and non-NaN values in the specified columns. * * @since 1.3.1 */ @@ -145,16 +145,16 @@ abstract class DataFrameNaFunctions[DS[U] <: Dataset[U, DS]] { def fill(value: String): DS[Row] /** - * Returns a new `DataFrame` that replaces null or NaN values in specified numeric columns. - * If a specified column is not a numeric column, it is ignored. + * Returns a new `DataFrame` that replaces null or NaN values in specified numeric columns. If a + * specified column is not a numeric column, it is ignored. * * @since 2.2.0 */ def fill(value: Long, cols: Array[String]): DS[Row] = fill(value, cols.toImmutableArraySeq) /** - * Returns a new `DataFrame` that replaces null or NaN values in specified numeric columns. - * If a specified column is not a numeric column, it is ignored. + * Returns a new `DataFrame` that replaces null or NaN values in specified numeric columns. If a + * specified column is not a numeric column, it is ignored. * * @since 1.3.1 */ @@ -176,18 +176,17 @@ abstract class DataFrameNaFunctions[DS[U] <: Dataset[U, DS]] { */ def fill(value: Double, cols: Seq[String]): DS[Row] - /** - * Returns a new `DataFrame` that replaces null values in specified string columns. - * If a specified column is not a string column, it is ignored. + * Returns a new `DataFrame` that replaces null values in specified string columns. If a + * specified column is not a string column, it is ignored. * * @since 1.3.1 */ def fill(value: String, cols: Array[String]): DS[Row] = fill(value, cols.toImmutableArraySeq) /** - * (Scala-specific) Returns a new `DataFrame` that replaces null values in - * specified string columns. If a specified column is not a string column, it is ignored. + * (Scala-specific) Returns a new `DataFrame` that replaces null values in specified string + * columns. If a specified column is not a string column, it is ignored. * * @since 1.3.1 */ @@ -201,16 +200,16 @@ abstract class DataFrameNaFunctions[DS[U] <: Dataset[U, DS]] { def fill(value: Boolean): DS[Row] /** - * (Scala-specific) Returns a new `DataFrame` that replaces null values in specified - * boolean columns. If a specified column is not a boolean column, it is ignored. + * (Scala-specific) Returns a new `DataFrame` that replaces null values in specified boolean + * columns. If a specified column is not a boolean column, it is ignored. * * @since 2.3.0 */ def fill(value: Boolean, cols: Seq[String]): DS[Row] /** - * Returns a new `DataFrame` that replaces null values in specified boolean columns. - * If a specified column is not a boolean column, it is ignored. + * Returns a new `DataFrame` that replaces null values in specified boolean columns. If a + * specified column is not a boolean column, it is ignored. * * @since 2.3.0 */ @@ -219,13 +218,12 @@ abstract class DataFrameNaFunctions[DS[U] <: Dataset[U, DS]] { /** * Returns a new `DataFrame` that replaces null values. * - * The key of the map is the column name, and the value of the map is the replacement value. - * The value must be of the following type: - * `Integer`, `Long`, `Float`, `Double`, `String`, `Boolean`. - * Replacement values are cast to the column data type. + * The key of the map is the column name, and the value of the map is the replacement value. The + * value must be of the following type: `Integer`, `Long`, `Float`, `Double`, `String`, + * `Boolean`. Replacement values are cast to the column data type. * - * For example, the following replaces null values in column "A" with string "unknown", and - * null values in column "B" with numeric value 1.0. + * For example, the following replaces null values in column "A" with string "unknown", and null + * values in column "B" with numeric value 1.0. * {{{ * import com.google.common.collect.ImmutableMap; * df.na.fill(ImmutableMap.of("A", "unknown", "B", 1.0)); @@ -238,12 +236,12 @@ abstract class DataFrameNaFunctions[DS[U] <: Dataset[U, DS]] { /** * (Scala-specific) Returns a new `DataFrame` that replaces null values. * - * The key of the map is the column name, and the value of the map is the replacement value. - * The value must be of the following type: `Int`, `Long`, `Float`, `Double`, `String`, `Boolean`. + * The key of the map is the column name, and the value of the map is the replacement value. The + * value must be of the following type: `Int`, `Long`, `Float`, `Double`, `String`, `Boolean`. * Replacement values are cast to the column data type. * - * For example, the following replaces null values in column "A" with string "unknown", and - * null values in column "B" with numeric value 1.0. + * For example, the following replaces null values in column "A" with string "unknown", and null + * values in column "B" with numeric value 1.0. * {{{ * df.na.fill(Map( * "A" -> "unknown", @@ -273,11 +271,12 @@ abstract class DataFrameNaFunctions[DS[U] <: Dataset[U, DS]] { * df.na.replace("*", ImmutableMap.of("UNKNOWN", "unnamed")); * }}} * - * @param col name of the column to apply the value replacement. If `col` is "*", - * replacement is applied on all string, numeric or boolean columns. - * @param replacement value replacement map. Key and value of `replacement` map must have - * the same type, and can only be doubles, strings or booleans. - * The map value can have nulls. + * @param col + * name of the column to apply the value replacement. If `col` is "*", replacement is applied + * on all string, numeric or boolean columns. + * @param replacement + * value replacement map. Key and value of `replacement` map must have the same type, and can + * only be doubles, strings or booleans. The map value can have nulls. * * @since 1.3.1 */ @@ -298,11 +297,12 @@ abstract class DataFrameNaFunctions[DS[U] <: Dataset[U, DS]] { * df.na.replace(new String[] {"firstname", "lastname"}, ImmutableMap.of("UNKNOWN", "unnamed")); * }}} * - * @param cols list of columns to apply the value replacement. If `col` is "*", - * replacement is applied on all string, numeric or boolean columns. - * @param replacement value replacement map. Key and value of `replacement` map must have - * the same type, and can only be doubles, strings or booleans. - * The map value can have nulls. + * @param cols + * list of columns to apply the value replacement. If `col` is "*", replacement is applied on + * all string, numeric or boolean columns. + * @param replacement + * value replacement map. Key and value of `replacement` map must have the same type, and can + * only be doubles, strings or booleans. The map value can have nulls. * * @since 1.3.1 */ @@ -324,11 +324,12 @@ abstract class DataFrameNaFunctions[DS[U] <: Dataset[U, DS]] { * df.na.replace("*", Map("UNKNOWN" -> "unnamed")); * }}} * - * @param col name of the column to apply the value replacement. If `col` is "*", - * replacement is applied on all string, numeric or boolean columns. - * @param replacement value replacement map. Key and value of `replacement` map must have - * the same type, and can only be doubles, strings or booleans. - * The map value can have nulls. + * @param col + * name of the column to apply the value replacement. If `col` is "*", replacement is applied + * on all string, numeric or boolean columns. + * @param replacement + * value replacement map. Key and value of `replacement` map must have the same type, and can + * only be doubles, strings or booleans. The map value can have nulls. * * @since 1.3.1 */ @@ -345,11 +346,12 @@ abstract class DataFrameNaFunctions[DS[U] <: Dataset[U, DS]] { * df.na.replace("firstname" :: "lastname" :: Nil, Map("UNKNOWN" -> "unnamed")); * }}} * - * @param cols list of columns to apply the value replacement. If `col` is "*", - * replacement is applied on all string, numeric or boolean columns. - * @param replacement value replacement map. Key and value of `replacement` map must have - * the same type, and can only be doubles, strings or booleans. - * The map value can have nulls. + * @param cols + * list of columns to apply the value replacement. If `col` is "*", replacement is applied on + * all string, numeric or boolean columns. + * @param replacement + * value replacement map. Key and value of `replacement` map must have the same type, and can + * only be doubles, strings or booleans. The map value can have nulls. * * @since 1.3.1 */ diff --git a/sql/api/src/main/scala/org/apache/spark/sql/api/DataFrameReader.scala b/sql/api/src/main/scala/org/apache/spark/sql/api/DataFrameReader.scala index b9910e8468267..6e6ab7b9d95a4 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/api/DataFrameReader.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/api/DataFrameReader.scala @@ -35,7 +35,8 @@ import org.apache.spark.sql.types.StructType */ @Stable abstract class DataFrameReader[DS[U] <: Dataset[U, DS]] { -/** + + /** * Specifies the input data source format. * * @since 1.4.0 @@ -47,8 +48,8 @@ abstract class DataFrameReader[DS[U] <: Dataset[U, DS]] { /** * Specifies the input schema. Some data sources (e.g. JSON) can infer the input schema - * automatically from data. By specifying the schema here, the underlying data source can - * skip the schema inference step, and thus speed up data loading. + * automatically from data. By specifying the schema here, the underlying data source can skip + * the schema inference step, and thus speed up data loading. * * @since 1.4.0 */ @@ -62,9 +63,9 @@ abstract class DataFrameReader[DS[U] <: Dataset[U, DS]] { } /** - * Specifies the schema by using the input DDL-formatted string. Some data sources (e.g. JSON) can - * infer the input schema automatically from data. By specifying the schema here, the underlying - * data source can skip the schema inference step, and thus speed up data loading. + * Specifies the schema by using the input DDL-formatted string. Some data sources (e.g. JSON) + * can infer the input schema automatically from data. By specifying the schema here, the + * underlying data source can skip the schema inference step, and thus speed up data loading. * * {{{ * spark.read.schema("a INT, b STRING, c DOUBLE").csv("test.csv") @@ -77,8 +78,8 @@ abstract class DataFrameReader[DS[U] <: Dataset[U, DS]] { /** * Adds an input option for the underlying data source. * - * All options are maintained in a case-insensitive way in terms of key names. - * If a new option has the same key case-insensitively, it will override the existing option. + * All options are maintained in a case-insensitive way in terms of key names. If a new option + * has the same key case-insensitively, it will override the existing option. * * @since 1.4.0 */ @@ -91,8 +92,8 @@ abstract class DataFrameReader[DS[U] <: Dataset[U, DS]] { /** * Adds an input option for the underlying data source. * - * All options are maintained in a case-insensitive way in terms of key names. - * If a new option has the same key case-insensitively, it will override the existing option. + * All options are maintained in a case-insensitive way in terms of key names. If a new option + * has the same key case-insensitively, it will override the existing option. * * @since 2.0.0 */ @@ -101,8 +102,8 @@ abstract class DataFrameReader[DS[U] <: Dataset[U, DS]] { /** * Adds an input option for the underlying data source. * - * All options are maintained in a case-insensitive way in terms of key names. - * If a new option has the same key case-insensitively, it will override the existing option. + * All options are maintained in a case-insensitive way in terms of key names. If a new option + * has the same key case-insensitively, it will override the existing option. * * @since 2.0.0 */ @@ -111,8 +112,8 @@ abstract class DataFrameReader[DS[U] <: Dataset[U, DS]] { /** * Adds an input option for the underlying data source. * - * All options are maintained in a case-insensitive way in terms of key names. - * If a new option has the same key case-insensitively, it will override the existing option. + * All options are maintained in a case-insensitive way in terms of key names. If a new option + * has the same key case-insensitively, it will override the existing option. * * @since 2.0.0 */ @@ -121,8 +122,8 @@ abstract class DataFrameReader[DS[U] <: Dataset[U, DS]] { /** * (Scala-specific) Adds input options for the underlying data source. * - * All options are maintained in a case-insensitive way in terms of key names. - * If a new option has the same key case-insensitively, it will override the existing option. + * All options are maintained in a case-insensitive way in terms of key names. If a new option + * has the same key case-insensitively, it will override the existing option. * * @since 1.4.0 */ @@ -135,8 +136,8 @@ abstract class DataFrameReader[DS[U] <: Dataset[U, DS]] { /** * Adds input options for the underlying data source. * - * All options are maintained in a case-insensitive way in terms of key names. - * If a new option has the same key case-insensitively, it will override the existing option. + * All options are maintained in a case-insensitive way in terms of key names. If a new option + * has the same key case-insensitively, it will override the existing option. * * @since 1.4.0 */ @@ -151,16 +152,16 @@ abstract class DataFrameReader[DS[U] <: Dataset[U, DS]] { def load(): DS[Row] /** - * Loads input in as a `DataFrame`, for data sources that require a path (e.g. data backed by - * a local or distributed file system). + * Loads input in as a `DataFrame`, for data sources that require a path (e.g. data backed by a + * local or distributed file system). * * @since 1.4.0 */ def load(path: String): DS[Row] /** - * Loads input in as a `DataFrame`, for data sources that support multiple paths. - * Only works if the source is a HadoopFsRelationProvider. + * Loads input in as a `DataFrame`, for data sources that support multiple paths. Only works if + * the source is a HadoopFsRelationProvider. * * @since 1.6.0 */ @@ -168,13 +169,13 @@ abstract class DataFrameReader[DS[U] <: Dataset[U, DS]] { def load(paths: String*): DS[Row] /** - * Construct a `DataFrame` representing the database table accessible via JDBC URL - * url named table and connection properties. + * Construct a `DataFrame` representing the database table accessible via JDBC URL url named + * table and connection properties. * - * You can find the JDBC-specific option and parameter documentation for reading tables - * via JDBC in - * - * Data Source Option in the version you use. + * You can find the JDBC-specific option and parameter documentation for reading tables via JDBC + * in + * Data Source Option in the version you use. * * @since 1.4.0 */ @@ -189,26 +190,29 @@ abstract class DataFrameReader[DS[U] <: Dataset[U, DS]] { // scalastyle:off line.size.limit /** - * Construct a `DataFrame` representing the database table accessible via JDBC URL - * url named table. Partitions of the table will be retrieved in parallel based on the parameters - * passed to this function. + * Construct a `DataFrame` representing the database table accessible via JDBC URL url named + * table. Partitions of the table will be retrieved in parallel based on the parameters passed + * to this function. * * Don't create too many partitions in parallel on a large cluster; otherwise Spark might crash * your external database systems. * - * You can find the JDBC-specific option and parameter documentation for reading tables via JDBC in - * - * Data Source Option in the version you use. + * You can find the JDBC-specific option and parameter documentation for reading tables via JDBC + * in + * Data Source Option in the version you use. * - * @param table Name of the table in the external database. - * @param columnName Alias of `partitionColumn` option. Refer to `partitionColumn` in - * - * Data Source Option in the version you use. - * @param connectionProperties JDBC database connection arguments, a list of arbitrary string - * tag/value. Normally at least a "user" and "password" property - * should be included. "fetchsize" can be used to control the - * number of rows per fetch and "queryTimeout" can be used to wait - * for a Statement object to execute to the given number of seconds. + * @param table + * Name of the table in the external database. + * @param columnName + * Alias of `partitionColumn` option. Refer to `partitionColumn` in + * Data Source Option in the version you use. + * @param connectionProperties + * JDBC database connection arguments, a list of arbitrary string tag/value. Normally at least + * a "user" and "password" property should be included. "fetchsize" can be used to control the + * number of rows per fetch and "queryTimeout" can be used to wait for a Statement object to + * execute to the given number of seconds. * @since 1.4.0 */ // scalastyle:on line.size.limit @@ -230,25 +234,26 @@ abstract class DataFrameReader[DS[U] <: Dataset[U, DS]] { } /** - * Construct a `DataFrame` representing the database table accessible via JDBC URL - * url named table using connection properties. The `predicates` parameter gives a list - * expressions suitable for inclusion in WHERE clauses; each one defines one partition - * of the `DataFrame`. + * Construct a `DataFrame` representing the database table accessible via JDBC URL url named + * table using connection properties. The `predicates` parameter gives a list expressions + * suitable for inclusion in WHERE clauses; each one defines one partition of the `DataFrame`. * * Don't create too many partitions in parallel on a large cluster; otherwise Spark might crash * your external database systems. * - * You can find the JDBC-specific option and parameter documentation for reading tables - * via JDBC in - * - * Data Source Option in the version you use. + * You can find the JDBC-specific option and parameter documentation for reading tables via JDBC + * in + * Data Source Option in the version you use. * - * @param table Name of the table in the external database. - * @param predicates Condition in the where clause for each partition. - * @param connectionProperties JDBC database connection arguments, a list of arbitrary string - * tag/value. Normally at least a "user" and "password" property - * should be included. "fetchsize" can be used to control the - * number of rows per fetch. + * @param table + * Name of the table in the external database. + * @param predicates + * Condition in the where clause for each partition. + * @param connectionProperties + * JDBC database connection arguments, a list of arbitrary string tag/value. Normally at least + * a "user" and "password" property should be included. "fetchsize" can be used to control the + * number of rows per fetch. * @since 1.4.0 */ def jdbc( @@ -278,8 +283,8 @@ abstract class DataFrameReader[DS[U] <: Dataset[U, DS]] { * This function goes through the input once to determine the input schema. If you know the * schema in advance, use the version that specifies the schema to avoid the extra scan. * - * You can find the JSON-specific options for reading JSON files in - * + * You can find the JSON-specific options for reading JSON files in * Data Source Option in the version you use. * * @since 2.0.0 @@ -294,17 +299,18 @@ abstract class DataFrameReader[DS[U] <: Dataset[U, DS]] { * Loads a `Dataset[String]` storing JSON objects (JSON Lines * text format or newline-delimited JSON) and returns the result as a `DataFrame`. * - * Unless the schema is specified using `schema` function, this function goes through the - * input once to determine the input schema. + * Unless the schema is specified using `schema` function, this function goes through the input + * once to determine the input schema. * - * @param jsonDataset input Dataset with one JSON object per record + * @param jsonDataset + * input Dataset with one JSON object per record * @since 2.2.0 */ - def json(jsonDataset: DS[String] ): DS[Row] + def json(jsonDataset: DS[String]): DS[Row] /** - * Loads a CSV file and returns the result as a `DataFrame`. See the documentation on the - * other overloaded `csv()` method for more details. + * Loads a CSV file and returns the result as a `DataFrame`. See the documentation on the other + * overloaded `csv()` method for more details. * * @since 2.0.0 */ @@ -323,13 +329,15 @@ abstract class DataFrameReader[DS[U] <: Dataset[U, DS]] { * it determines the columns as string types and it reads only the first line to determine the * names and the number of fields. * - * If the enforceSchema is set to `false`, only the CSV header in the first line is checked - * to conform specified or inferred schema. + * If the enforceSchema is set to `false`, only the CSV header in the first line is checked to + * conform specified or inferred schema. * - * @note if `header` option is set to `true` when calling this API, all lines same with - * the header will be removed if exists. + * @note + * if `header` option is set to `true` when calling this API, all lines same with the header + * will be removed if exists. * - * @param csvDataset input Dataset with one CSV row per record + * @param csvDataset + * input Dataset with one CSV row per record * @since 2.2.0 */ def csv(csvDataset: DS[String]): DS[Row] @@ -341,18 +349,18 @@ abstract class DataFrameReader[DS[U] <: Dataset[U, DS]] { * is enabled. To avoid going through the entire data once, disable `inferSchema` option or * specify the schema explicitly using `schema`. * - * You can find the CSV-specific options for reading CSV files in - * - * Data Source Option in the version you use. + * You can find the CSV-specific options for reading CSV files in + * Data Source Option in the version you use. * * @since 2.0.0 */ @scala.annotation.varargs - def csv(paths: String*): DS[Row] = format("csv").load(paths : _*) + def csv(paths: String*): DS[Row] = format("csv").load(paths: _*) /** - * Loads a XML file and returns the result as a `DataFrame`. See the documentation on the - * other overloaded `xml()` method for more details. + * Loads a XML file and returns the result as a `DataFrame`. See the documentation on the other + * overloaded `xml()` method for more details. * * @since 4.0.0 */ @@ -368,8 +376,8 @@ abstract class DataFrameReader[DS[U] <: Dataset[U, DS]] { * is enabled. To avoid going through the entire data once, disable `inferSchema` option or * specify the schema explicitly using `schema`. * - * You can find the XML-specific options for reading XML files in - * + * You can find the XML-specific options for reading XML files in * Data Source Option in the version you use. * * @since 4.0.0 @@ -386,14 +394,15 @@ abstract class DataFrameReader[DS[U] <: Dataset[U, DS]] { * If the schema is not specified using `schema` function and `inferSchema` option is enabled, * this function goes through the input once to determine the input schema. * - * @param xmlDataset input Dataset with one XML object per record + * @param xmlDataset + * input Dataset with one XML object per record * @since 4.0.0 */ def xml(xmlDataset: DS[String]): DS[Row] /** - * Loads a Parquet file, returning the result as a `DataFrame`. See the documentation - * on the other overloaded `parquet()` method for more details. + * Loads a Parquet file, returning the result as a `DataFrame`. See the documentation on the + * other overloaded `parquet()` method for more details. * * @since 2.0.0 */ @@ -405,10 +414,9 @@ abstract class DataFrameReader[DS[U] <: Dataset[U, DS]] { /** * Loads a Parquet file, returning the result as a `DataFrame`. * - * Parquet-specific option(s) for reading Parquet files can be found in - * - * Data Source Option in the version you use. + * Parquet-specific option(s) for reading Parquet files can be found in Data + * Source Option in the version you use. * * @since 1.4.0 */ @@ -418,7 +426,8 @@ abstract class DataFrameReader[DS[U] <: Dataset[U, DS]] { /** * Loads an ORC file and returns the result as a `DataFrame`. * - * @param path input path + * @param path + * input path * @since 1.5.0 */ def orc(path: String): DS[Row] = { @@ -429,12 +438,12 @@ abstract class DataFrameReader[DS[U] <: Dataset[U, DS]] { /** * Loads ORC files and returns the result as a `DataFrame`. * - * ORC-specific option(s) for reading ORC files can be found in - * - * Data Source Option in the version you use. + * ORC-specific option(s) for reading ORC files can be found in Data + * Source Option in the version you use. * - * @param paths input paths + * @param paths + * input paths * @since 2.0.0 */ @scala.annotation.varargs @@ -442,23 +451,23 @@ abstract class DataFrameReader[DS[U] <: Dataset[U, DS]] { /** * Returns the specified table/view as a `DataFrame`. If it's a table, it must support batch - * reading and the returned DataFrame is the batch scan query plan of this table. If it's a view, - * the returned DataFrame is simply the query plan of the view, which can either be a batch or - * streaming query plan. - * - * @param tableName is either a qualified or unqualified name that designates a table or view. - * If a database is specified, it identifies the table/view from the database. - * Otherwise, it first attempts to find a temporary view with the given name - * and then match the table/view from the current database. - * Note that, the global temporary view database is also valid here. + * reading and the returned DataFrame is the batch scan query plan of this table. If it's a + * view, the returned DataFrame is simply the query plan of the view, which can either be a + * batch or streaming query plan. + * + * @param tableName + * is either a qualified or unqualified name that designates a table or view. If a database is + * specified, it identifies the table/view from the database. Otherwise, it first attempts to + * find a temporary view with the given name and then match the table/view from the current + * database. Note that, the global temporary view database is also valid here. * @since 1.4.0 */ def table(tableName: String): DS[Row] /** * Loads text files and returns a `DataFrame` whose schema starts with a string column named - * "value", and followed by partitioned columns if there are any. See the documentation on - * the other overloaded `text()` method for more details. + * "value", and followed by partitioned columns if there are any. See the documentation on the + * other overloaded `text()` method for more details. * * @since 2.0.0 */ @@ -469,8 +478,8 @@ abstract class DataFrameReader[DS[U] <: Dataset[U, DS]] { /** * Loads text files and returns a `DataFrame` whose schema starts with a string column named - * "value", and followed by partitioned columns if there are any. - * The text files must be encoded as UTF-8. + * "value", and followed by partitioned columns if there are any. The text files must be encoded + * as UTF-8. * * By default, each line in the text files is a new row in the resulting DataFrame. For example: * {{{ @@ -481,19 +490,20 @@ abstract class DataFrameReader[DS[U] <: Dataset[U, DS]] { * spark.read().text("/path/to/spark/README.md") * }}} * - * You can find the text-specific options for reading text files in - * - * Data Source Option in the version you use. + * You can find the text-specific options for reading text files in + * Data Source Option in the version you use. * - * @param paths input paths + * @param paths + * input paths * @since 1.6.0 */ @scala.annotation.varargs - def text(paths: String*): DS[Row] = format("text").load(paths : _*) + def text(paths: String*): DS[Row] = format("text").load(paths: _*) /** - * Loads text files and returns a [[Dataset]] of String. See the documentation on the - * other overloaded `textFile()` method for more details. + * Loads text files and returns a [[Dataset]] of String. See the documentation on the other + * overloaded `textFile()` method for more details. * @since 2.0.0 */ def textFile(path: String): DS[String] = { @@ -503,8 +513,7 @@ abstract class DataFrameReader[DS[U] <: Dataset[U, DS]] { /** * Loads text files and returns a [[Dataset]] of String. The underlying schema of the Dataset - * contains a single string column named "value". - * The text files must be encoded as UTF-8. + * contains a single string column named "value". The text files must be encoded as UTF-8. * * If the directory structure of the text files contains partitioning information, those are * ignored in the resulting Dataset. To include partitioning information as columns, use `text`. @@ -520,13 +529,14 @@ abstract class DataFrameReader[DS[U] <: Dataset[U, DS]] { * * You can set the text-specific options as specified in `DataFrameReader.text`. * - * @param paths input path + * @param paths + * input path * @since 2.0.0 */ @scala.annotation.varargs def textFile(paths: String*): DS[String] = { assertNoSpecifiedSchema("textFile") - text(paths : _*).select("value").as(StringEncoder) + text(paths: _*).select("value").as(StringEncoder) } /** diff --git a/sql/api/src/main/scala/org/apache/spark/sql/api/DataFrameStatFunctions.scala b/sql/api/src/main/scala/org/apache/spark/sql/api/DataFrameStatFunctions.scala index c3ecc7b90d5b4..fc1680231be5b 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/api/DataFrameStatFunctions.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/api/DataFrameStatFunctions.scala @@ -40,32 +40,35 @@ abstract class DataFrameStatFunctions[DS[U] <: Dataset[U, DS]] { /** * Calculates the approximate quantiles of a numerical column of a DataFrame. * - * The result of this algorithm has the following deterministic bound: - * If the DataFrame has N elements and if we request the quantile at probability `p` up to error - * `err`, then the algorithm will return a sample `x` from the DataFrame so that the *exact* rank - * of `x` is close to (p * N). - * More precisely, + * The result of this algorithm has the following deterministic bound: If the DataFrame has N + * elements and if we request the quantile at probability `p` up to error `err`, then the + * algorithm will return a sample `x` from the DataFrame so that the *exact* rank of `x` is + * close to (p * N). More precisely, * * {{{ * floor((p - err) * N) <= rank(x) <= ceil((p + err) * N) * }}} * * This method implements a variation of the Greenwald-Khanna algorithm (with some speed - * optimizations). - * The algorithm was first present in - * Space-efficient Online Computation of Quantile Summaries by Greenwald and Khanna. - * - * @param col the name of the numerical column - * @param probabilities a list of quantile probabilities - * Each number must belong to [0, 1]. - * For example 0 is the minimum, 0.5 is the median, 1 is the maximum. - * @param relativeError The relative target precision to achieve (greater than or equal to 0). - * If set to zero, the exact quantiles are computed, which could be very expensive. - * Note that values greater than 1 are accepted but give the same result as 1. - * @return the approximate quantiles at the given probabilities - * - * @note null and NaN values will be removed from the numerical column before calculation. If - * the dataframe is empty or the column only contains null or NaN, an empty array is returned. + * optimizations). The algorithm was first present in Space-efficient Online Computation of Quantile + * Summaries by Greenwald and Khanna. + * + * @param col + * the name of the numerical column + * @param probabilities + * a list of quantile probabilities Each number must belong to [0, 1]. For example 0 is the + * minimum, 0.5 is the median, 1 is the maximum. + * @param relativeError + * The relative target precision to achieve (greater than or equal to 0). If set to zero, the + * exact quantiles are computed, which could be very expensive. Note that values greater than + * 1 are accepted but give the same result as 1. + * @return + * the approximate quantiles at the given probabilities + * + * @note + * null and NaN values will be removed from the numerical column before calculation. If the + * dataframe is empty or the column only contains null or NaN, an empty array is returned. * * @since 2.0.0 */ @@ -78,19 +81,24 @@ abstract class DataFrameStatFunctions[DS[U] <: Dataset[U, DS]] { /** * Calculates the approximate quantiles of numerical columns of a DataFrame. - * @see `approxQuantile(col:Str* approxQuantile)` for detailed description. - * - * @param cols the names of the numerical columns - * @param probabilities a list of quantile probabilities - * Each number must belong to [0, 1]. - * For example 0 is the minimum, 0.5 is the median, 1 is the maximum. - * @param relativeError The relative target precision to achieve (greater than or equal to 0). - * If set to zero, the exact quantiles are computed, which could be very expensive. - * Note that values greater than 1 are accepted but give the same result as 1. - * @return the approximate quantiles at the given probabilities of each column - * - * @note null and NaN values will be ignored in numerical columns before calculation. For - * columns only containing null or NaN values, an empty array is returned. + * @see + * `approxQuantile(col:Str* approxQuantile)` for detailed description. + * + * @param cols + * the names of the numerical columns + * @param probabilities + * a list of quantile probabilities Each number must belong to [0, 1]. For example 0 is the + * minimum, 0.5 is the median, 1 is the maximum. + * @param relativeError + * The relative target precision to achieve (greater than or equal to 0). If set to zero, the + * exact quantiles are computed, which could be very expensive. Note that values greater than + * 1 are accepted but give the same result as 1. + * @return + * the approximate quantiles at the given probabilities of each column + * + * @note + * null and NaN values will be ignored in numerical columns before calculation. For columns + * only containing null or NaN values, an empty array is returned. * * @since 2.2.0 */ @@ -102,9 +110,12 @@ abstract class DataFrameStatFunctions[DS[U] <: Dataset[U, DS]] { /** * Calculate the sample covariance of two numerical columns of a DataFrame. * - * @param col1 the name of the first column - * @param col2 the name of the second column - * @return the covariance of the two columns. + * @param col1 + * the name of the first column + * @param col2 + * the name of the second column + * @return + * the covariance of the two columns. * * {{{ * val df = sc.parallelize(0 until 10).toDF("id").withColumn("rand1", rand(seed=10)) @@ -121,9 +132,12 @@ abstract class DataFrameStatFunctions[DS[U] <: Dataset[U, DS]] { * Correlation Coefficient. For Spearman Correlation, consider using RDD methods found in * MLlib's Statistics. * - * @param col1 the name of the column - * @param col2 the name of the column to calculate the correlation against - * @return The Pearson Correlation Coefficient as a Double. + * @param col1 + * the name of the column + * @param col2 + * the name of the column to calculate the correlation against + * @return + * The Pearson Correlation Coefficient as a Double. * * {{{ * val df = sc.parallelize(0 until 10).toDF("id").withColumn("rand1", rand(seed=10)) @@ -138,9 +152,12 @@ abstract class DataFrameStatFunctions[DS[U] <: Dataset[U, DS]] { /** * Calculates the Pearson Correlation Coefficient of two columns of a DataFrame. * - * @param col1 the name of the column - * @param col2 the name of the column to calculate the correlation against - * @return The Pearson Correlation Coefficient as a Double. + * @param col1 + * the name of the column + * @param col2 + * the name of the column to calculate the correlation against + * @return + * The Pearson Correlation Coefficient as a Double. * * {{{ * val df = sc.parallelize(0 until 10).toDF("id").withColumn("rand1", rand(seed=10)) @@ -159,14 +176,15 @@ abstract class DataFrameStatFunctions[DS[U] <: Dataset[U, DS]] { * The first column of each row will be the distinct values of `col1` and the column names will * be the distinct values of `col2`. The name of the first column will be `col1_col2`. Counts * will be returned as `Long`s. Pairs that have no occurrences will have zero as their counts. - * Null elements will be replaced by "null", and back ticks will be dropped from elements if they - * exist. + * Null elements will be replaced by "null", and back ticks will be dropped from elements if + * they exist. * - * @param col1 The name of the first column. Distinct items will make the first item of - * each row. - * @param col2 The name of the second column. Distinct items will make the column names - * of the DataFrame. - * @return A DataFrame containing for the contingency table. + * @param col1 + * The name of the first column. Distinct items will make the first item of each row. + * @param col2 + * The name of the second column. Distinct items will make the column names of the DataFrame. + * @return + * A DataFrame containing for the contingency table. * * {{{ * val df = spark.createDataFrame(Seq((1, 1), (1, 2), (2, 1), (2, 1), (2, 3), (3, 2), (3, 3))) @@ -187,19 +205,19 @@ abstract class DataFrameStatFunctions[DS[U] <: Dataset[U, DS]] { def crosstab(col1: String, col2: String): DS[Row] /** - * Finding frequent items for columns, possibly with false positives. Using the - * frequent element count algorithm described in - * here, proposed by Karp, - * Schenker, and Papadimitriou. - * The `support` should be greater than 1e-4. + * Finding frequent items for columns, possibly with false positives. Using the frequent element + * count algorithm described in here, + * proposed by Karp, Schenker, and Papadimitriou. The `support` should be greater than 1e-4. * * This function is meant for exploratory data analysis, as we make no guarantee about the * backward compatibility of the schema of the resulting `DataFrame`. * - * @param cols the names of the columns to search frequent items in. - * @param support The minimum frequency for an item to be considered `frequent`. Should be greater - * than 1e-4. - * @return A Local DataFrame with the Array of frequent items for each column. + * @param cols + * the names of the columns to search frequent items in. + * @param support + * The minimum frequency for an item to be considered `frequent`. Should be greater than 1e-4. + * @return + * A Local DataFrame with the Array of frequent items for each column. * * {{{ * val rows = Seq.tabulate(100) { i => @@ -232,32 +250,34 @@ abstract class DataFrameStatFunctions[DS[U] <: Dataset[U, DS]] { freqItems(cols.toImmutableArraySeq, support) /** - * Finding frequent items for columns, possibly with false positives. Using the - * frequent element count algorithm described in - * here, proposed by Karp, - * Schenker, and Papadimitriou. - * Uses a `default` support of 1%. + * Finding frequent items for columns, possibly with false positives. Using the frequent element + * count algorithm described in here, + * proposed by Karp, Schenker, and Papadimitriou. Uses a `default` support of 1%. * * This function is meant for exploratory data analysis, as we make no guarantee about the * backward compatibility of the schema of the resulting `DataFrame`. * - * @param cols the names of the columns to search frequent items in. - * @return A Local DataFrame with the Array of frequent items for each column. + * @param cols + * the names of the columns to search frequent items in. + * @return + * A Local DataFrame with the Array of frequent items for each column. * @since 1.4.0 */ def freqItems(cols: Array[String]): DS[Row] = freqItems(cols, 0.01) /** * (Scala-specific) Finding frequent items for columns, possibly with false positives. Using the - * frequent element count algorithm described in - * here, proposed by Karp, Schenker, - * and Papadimitriou. + * frequent element count algorithm described in here, proposed by Karp, Schenker, and + * Papadimitriou. * * This function is meant for exploratory data analysis, as we make no guarantee about the * backward compatibility of the schema of the resulting `DataFrame`. * - * @param cols the names of the columns to search frequent items in. - * @return A Local DataFrame with the Array of frequent items for each column. + * @param cols + * the names of the columns to search frequent items in. + * @return + * A Local DataFrame with the Array of frequent items for each column. * * {{{ * val rows = Seq.tabulate(100) { i => @@ -291,28 +311,34 @@ abstract class DataFrameStatFunctions[DS[U] <: Dataset[U, DS]] { /** * (Scala-specific) Finding frequent items for columns, possibly with false positives. Using the - * frequent element count algorithm described in - * here, proposed by Karp, Schenker, - * and Papadimitriou. - * Uses a `default` support of 1%. + * frequent element count algorithm described in here, proposed by Karp, Schenker, and + * Papadimitriou. Uses a `default` support of 1%. * * This function is meant for exploratory data analysis, as we make no guarantee about the * backward compatibility of the schema of the resulting `DataFrame`. * - * @param cols the names of the columns to search frequent items in. - * @return A Local DataFrame with the Array of frequent items for each column. + * @param cols + * the names of the columns to search frequent items in. + * @return + * A Local DataFrame with the Array of frequent items for each column. * @since 1.4.0 */ def freqItems(cols: Seq[String]): DS[Row] = freqItems(cols, 0.01) /** * Returns a stratified sample without replacement based on the fraction given on each stratum. - * @param col column that defines strata - * @param fractions sampling fraction for each stratum. If a stratum is not specified, we treat - * its fraction as zero. - * @param seed random seed - * @tparam T stratum type - * @return a new `DataFrame` that represents the stratified sample + * @param col + * column that defines strata + * @param fractions + * sampling fraction for each stratum. If a stratum is not specified, we treat its fraction as + * zero. + * @param seed + * random seed + * @tparam T + * stratum type + * @return + * a new `DataFrame` that represents the stratified sample * * {{{ * val df = spark.createDataFrame(Seq((1, 1), (1, 2), (2, 1), (2, 1), (2, 3), (3, 2), @@ -336,12 +362,17 @@ abstract class DataFrameStatFunctions[DS[U] <: Dataset[U, DS]] { /** * Returns a stratified sample without replacement based on the fraction given on each stratum. - * @param col column that defines strata - * @param fractions sampling fraction for each stratum. If a stratum is not specified, we treat - * its fraction as zero. - * @param seed random seed - * @tparam T stratum type - * @return a new `DataFrame` that represents the stratified sample + * @param col + * column that defines strata + * @param fractions + * sampling fraction for each stratum. If a stratum is not specified, we treat its fraction as + * zero. + * @param seed + * random seed + * @tparam T + * stratum type + * @return + * a new `DataFrame` that represents the stratified sample * * @since 1.5.0 */ @@ -351,12 +382,17 @@ abstract class DataFrameStatFunctions[DS[U] <: Dataset[U, DS]] { /** * Returns a stratified sample without replacement based on the fraction given on each stratum. - * @param col column that defines strata - * @param fractions sampling fraction for each stratum. If a stratum is not specified, we treat - * its fraction as zero. - * @param seed random seed - * @tparam T stratum type - * @return a new `DataFrame` that represents the stratified sample + * @param col + * column that defines strata + * @param fractions + * sampling fraction for each stratum. If a stratum is not specified, we treat its fraction as + * zero. + * @param seed + * random seed + * @tparam T + * stratum type + * @return + * a new `DataFrame` that represents the stratified sample * * The stratified sample can be performed over multiple columns: * {{{ @@ -379,17 +415,21 @@ abstract class DataFrameStatFunctions[DS[U] <: Dataset[U, DS]] { */ def sampleBy[T](col: Column, fractions: Map[T, Double], seed: Long): DS[Row] - /** * (Java-specific) Returns a stratified sample without replacement based on the fraction given * on each stratum. * - * @param col column that defines strata - * @param fractions sampling fraction for each stratum. If a stratum is not specified, we treat - * its fraction as zero. - * @param seed random seed - * @tparam T stratum type - * @return a new `DataFrame` that represents the stratified sample + * @param col + * column that defines strata + * @param fractions + * sampling fraction for each stratum. If a stratum is not specified, we treat its fraction as + * zero. + * @param seed + * random seed + * @tparam T + * stratum type + * @return + * a new `DataFrame` that represents the stratified sample * @since 3.0.0 */ def sampleBy[T](col: Column, fractions: ju.Map[T, jl.Double], seed: Long): DS[Row] = { @@ -399,11 +439,16 @@ abstract class DataFrameStatFunctions[DS[U] <: Dataset[U, DS]] { /** * Builds a Count-min Sketch over a specified column. * - * @param colName name of the column over which the sketch is built - * @param depth depth of the sketch - * @param width width of the sketch - * @param seed random seed - * @return a `CountMinSketch` over column `colName` + * @param colName + * name of the column over which the sketch is built + * @param depth + * depth of the sketch + * @param width + * width of the sketch + * @param seed + * random seed + * @return + * a `CountMinSketch` over column `colName` * @since 2.0.0 */ def countMinSketch(colName: String, depth: Int, width: Int, seed: Int): CountMinSketch = { @@ -413,26 +458,39 @@ abstract class DataFrameStatFunctions[DS[U] <: Dataset[U, DS]] { /** * Builds a Count-min Sketch over a specified column. * - * @param colName name of the column over which the sketch is built - * @param eps relative error of the sketch - * @param confidence confidence of the sketch - * @param seed random seed - * @return a `CountMinSketch` over column `colName` + * @param colName + * name of the column over which the sketch is built + * @param eps + * relative error of the sketch + * @param confidence + * confidence of the sketch + * @param seed + * random seed + * @return + * a `CountMinSketch` over column `colName` * @since 2.0.0 */ def countMinSketch( - colName: String, eps: Double, confidence: Double, seed: Int): CountMinSketch = { + colName: String, + eps: Double, + confidence: Double, + seed: Int): CountMinSketch = { countMinSketch(Column(colName), eps, confidence, seed) } /** * Builds a Count-min Sketch over a specified column. * - * @param col the column over which the sketch is built - * @param depth depth of the sketch - * @param width width of the sketch - * @param seed random seed - * @return a `CountMinSketch` over column `colName` + * @param col + * the column over which the sketch is built + * @param depth + * depth of the sketch + * @param width + * width of the sketch + * @param seed + * random seed + * @return + * a `CountMinSketch` over column `colName` * @since 2.0.0 */ def countMinSketch(col: Column, depth: Int, width: Int, seed: Int): CountMinSketch = { @@ -444,29 +502,34 @@ abstract class DataFrameStatFunctions[DS[U] <: Dataset[U, DS]] { /** * Builds a Count-min Sketch over a specified column. * - * @param col the column over which the sketch is built - * @param eps relative error of the sketch - * @param confidence confidence of the sketch - * @param seed random seed - * @return a `CountMinSketch` over column `colName` + * @param col + * the column over which the sketch is built + * @param eps + * relative error of the sketch + * @param confidence + * confidence of the sketch + * @param seed + * random seed + * @return + * a `CountMinSketch` over column `colName` * @since 2.0.0 */ - def countMinSketch( - col: Column, - eps: Double, - confidence: Double, - seed: Int): CountMinSketch = withOrigin { - val cms = count_min_sketch(col, lit(eps), lit(confidence), lit(seed)) - val bytes: Array[Byte] = df.select(cms).as(BinaryEncoder).head() - CountMinSketch.readFrom(bytes) - } + def countMinSketch(col: Column, eps: Double, confidence: Double, seed: Int): CountMinSketch = + withOrigin { + val cms = count_min_sketch(col, lit(eps), lit(confidence), lit(seed)) + val bytes: Array[Byte] = df.select(cms).as(BinaryEncoder).head() + CountMinSketch.readFrom(bytes) + } /** * Builds a Bloom filter over a specified column. * - * @param colName name of the column over which the filter is built - * @param expectedNumItems expected number of items which will be put into the filter. - * @param fpp expected false positive probability of the filter. + * @param colName + * name of the column over which the filter is built + * @param expectedNumItems + * expected number of items which will be put into the filter. + * @param fpp + * expected false positive probability of the filter. * @since 2.0.0 */ def bloomFilter(colName: String, expectedNumItems: Long, fpp: Double): BloomFilter = { @@ -476,9 +539,12 @@ abstract class DataFrameStatFunctions[DS[U] <: Dataset[U, DS]] { /** * Builds a Bloom filter over a specified column. * - * @param col the column over which the filter is built - * @param expectedNumItems expected number of items which will be put into the filter. - * @param fpp expected false positive probability of the filter. + * @param col + * the column over which the filter is built + * @param expectedNumItems + * expected number of items which will be put into the filter. + * @param fpp + * expected false positive probability of the filter. * @since 2.0.0 */ def bloomFilter(col: Column, expectedNumItems: Long, fpp: Double): BloomFilter = { @@ -489,9 +555,12 @@ abstract class DataFrameStatFunctions[DS[U] <: Dataset[U, DS]] { /** * Builds a Bloom filter over a specified column. * - * @param colName name of the column over which the filter is built - * @param expectedNumItems expected number of items which will be put into the filter. - * @param numBits expected number of bits of the filter. + * @param colName + * name of the column over which the filter is built + * @param expectedNumItems + * expected number of items which will be put into the filter. + * @param numBits + * expected number of bits of the filter. * @since 2.0.0 */ def bloomFilter(colName: String, expectedNumItems: Long, numBits: Long): BloomFilter = { @@ -501,9 +570,12 @@ abstract class DataFrameStatFunctions[DS[U] <: Dataset[U, DS]] { /** * Builds a Bloom filter over a specified column. * - * @param col the column over which the filter is built - * @param expectedNumItems expected number of items which will be put into the filter. - * @param numBits expected number of bits of the filter. + * @param col + * the column over which the filter is built + * @param expectedNumItems + * expected number of items which will be put into the filter. + * @param numBits + * expected number of bits of the filter. * @since 2.0.0 */ def bloomFilter(col: Column, expectedNumItems: Long, numBits: Long): BloomFilter = withOrigin { diff --git a/sql/api/src/main/scala/org/apache/spark/sql/api/Dataset.scala b/sql/api/src/main/scala/org/apache/spark/sql/api/Dataset.scala index 49237ee7987a1..fb8b6f2f483a1 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/api/Dataset.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/api/Dataset.scala @@ -31,8 +31,8 @@ import org.apache.spark.util.ArrayImplicits._ import org.apache.spark.util.SparkClassUtils /** - * A Dataset is a strongly typed collection of domain-specific objects that can be transformed - * in parallel using functional or relational operations. Each Dataset also has an untyped view + * A Dataset is a strongly typed collection of domain-specific objects that can be transformed in + * parallel using functional or relational operations. Each Dataset also has an untyped view * called a `DataFrame`, which is a Dataset of [[org.apache.spark.sql.Row]]. * * Operations available on Datasets are divided into transformations and actions. Transformations @@ -40,29 +40,29 @@ import org.apache.spark.util.SparkClassUtils * return results. Example transformations include map, filter, select, and aggregate (`groupBy`). * Example actions count, show, or writing data out to file systems. * - * Datasets are "lazy", i.e. computations are only triggered when an action is invoked. Internally, - * a Dataset represents a logical plan that describes the computation required to produce the data. - * When an action is invoked, Spark's query optimizer optimizes the logical plan and generates a - * physical plan for efficient execution in a parallel and distributed manner. To explore the - * logical plan as well as optimized physical plan, use the `explain` function. + * Datasets are "lazy", i.e. computations are only triggered when an action is invoked. + * Internally, a Dataset represents a logical plan that describes the computation required to + * produce the data. When an action is invoked, Spark's query optimizer optimizes the logical plan + * and generates a physical plan for efficient execution in a parallel and distributed manner. To + * explore the logical plan as well as optimized physical plan, use the `explain` function. * - * To efficiently support domain-specific objects, an [[org.apache.spark.sql.Encoder]] is required. - * The encoder maps the domain specific type `T` to Spark's internal type system. For example, given - * a class `Person` with two fields, `name` (string) and `age` (int), an encoder is used to tell - * Spark to generate code at runtime to serialize the `Person` object into a binary structure. This - * binary structure often has much lower memory footprint as well as are optimized for efficiency - * in data processing (e.g. in a columnar format). To understand the internal binary representation - * for data, use the `schema` function. + * To efficiently support domain-specific objects, an [[org.apache.spark.sql.Encoder]] is + * required. The encoder maps the domain specific type `T` to Spark's internal type system. For + * example, given a class `Person` with two fields, `name` (string) and `age` (int), an encoder is + * used to tell Spark to generate code at runtime to serialize the `Person` object into a binary + * structure. This binary structure often has much lower memory footprint as well as are optimized + * for efficiency in data processing (e.g. in a columnar format). To understand the internal + * binary representation for data, use the `schema` function. * - * There are typically two ways to create a Dataset. The most common way is by pointing Spark - * to some files on storage systems, using the `read` function available on a `SparkSession`. + * There are typically two ways to create a Dataset. The most common way is by pointing Spark to + * some files on storage systems, using the `read` function available on a `SparkSession`. * {{{ * val people = spark.read.parquet("...").as[Person] // Scala * Dataset people = spark.read().parquet("...").as(Encoders.bean(Person.class)); // Java * }}} * - * Datasets can also be created through transformations available on existing Datasets. For example, - * the following creates a new Dataset by applying a filter on the existing one: + * Datasets can also be created through transformations available on existing Datasets. For + * example, the following creates a new Dataset by applying a filter on the existing one: * {{{ * val names = people.map(_.name) // in Scala; names is a Dataset[String] * Dataset names = people.map( @@ -71,8 +71,8 @@ import org.apache.spark.util.SparkClassUtils * * Dataset operations can also be untyped, through various domain-specific-language (DSL) * functions defined in: Dataset (this class), [[org.apache.spark.sql.Column]], and - * [[org.apache.spark.sql.functions]]. These operations are very similar to the operations available - * in the data frame abstraction in R or Python. + * [[org.apache.spark.sql.functions]]. These operations are very similar to the operations + * available in the data frame abstraction in R or Python. * * To select a column from the Dataset, use `apply` method in Scala and `col` in Java. * {{{ @@ -139,23 +139,20 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { def toDF(): DS[Row] /** - * Returns a new Dataset where each record has been mapped on to the specified type. The - * method used to map columns depend on the type of `U`: - *
    - *
  • When `U` is a class, fields for the class will be mapped to columns of the same name - * (case sensitivity is determined by `spark.sql.caseSensitive`).
  • - *
  • When `U` is a tuple, the columns will be mapped by ordinal (i.e. the first column will - * be assigned to `_1`).
  • - *
  • When `U` is a primitive type (i.e. String, Int, etc), then the first column of the - * `DataFrame` will be used.
  • + * Returns a new Dataset where each record has been mapped on to the specified type. The method + * used to map columns depend on the type of `U`:
    • When `U` is a class, fields for the + * class will be mapped to columns of the same name (case sensitivity is determined by + * `spark.sql.caseSensitive`).
    • When `U` is a tuple, the columns will be mapped by + * ordinal (i.e. the first column will be assigned to `_1`).
    • When `U` is a primitive + * type (i.e. String, Int, etc), then the first column of the `DataFrame` will be used.
    • *
    * - * If the schema of the Dataset does not match the desired `U` type, you can use `select` - * along with `alias` or `as` to rearrange or rename as required. + * If the schema of the Dataset does not match the desired `U` type, you can use `select` along + * with `alias` or `as` to rearrange or rename as required. * - * Note that `as[]` only changes the view of the data that is passed into typed operations, - * such as `map()`, and does not eagerly project away any columns that are not present in - * the specified class. + * Note that `as[]` only changes the view of the data that is passed into typed operations, such + * as `map()`, and does not eagerly project away any columns that are not present in the + * specified class. * * @group basic * @since 1.6.0 @@ -163,20 +160,17 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { def as[U: Encoder]: DS[U] /** - * Returns a new DataFrame where each row is reconciled to match the specified schema. Spark will: - *
      - *
    • Reorder columns and/or inner fields by name to match the specified schema.
    • - *
    • Project away columns and/or inner fields that are not needed by the specified schema. - * Missing columns and/or inner fields (present in the specified schema but not input DataFrame) - * lead to failures.
    • - *
    • Cast the columns and/or inner fields to match the data types in the specified schema, if - * the types are compatible, e.g., numeric to numeric (error if overflows), but not string to - * int.
    • - *
    • Carry over the metadata from the specified schema, while the columns and/or inner fields - * still keep their own metadata if not overwritten by the specified schema.
    • - *
    • Fail if the nullability is not compatible. For example, the column and/or inner field is - * nullable but the specified schema requires them to be not nullable.
    • - *
    + * Returns a new DataFrame where each row is reconciled to match the specified schema. Spark + * will:
    • Reorder columns and/or inner fields by name to match the specified + * schema.
    • Project away columns and/or inner fields that are not needed by the + * specified schema. Missing columns and/or inner fields (present in the specified schema but + * not input DataFrame) lead to failures.
    • Cast the columns and/or inner fields to match + * the data types in the specified schema, if the types are compatible, e.g., numeric to numeric + * (error if overflows), but not string to int.
    • Carry over the metadata from the + * specified schema, while the columns and/or inner fields still keep their own metadata if not + * overwritten by the specified schema.
    • Fail if the nullability is not compatible. For + * example, the column and/or inner field is nullable but the specified schema requires them to + * be not nullable.
    * * @group basic * @since 3.4.0 @@ -229,16 +223,12 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { /** * Prints the plans (logical and physical) with a format specified by a given explain mode. * - * @param mode specifies the expected output format of plans. - *
      - *
    • `simple` Print only a physical plan.
    • - *
    • `extended`: Print both logical and physical plans.
    • - *
    • `codegen`: Print a physical plan and generated codes if they are - * available.
    • - *
    • `cost`: Print a logical plan and statistics if they are available.
    • - *
    • `formatted`: Split explain output into two sections: a physical plan outline - * and node details.
    • - *
    + * @param mode + * specifies the expected output format of plans.
    • `simple` Print only a physical + * plan.
    • `extended`: Print both logical and physical plans.
    • `codegen`: Print + * a physical plan and generated codes if they are available.
    • `cost`: Print a logical + * plan and statistics if they are available.
    • `formatted`: Split explain output into + * two sections: a physical plan outline and node details.
    * @group basic * @since 3.0.0 */ @@ -247,7 +237,8 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { /** * Prints the plans (logical and physical) to the console for debugging purposes. * - * @param extended default `false`. If `false`, prints only the physical plan. + * @param extended + * default `false`. If `false`, prints only the physical plan. * @group basic * @since 1.6.0 */ @@ -284,8 +275,8 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { def columns: Array[String] = schema.fields.map(_.name) /** - * Returns true if the `collect` and `take` methods can be run locally - * (without any Spark executors). + * Returns true if the `collect` and `take` methods can be run locally (without any Spark + * executors). * * @group basic * @since 1.6.0 @@ -301,12 +292,11 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { def isEmpty: Boolean /** - * Returns true if this Dataset contains one or more sources that continuously - * return data as it arrives. A Dataset that reads data from a streaming source - * must be executed as a `StreamingQuery` using the `start()` method in - * `DataStreamWriter`. Methods that return a single answer, e.g. `count()` or - * `collect()`, will throw an [[org.apache.spark.sql.AnalysisException]] when there is a - * streaming source present. + * Returns true if this Dataset contains one or more sources that continuously return data as it + * arrives. A Dataset that reads data from a streaming source must be executed as a + * `StreamingQuery` using the `start()` method in `DataStreamWriter`. Methods that return a + * single answer, e.g. `count()` or `collect()`, will throw an + * [[org.apache.spark.sql.AnalysisException]] when there is a streaming source present. * * @group streaming * @since 2.0.0 @@ -314,9 +304,9 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { def isStreaming: Boolean /** - * Eagerly checkpoint a Dataset and return the new Dataset. Checkpointing can be used to truncate - * the logical plan of this Dataset, which is especially useful in iterative algorithms where the - * plan may grow exponentially. It will be saved to files inside the checkpoint + * Eagerly checkpoint a Dataset and return the new Dataset. Checkpointing can be used to + * truncate the logical plan of this Dataset, which is especially useful in iterative algorithms + * where the plan may grow exponentially. It will be saved to files inside the checkpoint * directory set with `SparkContext#setCheckpointDir`. * * @group basic @@ -327,23 +317,25 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { /** * Returns a checkpointed version of this Dataset. Checkpointing can be used to truncate the * logical plan of this Dataset, which is especially useful in iterative algorithms where the - * plan may grow exponentially. It will be saved to files inside the checkpoint - * directory set with `SparkContext#setCheckpointDir`. - * - * @param eager Whether to checkpoint this dataframe immediately - * @note When checkpoint is used with eager = false, the final data that is checkpointed after - * the first action may be different from the data that was used during the job due to - * non-determinism of the underlying operation and retries. If checkpoint is used to achieve - * saving a deterministic snapshot of the data, eager = true should be used. Otherwise, - * it is only deterministic after the first execution, after the checkpoint was finalized. + * plan may grow exponentially. It will be saved to files inside the checkpoint directory set + * with `SparkContext#setCheckpointDir`. + * + * @param eager + * Whether to checkpoint this dataframe immediately + * @note + * When checkpoint is used with eager = false, the final data that is checkpointed after the + * first action may be different from the data that was used during the job due to + * non-determinism of the underlying operation and retries. If checkpoint is used to achieve + * saving a deterministic snapshot of the data, eager = true should be used. Otherwise, it is + * only deterministic after the first execution, after the checkpoint was finalized. * @group basic * @since 2.1.0 */ def checkpoint(eager: Boolean): DS[T] = checkpoint(eager = eager, reliableCheckpoint = true) /** - * Eagerly locally checkpoints a Dataset and return the new Dataset. Checkpointing can be - * used to truncate the logical plan of this Dataset, which is especially useful in iterative + * Eagerly locally checkpoints a Dataset and return the new Dataset. Checkpointing can be used + * to truncate the logical plan of this Dataset, which is especially useful in iterative * algorithms where the plan may grow exponentially. Local checkpoints are written to executor * storage and despite potentially faster they are unreliable and may compromise job completion. * @@ -353,32 +345,33 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { def localCheckpoint(): DS[T] = checkpoint(eager = true, reliableCheckpoint = false) /** - * Locally checkpoints a Dataset and return the new Dataset. Checkpointing can be used to truncate - * the logical plan of this Dataset, which is especially useful in iterative algorithms where the - * plan may grow exponentially. Local checkpoints are written to executor storage and despite - * potentially faster they are unreliable and may compromise job completion. + * Locally checkpoints a Dataset and return the new Dataset. Checkpointing can be used to + * truncate the logical plan of this Dataset, which is especially useful in iterative algorithms + * where the plan may grow exponentially. Local checkpoints are written to executor storage and + * despite potentially faster they are unreliable and may compromise job completion. * - * @param eager Whether to checkpoint this dataframe immediately - * @note When checkpoint is used with eager = false, the final data that is checkpointed after - * the first action may be different from the data that was used during the job due to - * non-determinism of the underlying operation and retries. If checkpoint is used to achieve - * saving a deterministic snapshot of the data, eager = true should be used. Otherwise, - * it is only deterministic after the first execution, after the checkpoint was finalized. + * @param eager + * Whether to checkpoint this dataframe immediately + * @note + * When checkpoint is used with eager = false, the final data that is checkpointed after the + * first action may be different from the data that was used during the job due to + * non-determinism of the underlying operation and retries. If checkpoint is used to achieve + * saving a deterministic snapshot of the data, eager = true should be used. Otherwise, it is + * only deterministic after the first execution, after the checkpoint was finalized. * @group basic * @since 2.3.0 */ - def localCheckpoint(eager: Boolean): DS[T] = checkpoint( - eager = eager, - reliableCheckpoint = false - ) + def localCheckpoint(eager: Boolean): DS[T] = + checkpoint(eager = eager, reliableCheckpoint = false) /** * Returns a checkpointed version of this Dataset. * - * @param eager Whether to checkpoint this dataframe immediately - * @param reliableCheckpoint Whether to create a reliable checkpoint saved to files inside the - * checkpoint directory. If false creates a local checkpoint using - * the caching subsystem + * @param eager + * Whether to checkpoint this dataframe immediately + * @param reliableCheckpoint + * Whether to create a reliable checkpoint saved to files inside the checkpoint directory. If + * false creates a local checkpoint using the caching subsystem */ protected def checkpoint(eager: Boolean, reliableCheckpoint: Boolean): DS[T] @@ -386,23 +379,22 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { * Defines an event time watermark for this [[Dataset]]. A watermark tracks a point in time * before which we assume no more late data is going to arrive. * - * Spark will use this watermark for several purposes: - *
      - *
    • To know when a given time window aggregation can be finalized and thus can be emitted - * when using output modes that do not allow updates.
    • - *
    • To minimize the amount of state that we need to keep for on-going aggregations, - * `mapGroupsWithState` and `dropDuplicates` operators.
    • - *
    - * The current watermark is computed by looking at the `MAX(eventTime)` seen across - * all of the partitions in the query minus a user specified `delayThreshold`. Due to the cost - * of coordinating this value across partitions, the actual watermark used is only guaranteed - * to be at least `delayThreshold` behind the actual event time. In some cases we may still - * process records that arrive more than `delayThreshold` late. - * - * @param eventTime the name of the column that contains the event time of the row. - * @param delayThreshold the minimum delay to wait to data to arrive late, relative to the latest - * record that has been processed in the form of an interval - * (e.g. "1 minute" or "5 hours"). NOTE: This should not be negative. + * Spark will use this watermark for several purposes:
    • To know when a given time window + * aggregation can be finalized and thus can be emitted when using output modes that do not + * allow updates.
    • To minimize the amount of state that we need to keep for on-going + * aggregations, `mapGroupsWithState` and `dropDuplicates` operators.
    The current + * watermark is computed by looking at the `MAX(eventTime)` seen across all of the partitions in + * the query minus a user specified `delayThreshold`. Due to the cost of coordinating this value + * across partitions, the actual watermark used is only guaranteed to be at least + * `delayThreshold` behind the actual event time. In some cases we may still process records + * that arrive more than `delayThreshold` late. + * + * @param eventTime + * the name of the column that contains the event time of the row. + * @param delayThreshold + * the minimum delay to wait to data to arrive late, relative to the latest record that has + * been processed in the form of an interval (e.g. "1 minute" or "5 hours"). NOTE: This should + * not be negative. * @group streaming * @since 2.1.0 */ @@ -410,7 +402,7 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { // defined on a derived column cannot referenced elsewhere in the plan. def withWatermark(eventTime: String, delayThreshold: String): DS[T] - /** + /** * Displays the Dataset in a tabular form. Strings more than 20 characters will be truncated, * and all cells will be aligned right. For example: * {{{ @@ -422,7 +414,8 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { * 1984 04 0.450090 0.483521 * }}} * - * @param numRows Number of rows to show + * @param numRows + * Number of rows to show * * @group action * @since 1.6.0 @@ -430,8 +423,8 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { def show(numRows: Int): Unit = show(numRows, truncate = true) /** - * Displays the top 20 rows of Dataset in a tabular form. Strings more than 20 characters - * will be truncated, and all cells will be aligned right. + * Displays the top 20 rows of Dataset in a tabular form. Strings more than 20 characters will + * be truncated, and all cells will be aligned right. * * @group action * @since 1.6.0 @@ -441,8 +434,9 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { /** * Displays the top 20 rows of Dataset in a tabular form. * - * @param truncate Whether truncate long strings. If true, strings more than 20 characters will - * be truncated and all cells will be aligned right + * @param truncate + * Whether truncate long strings. If true, strings more than 20 characters will be truncated + * and all cells will be aligned right * * @group action * @since 1.6.0 @@ -459,9 +453,11 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { * 1983 03 0.410516 0.442194 * 1984 04 0.450090 0.483521 * }}} - * @param numRows Number of rows to show - * @param truncate Whether truncate long strings. If true, strings more than 20 characters will - * be truncated and all cells will be aligned right + * @param numRows + * Number of rows to show + * @param truncate + * Whether truncate long strings. If true, strings more than 20 characters will be truncated + * and all cells will be aligned right * * @group action * @since 1.6.0 @@ -480,9 +476,11 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { * 1984 04 0.450090 0.483521 * }}} * - * @param numRows Number of rows to show - * @param truncate If set to more than 0, truncates strings to `truncate` characters and - * all cells will be aligned right. + * @param numRows + * Number of rows to show + * @param truncate + * If set to more than 0, truncates strings to `truncate` characters and all cells will be + * aligned right. * @group action * @since 1.6.0 */ @@ -499,7 +497,8 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { * 1984 04 0.450090 0.483521 * }}} * - * If `vertical` enabled, this command prints output rows vertically (one line per column value)? + * If `vertical` enabled, this command prints output rows vertically (one line per column + * value)? * * {{{ * -RECORD 0------------------- @@ -529,10 +528,13 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { * AVG('Adj Close) | 0.483521 * }}} * - * @param numRows Number of rows to show - * @param truncate If set to more than 0, truncates strings to `truncate` characters and - * all cells will be aligned right. - * @param vertical If set to true, prints output rows vertically (one line per column value). + * @param numRows + * Number of rows to show + * @param truncate + * If set to more than 0, truncates strings to `truncate` characters and all cells will be + * aligned right. + * @param vertical + * If set to true, prints output rows vertically (one line per column value). * @group action * @since 2.3.0 */ @@ -568,13 +570,14 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { * * Behaves as an INNER JOIN and requires a subsequent join predicate. * - * @param right Right side of the join operation. + * @param right + * Right side of the join operation. * @group untypedrel * @since 2.0.0 */ def join(right: DS[_]): DS[Row] - /** + /** * Inner equi-join with another `DataFrame` using the given column. * * Different from other join functions, the join column will only appear once in the output, @@ -585,12 +588,15 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { * df1.join(df2, "user_id") * }}} * - * @param right Right side of the join operation. - * @param usingColumn Name of the column to join on. This column must exist on both sides. + * @param right + * Right side of the join operation. + * @param usingColumn + * Name of the column to join on. This column must exist on both sides. * - * @note If you perform a self-join using this function without aliasing the input - * `DataFrame`s, you will NOT be able to reference any columns after the join, since - * there is no way to disambiguate which side of the join you would like to reference. + * @note + * If you perform a self-join using this function without aliasing the input `DataFrame`s, you + * will NOT be able to reference any columns after the join, since there is no way to + * disambiguate which side of the join you would like to reference. * * @group untypedrel * @since 2.0.0 @@ -603,8 +609,10 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { * (Java-specific) Inner equi-join with another `DataFrame` using the given columns. See the * Scala-specific overload for more details. * - * @param right Right side of the join operation. - * @param usingColumns Names of the columns to join on. This columns must exist on both sides. + * @param right + * Right side of the join operation. + * @param usingColumns + * Names of the columns to join on. This columns must exist on both sides. * * @group untypedrel * @since 3.4.0 @@ -624,12 +632,15 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { * df1.join(df2, Seq("user_id", "user_name")) * }}} * - * @param right Right side of the join operation. - * @param usingColumns Names of the columns to join on. This columns must exist on both sides. + * @param right + * Right side of the join operation. + * @param usingColumns + * Names of the columns to join on. This columns must exist on both sides. * - * @note If you perform a self-join using this function without aliasing the input - * `DataFrame`s, you will NOT be able to reference any columns after the join, since - * there is no way to disambiguate which side of the join you would like to reference. + * @note + * If you perform a self-join using this function without aliasing the input `DataFrame`s, you + * will NOT be able to reference any columns after the join, since there is no way to + * disambiguate which side of the join you would like to reference. * * @group untypedrel * @since 2.0.0 @@ -639,23 +650,27 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { } /** - * Equi-join with another `DataFrame` using the given column. A cross join with a predicate - * is specified as an inner join. If you would explicitly like to perform a cross join use the + * Equi-join with another `DataFrame` using the given column. A cross join with a predicate is + * specified as an inner join. If you would explicitly like to perform a cross join use the * `crossJoin` method. * * Different from other join functions, the join column will only appear once in the output, * i.e. similar to SQL's `JOIN USING` syntax. * - * @param right Right side of the join operation. - * @param usingColumn Name of the column to join on. This column must exist on both sides. - * @param joinType Type of join to perform. Default `inner`. Must be one of: - * `inner`, `cross`, `outer`, `full`, `fullouter`, `full_outer`, `left`, - * `leftouter`, `left_outer`, `right`, `rightouter`, `right_outer`, - * `semi`, `leftsemi`, `left_semi`, `anti`, `leftanti`, `left_anti`. - * - * @note If you perform a self-join using this function without aliasing the input - * `DataFrame`s, you will NOT be able to reference any columns after the join, since - * there is no way to disambiguate which side of the join you would like to reference. + * @param right + * Right side of the join operation. + * @param usingColumn + * Name of the column to join on. This column must exist on both sides. + * @param joinType + * Type of join to perform. Default `inner`. Must be one of: `inner`, `cross`, `outer`, + * `full`, `fullouter`, `full_outer`, `left`, `leftouter`, `left_outer`, `right`, + * `rightouter`, `right_outer`, `semi`, `leftsemi`, `left_semi`, `anti`, `leftanti`, + * `left_anti`. + * + * @note + * If you perform a self-join using this function without aliasing the input `DataFrame`s, you + * will NOT be able to reference any columns after the join, since there is no way to + * disambiguate which side of the join you would like to reference. * * @group untypedrel * @since 3.4.0 @@ -668,12 +683,15 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { * (Java-specific) Equi-join with another `DataFrame` using the given columns. See the * Scala-specific overload for more details. * - * @param right Right side of the join operation. - * @param usingColumns Names of the columns to join on. This columns must exist on both sides. - * @param joinType Type of join to perform. Default `inner`. Must be one of: - * `inner`, `cross`, `outer`, `full`, `fullouter`, `full_outer`, `left`, - * `leftouter`, `left_outer`, `right`, `rightouter`, `right_outer`, - * `semi`, `leftsemi`, `left_semi`, `anti`, `leftanti`, `left_anti`. + * @param right + * Right side of the join operation. + * @param usingColumns + * Names of the columns to join on. This columns must exist on both sides. + * @param joinType + * Type of join to perform. Default `inner`. Must be one of: `inner`, `cross`, `outer`, + * `full`, `fullouter`, `full_outer`, `left`, `leftouter`, `left_outer`, `right`, + * `rightouter`, `right_outer`, `semi`, `leftsemi`, `left_semi`, `anti`, `leftanti`, + * `left_anti`. * * @group untypedrel * @since 3.4.0 @@ -690,16 +708,20 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { * Different from other join functions, the join columns will only appear once in the output, * i.e. similar to SQL's `JOIN USING` syntax. * - * @param right Right side of the join operation. - * @param usingColumns Names of the columns to join on. This columns must exist on both sides. - * @param joinType Type of join to perform. Default `inner`. Must be one of: - * `inner`, `cross`, `outer`, `full`, `fullouter`, `full_outer`, `left`, - * `leftouter`, `left_outer`, `right`, `rightouter`, `right_outer`, - * `semi`, `leftsemi`, `left_semi`, `anti`, `leftanti`, `left_anti`. - * - * @note If you perform a self-join using this function without aliasing the input - * `DataFrame`s, you will NOT be able to reference any columns after the join, since - * there is no way to disambiguate which side of the join you would like to reference. + * @param right + * Right side of the join operation. + * @param usingColumns + * Names of the columns to join on. This columns must exist on both sides. + * @param joinType + * Type of join to perform. Default `inner`. Must be one of: `inner`, `cross`, `outer`, + * `full`, `fullouter`, `full_outer`, `left`, `leftouter`, `left_outer`, `right`, + * `rightouter`, `right_outer`, `semi`, `leftsemi`, `left_semi`, `anti`, `leftanti`, + * `left_anti`. + * + * @note + * If you perform a self-join using this function without aliasing the input `DataFrame`s, you + * will NOT be able to reference any columns after the join, since there is no way to + * disambiguate which side of the join you would like to reference. * * @group untypedrel * @since 2.0.0 @@ -722,8 +744,8 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { join(right, joinExprs, "inner") /** - * Join with another `DataFrame`, using the given join expression. The following performs - * a full outer join between `df1` and `df2`. + * Join with another `DataFrame`, using the given join expression. The following performs a full + * outer join between `df1` and `df2`. * * {{{ * // Scala: @@ -735,12 +757,15 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { * df1.join(df2, col("df1Key").equalTo(col("df2Key")), "outer"); * }}} * - * @param right Right side of the join. - * @param joinExprs Join expression. - * @param joinType Type of join to perform. Default `inner`. Must be one of: - * `inner`, `cross`, `outer`, `full`, `fullouter`, `full_outer`, `left`, - * `leftouter`, `left_outer`, `right`, `rightouter`, `right_outer`, - * `semi`, `leftsemi`, `left_semi`, `anti`, `leftanti`, `left_anti`. + * @param right + * Right side of the join. + * @param joinExprs + * Join expression. + * @param joinType + * Type of join to perform. Default `inner`. Must be one of: `inner`, `cross`, `outer`, + * `full`, `fullouter`, `full_outer`, `left`, `leftouter`, `left_outer`, `right`, + * `rightouter`, `right_outer`, `semi`, `leftsemi`, `left_semi`, `anti`, `leftanti`, + * `left_anti`. * * @group untypedrel * @since 2.0.0 @@ -750,41 +775,47 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { /** * Explicit cartesian join with another `DataFrame`. * - * @param right Right side of the join operation. - * @note Cartesian joins are very expensive without an extra filter that can be pushed down. + * @param right + * Right side of the join operation. + * @note + * Cartesian joins are very expensive without an extra filter that can be pushed down. * @group untypedrel * @since 2.1.0 */ def crossJoin(right: DS[_]): DS[Row] /** - * Joins this Dataset returning a `Tuple2` for each pair where `condition` evaluates to - * true. + * Joins this Dataset returning a `Tuple2` for each pair where `condition` evaluates to true. * - * This is similar to the relation `join` function with one important difference in the - * result schema. Since `joinWith` preserves objects present on either side of the join, the - * result schema is similarly nested into a tuple under the column names `_1` and `_2`. + * This is similar to the relation `join` function with one important difference in the result + * schema. Since `joinWith` preserves objects present on either side of the join, the result + * schema is similarly nested into a tuple under the column names `_1` and `_2`. * * This type of join can be useful both for preserving type-safety with the original object - * types as well as working with relational data where either side of the join has column - * names in common. - * - * @param other Right side of the join. - * @param condition Join expression. - * @param joinType Type of join to perform. Default `inner`. Must be one of: - * `inner`, `cross`, `outer`, `full`, `fullouter`,`full_outer`, `left`, - * `leftouter`, `left_outer`, `right`, `rightouter`, `right_outer`. + * types as well as working with relational data where either side of the join has column names + * in common. + * + * @param other + * Right side of the join. + * @param condition + * Join expression. + * @param joinType + * Type of join to perform. Default `inner`. Must be one of: `inner`, `cross`, `outer`, + * `full`, `fullouter`,`full_outer`, `left`, `leftouter`, `left_outer`, `right`, `rightouter`, + * `right_outer`. * @group typedrel * @since 1.6.0 */ def joinWith[U](other: DS[U], condition: Column, joinType: String): DS[(T, U)] /** - * Using inner equi-join to join this Dataset returning a `Tuple2` for each pair - * where `condition` evaluates to true. + * Using inner equi-join to join this Dataset returning a `Tuple2` for each pair where + * `condition` evaluates to true. * - * @param other Right side of the join. - * @param condition Join expression. + * @param other + * Right side of the join. + * @param condition + * Join expression. * @group typedrel * @since 1.6.0 */ @@ -804,7 +835,7 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { */ @scala.annotation.varargs def sortWithinPartitions(sortCol: String, sortCols: String*): DS[T] = { - sortWithinPartitions((sortCol +: sortCols).map(Column(_)) : _*) + sortWithinPartitions((sortCol +: sortCols).map(Column(_)): _*) } /** @@ -834,7 +865,7 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { */ @scala.annotation.varargs def sort(sortCol: String, sortCols: String*): DS[T] = { - sort((sortCol +: sortCols).map(Column(_)) : _*) + sort((sortCol +: sortCols).map(Column(_)): _*) } /** @@ -852,28 +883,28 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { } /** - * Returns a new Dataset sorted by the given expressions. - * This is an alias of the `sort` function. + * Returns a new Dataset sorted by the given expressions. This is an alias of the `sort` + * function. * * @group typedrel * @since 2.0.0 */ @scala.annotation.varargs - def orderBy(sortCol: String, sortCols: String*): DS[T] = sort(sortCol, sortCols : _*) + def orderBy(sortCol: String, sortCols: String*): DS[T] = sort(sortCol, sortCols: _*) /** - * Returns a new Dataset sorted by the given expressions. - * This is an alias of the `sort` function. + * Returns a new Dataset sorted by the given expressions. This is an alias of the `sort` + * function. * * @group typedrel * @since 2.0.0 */ @scala.annotation.varargs - def orderBy(sortExprs: Column*): DS[T] = sort(sortExprs : _*) + def orderBy(sortExprs: Column*): DS[T] = sort(sortExprs: _*) /** - * Specifies some hint on the current Dataset. As an example, the following code specifies - * that one of the plan can be broadcasted: + * Specifies some hint on the current Dataset. As an example, the following code specifies that + * one of the plan can be broadcasted: * * {{{ * df1.join(df2.hint("broadcast")) @@ -886,9 +917,11 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { * df1.hint("rebalance", 10) * }}} * - * @param name the name of the hint - * @param parameters the parameters of the hint, all the parameters should be a `Column` or - * `Expression` or `Symbol` or could be converted into a `Literal` + * @param name + * the name of the hint + * @param parameters + * the parameters of the hint, all the parameters should be a `Column` or `Expression` or + * `Symbol` or could be converted into a `Literal` * @group basic * @since 2.2.0 */ @@ -898,7 +931,8 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { /** * Selects column based on the column name and returns it as a [[org.apache.spark.sql.Column]]. * - * @note The column name can also reference to a nested column like `a.b`. + * @note + * The column name can also reference to a nested column like `a.b`. * @group untypedrel * @since 2.0.0 */ @@ -907,7 +941,8 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { /** * Selects column based on the column name and returns it as a [[org.apache.spark.sql.Column]]. * - * @note The column name can also reference to a nested column like `a.b`. + * @note + * The column name can also reference to a nested column like `a.b`. * @group untypedrel * @since 2.0.0 */ @@ -979,8 +1014,8 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { def select(cols: Column*): DS[Row] /** - * Selects a set of columns. This is a variant of `select` that can only select - * existing columns using column names (i.e. cannot construct expressions). + * Selects a set of columns. This is a variant of `select` that can only select existing columns + * using column names (i.e. cannot construct expressions). * * {{{ * // The following two are equivalent: @@ -995,8 +1030,7 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { def select(col: String, cols: String*): DS[Row] = select((col +: cols).map(Column(_)): _*) /** - * Selects a set of SQL expressions. This is a variant of `select` that accepts - * SQL expressions. + * Selects a set of SQL expressions. This is a variant of `select` that accepts SQL expressions. * * {{{ * // The following are equivalent: @@ -1026,8 +1060,8 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { /** * Internal helper function for building typed selects that return tuples. For simplicity and - * code reuse, we do this without the help of the type system and then use helper functions - * that cast appropriately for the user facing interface. + * code reuse, we do this without the help of the type system and then use helper functions that + * cast appropriately for the user facing interface. */ protected def selectUntyped(columns: TypedColumn[_, _]*): DS[_] @@ -1109,8 +1143,8 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { filter(functions.expr(conditionExpr)) /** - * (Scala-specific) - * Returns a new Dataset that only contains elements where `func` returns `true`. + * (Scala-specific) Returns a new Dataset that only contains elements where `func` returns + * `true`. * * @group typedrel * @since 1.6.0 @@ -1118,8 +1152,8 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { def filter(func: T => Boolean): DS[T] /** - * (Java-specific) - * Returns a new Dataset that only contains elements where `func` returns `true`. + * (Java-specific) Returns a new Dataset that only contains elements where `func` returns + * `true`. * * @group typedrel * @since 1.6.0 @@ -1172,11 +1206,11 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { def groupBy(cols: Column*): RGD /** - * Groups the Dataset using the specified columns, so that we can run aggregation on them. - * See [[RelationalGroupedDataset]] for all the available aggregate functions. + * Groups the Dataset using the specified columns, so that we can run aggregation on them. See + * [[RelationalGroupedDataset]] for all the available aggregate functions. * - * This is a variant of groupBy that can only group by existing columns using column names - * (i.e. cannot construct expressions). + * This is a variant of groupBy that can only group by existing columns using column names (i.e. + * cannot construct expressions). * * {{{ * // Compute the average for all numeric columns grouped by department. @@ -1196,9 +1230,9 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { def groupBy(col1: String, cols: String*): RGD = groupBy((col1 +: cols).map(col): _*) /** - * Create a multi-dimensional rollup for the current Dataset using the specified columns, - * so we can run aggregation on them. - * See [[RelationalGroupedDataset]] for all the available aggregate functions. + * Create a multi-dimensional rollup for the current Dataset using the specified columns, so we + * can run aggregation on them. See [[RelationalGroupedDataset]] for all the available aggregate + * functions. * * {{{ * // Compute the average for all numeric columns rolled up by department and group. @@ -1218,12 +1252,12 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { def rollup(cols: Column*): RGD /** - * Create a multi-dimensional rollup for the current Dataset using the specified columns, - * so we can run aggregation on them. - * See [[RelationalGroupedDataset]] for all the available aggregate functions. + * Create a multi-dimensional rollup for the current Dataset using the specified columns, so we + * can run aggregation on them. See [[RelationalGroupedDataset]] for all the available aggregate + * functions. * - * This is a variant of rollup that can only group by existing columns using column names - * (i.e. cannot construct expressions). + * This is a variant of rollup that can only group by existing columns using column names (i.e. + * cannot construct expressions). * * {{{ * // Compute the average for all numeric columns rolled up by department and group. @@ -1243,9 +1277,9 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { def rollup(col1: String, cols: String*): RGD = rollup((col1 +: cols).map(col): _*) /** - * Create a multi-dimensional cube for the current Dataset using the specified columns, - * so we can run aggregation on them. - * See [[RelationalGroupedDataset]] for all the available aggregate functions. + * Create a multi-dimensional cube for the current Dataset using the specified columns, so we + * can run aggregation on them. See [[RelationalGroupedDataset]] for all the available aggregate + * functions. * * {{{ * // Compute the average for all numeric columns cubed by department and group. @@ -1265,12 +1299,12 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { def cube(cols: Column*): RGD /** - * Create a multi-dimensional cube for the current Dataset using the specified columns, - * so we can run aggregation on them. - * See [[RelationalGroupedDataset]] for all the available aggregate functions. + * Create a multi-dimensional cube for the current Dataset using the specified columns, so we + * can run aggregation on them. See [[RelationalGroupedDataset]] for all the available aggregate + * functions. * - * This is a variant of cube that can only group by existing columns using column names - * (i.e. cannot construct expressions). + * This is a variant of cube that can only group by existing columns using column names (i.e. + * cannot construct expressions). * * {{{ * // Compute the average for all numeric columns cubed by department and group. @@ -1290,9 +1324,9 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { def cube(col1: String, cols: String*): RGD = cube((col1 +: cols).map(col): _*) /** - * Create multi-dimensional aggregation for the current Dataset using the specified grouping sets, - * so we can run aggregation on them. - * See [[RelationalGroupedDataset]] for all the available aggregate functions. + * Create multi-dimensional aggregation for the current Dataset using the specified grouping + * sets, so we can run aggregation on them. See [[RelationalGroupedDataset]] for all the + * available aggregate functions. * * {{{ * // Compute the average for all numeric columns group by specific grouping sets. @@ -1367,9 +1401,8 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { def agg(expr: Column, exprs: Column*): DS[Row] = groupBy().agg(expr, exprs: _*) /** - * (Scala-specific) - * Reduces the elements of this Dataset using the specified binary function. The given `func` - * must be commutative and associative or the result may be non-deterministic. + * (Scala-specific) Reduces the elements of this Dataset using the specified binary function. + * The given `func` must be commutative and associative or the result may be non-deterministic. * * @group action * @since 1.6.0 @@ -1377,9 +1410,8 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { def reduce(func: (T, T) => T): T /** - * (Java-specific) - * Reduces the elements of this Dataset using the specified binary function. The given `func` - * must be commutative and associative or the result may be non-deterministic. + * (Java-specific) Reduces the elements of this Dataset using the specified binary function. The + * given `func` must be commutative and associative or the result may be non-deterministic. * * @group action * @since 1.6.0 @@ -1387,14 +1419,14 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { def reduce(func: ReduceFunction[T]): T = reduce(ToScalaUDF(func)) /** - * Unpivot a DataFrame from wide format to long format, optionally leaving identifier columns set. - * This is the reverse to `groupBy(...).pivot(...).agg(...)`, except for the aggregation, + * Unpivot a DataFrame from wide format to long format, optionally leaving identifier columns + * set. This is the reverse to `groupBy(...).pivot(...).agg(...)`, except for the aggregation, * which cannot be reversed. * - * This function is useful to massage a DataFrame into a format where some - * columns are identifier columns ("ids"), while all other columns ("values") - * are "unpivoted" to the rows, leaving just two non-id columns, named as given - * by `variableColumnName` and `valueColumnName`. + * This function is useful to massage a DataFrame into a format where some columns are + * identifier columns ("ids"), while all other columns ("values") are "unpivoted" to the rows, + * leaving just two non-id columns, named as given by `variableColumnName` and + * `valueColumnName`. * * {{{ * val df = Seq((1, 11, 12L), (2, 21, 22L)).toDF("id", "int", "long") @@ -1424,18 +1456,22 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { * // |-- value: long (nullable = true) * }}} * - * When no "id" columns are given, the unpivoted DataFrame consists of only the - * "variable" and "value" columns. + * When no "id" columns are given, the unpivoted DataFrame consists of only the "variable" and + * "value" columns. * * All "value" columns must share a least common data type. Unless they are the same data type, - * all "value" columns are cast to the nearest common data type. For instance, - * types `IntegerType` and `LongType` are cast to `LongType`, while `IntegerType` and `StringType` - * do not have a common data type and `unpivot` fails with an `AnalysisException`. - * - * @param ids Id columns - * @param values Value columns to unpivot - * @param variableColumnName Name of the variable column - * @param valueColumnName Name of the value column + * all "value" columns are cast to the nearest common data type. For instance, types + * `IntegerType` and `LongType` are cast to `LongType`, while `IntegerType` and `StringType` do + * not have a common data type and `unpivot` fails with an `AnalysisException`. + * + * @param ids + * Id columns + * @param values + * Value columns to unpivot + * @param variableColumnName + * Name of the variable column + * @param valueColumnName + * Name of the value column * @group untypedrel * @since 3.4.0 */ @@ -1446,37 +1482,43 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { valueColumnName: String): DS[Row] /** - * Unpivot a DataFrame from wide format to long format, optionally leaving identifier columns set. - * This is the reverse to `groupBy(...).pivot(...).agg(...)`, except for the aggregation, + * Unpivot a DataFrame from wide format to long format, optionally leaving identifier columns + * set. This is the reverse to `groupBy(...).pivot(...).agg(...)`, except for the aggregation, * which cannot be reversed. * - * @see `org.apache.spark.sql.Dataset.unpivot(Array, Array, String, String)` + * @see + * `org.apache.spark.sql.Dataset.unpivot(Array, Array, String, String)` * - * This is equivalent to calling `Dataset#unpivot(Array, Array, String, String)` - * where `values` is set to all non-id columns that exist in the DataFrame. + * This is equivalent to calling `Dataset#unpivot(Array, Array, String, String)` where `values` + * is set to all non-id columns that exist in the DataFrame. * - * @param ids Id columns - * @param variableColumnName Name of the variable column - * @param valueColumnName Name of the value column + * @param ids + * Id columns + * @param variableColumnName + * Name of the variable column + * @param valueColumnName + * Name of the value column * * @group untypedrel * @since 3.4.0 */ - def unpivot( - ids: Array[Column], - variableColumnName: String, - valueColumnName: String): DS[Row] + def unpivot(ids: Array[Column], variableColumnName: String, valueColumnName: String): DS[Row] /** - * Unpivot a DataFrame from wide format to long format, optionally leaving identifier columns set. - * This is the reverse to `groupBy(...).pivot(...).agg(...)`, except for the aggregation, + * Unpivot a DataFrame from wide format to long format, optionally leaving identifier columns + * set. This is the reverse to `groupBy(...).pivot(...).agg(...)`, except for the aggregation, * which cannot be reversed. This is an alias for `unpivot`. * - * @see `org.apache.spark.sql.Dataset.unpivot(Array, Array, String, String)` - * @param ids Id columns - * @param values Value columns to unpivot - * @param variableColumnName Name of the variable column - * @param valueColumnName Name of the value column + * @see + * `org.apache.spark.sql.Dataset.unpivot(Array, Array, String, String)` + * @param ids + * Id columns + * @param values + * Value columns to unpivot + * @param variableColumnName + * Name of the variable column + * @param valueColumnName + * Name of the value column * @group untypedrel * @since 3.4.0 */ @@ -1488,24 +1530,25 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { unpivot(ids, values, variableColumnName, valueColumnName) /** - * Unpivot a DataFrame from wide format to long format, optionally leaving identifier columns set. - * This is the reverse to `groupBy(...).pivot(...).agg(...)`, except for the aggregation, + * Unpivot a DataFrame from wide format to long format, optionally leaving identifier columns + * set. This is the reverse to `groupBy(...).pivot(...).agg(...)`, except for the aggregation, * which cannot be reversed. This is an alias for `unpivot`. * - * @see `org.apache.spark.sql.Dataset.unpivot(Array, Array, String, String)` - * - * This is equivalent to calling `Dataset#unpivot(Array, Array, String, String)` - * where `values` is set to all non-id columns that exist in the DataFrame. - * @param ids Id columns - * @param variableColumnName Name of the variable column - * @param valueColumnName Name of the value column + * @see + * `org.apache.spark.sql.Dataset.unpivot(Array, Array, String, String)` + * + * This is equivalent to calling `Dataset#unpivot(Array, Array, String, String)` where `values` + * is set to all non-id columns that exist in the DataFrame. + * @param ids + * Id columns + * @param variableColumnName + * Name of the variable column + * @param valueColumnName + * Name of the value column * @group untypedrel * @since 3.4.0 */ - def melt( - ids: Array[Column], - variableColumnName: String, - valueColumnName: String): DS[Row] = + def melt(ids: Array[Column], variableColumnName: String, valueColumnName: String): DS[Row] = unpivot(ids, variableColumnName, valueColumnName) /** @@ -1513,12 +1556,12 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { * columns of the DataFrame. * * Please note: - * - All columns except the index column must share a least common data type. Unless they - * are the same data type, all columns are cast to the nearest common data type. - * - The name of the column into which the original column names are transposed defaults - * to "key". - * - null values in the index column are excluded from the column names for the - * transposed table, which are ordered in ascending order. + * - All columns except the index column must share a least common data type. Unless they are + * the same data type, all columns are cast to the nearest common data type. + * - The name of the column into which the original column names are transposed defaults to + * "key". + * - null values in the index column are excluded from the column names for the transposed + * table, which are ordered in ascending order. * * {{{ * val df = Seq(("A", 1, 2), ("B", 3, 4)).toDF("id", "val1", "val2") @@ -1561,9 +1604,9 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { * }}} * * @param indexColumn - * The single column that will be treated as the index for the transpose operation. This column - * will be used to pivot the data, transforming the DataFrame such that the values of the - * indexColumn become the new columns in the transposed DataFrame. + * The single column that will be treated as the index for the transpose operation. This + * column will be used to pivot the data, transforming the DataFrame such that the values of + * the indexColumn become the new columns in the transposed DataFrame. * * @group untypedrel * @since 4.0.0 @@ -1571,8 +1614,8 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { def transpose(indexColumn: Column): DS[Row] /** - * Transposes a DataFrame, switching rows to columns. This function transforms the DataFrame such - * that the values in the first column become the new columns of the DataFrame. + * Transposes a DataFrame, switching rows to columns. This function transforms the DataFrame + * such that the values in the first column become the new columns of the DataFrame. * * This is equivalent to calling `Dataset#transpose(Column)` where `indexColumn` is set to the * first column. @@ -1589,27 +1632,24 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { */ def transpose(): DS[Row] - /** - * Define (named) metrics to observe on the Dataset. This method returns an 'observed' Dataset - * that returns the same result as the input, with the following guarantees: - *
      - *
    • It will compute the defined aggregates (metrics) on all the data that is flowing through - * the Dataset at that point.
    • - *
    • It will report the value of the defined aggregate columns as soon as we reach a completion - * point. A completion point is either the end of a query (batch mode) or the end of a streaming - * epoch. The value of the aggregates only reflects the data processed since the previous - * completion point.
    • - *
    - * Please note that continuous execution is currently not supported. - * - * The metrics columns must either contain a literal (e.g. lit(42)), or should contain one or - * more aggregate functions (e.g. sum(a) or sum(a + b) + avg(c) - lit(1)). Expressions that - * contain references to the input Dataset's columns must always be wrapped in an aggregate - * function. - * - * @group typedrel - * @since 3.0.0 - */ + /** + * Define (named) metrics to observe on the Dataset. This method returns an 'observed' Dataset + * that returns the same result as the input, with the following guarantees:
    • It will + * compute the defined aggregates (metrics) on all the data that is flowing through the Dataset + * at that point.
    • It will report the value of the defined aggregate columns as soon as + * we reach a completion point. A completion point is either the end of a query (batch mode) or + * the end of a streaming epoch. The value of the aggregates only reflects the data processed + * since the previous completion point.
    Please note that continuous execution is + * currently not supported. + * + * The metrics columns must either contain a literal (e.g. lit(42)), or should contain one or + * more aggregate functions (e.g. sum(a) or sum(a + b) + avg(c) - lit(1)). Expressions that + * contain references to the input Dataset's columns must always be wrapped in an aggregate + * function. + * + * @group typedrel + * @since 3.0.0 + */ @scala.annotation.varargs def observe(name: String, expr: Column, exprs: Column*): DS[T] @@ -1627,7 +1667,8 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { * val metrics = observation.get * }}} * - * @throws IllegalArgumentException If this is a streaming Dataset (this.isStreaming == true) + * @throws IllegalArgumentException + * If this is a streaming Dataset (this.isStreaming == true) * * @group typedrel * @since 3.3.0 @@ -1636,9 +1677,9 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { def observe(observation: Observation, expr: Column, exprs: Column*): DS[T] /** - * Returns a new Dataset by taking the first `n` rows. The difference between this function - * and `head` is that `head` is an action and returns an array (by triggering query execution) - * while `limit` returns a new Dataset. + * Returns a new Dataset by taking the first `n` rows. The difference between this function and + * `head` is that `head` is an action and returns an array (by triggering query execution) while + * `limit` returns a new Dataset. * * @group typedrel * @since 2.0.0 @@ -1675,10 +1716,10 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { * // +----+----+----+ * }}} * - * Notice that the column positions in the schema aren't necessarily matched with the - * fields in the strongly typed objects in a Dataset. This function resolves columns - * by their positions in the schema, not the fields in the strongly typed objects. Use - * [[unionByName]] to resolve columns by field name in the typed objects. + * Notice that the column positions in the schema aren't necessarily matched with the fields in + * the strongly typed objects in a Dataset. This function resolves columns by their positions in + * the schema, not the fields in the strongly typed objects. Use [[unionByName]] to resolve + * columns by field name in the typed objects. * * @group typedrel * @since 2.0.0 @@ -1686,8 +1727,8 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { def union(other: DS[T]): DS[T] /** - * Returns a new Dataset containing union of rows in this Dataset and another Dataset. - * This is an alias for `union`. + * Returns a new Dataset containing union of rows in this Dataset and another Dataset. This is + * an alias for `union`. * * This is equivalent to `UNION ALL` in SQL. To do a SQL-style set union (that does * deduplication of elements), use this function followed by a [[distinct]]. @@ -1705,8 +1746,8 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { * This is different from both `UNION ALL` and `UNION DISTINCT` in SQL. To do a SQL-style set * union (that does deduplication of elements), use this function followed by a [[distinct]]. * - * The difference between this function and [[union]] is that this function - * resolves columns by name (not by position): + * The difference between this function and [[union]] is that this function resolves columns by + * name (not by position): * * {{{ * val df1 = Seq((1, 2, 3)).toDF("col0", "col1", "col2") @@ -1733,13 +1774,12 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { /** * Returns a new Dataset containing union of rows in this Dataset and another Dataset. * - * The difference between this function and [[union]] is that this function - * resolves columns by name (not by position). + * The difference between this function and [[union]] is that this function resolves columns by + * name (not by position). * - * When the parameter `allowMissingColumns` is `true`, the set of column names - * in this and other `Dataset` can differ; missing columns will be filled with null. - * Further, the missing columns of this `Dataset` will be added at the end - * in the schema of the union result: + * When the parameter `allowMissingColumns` is `true`, the set of column names in this and other + * `Dataset` can differ; missing columns will be filled with null. Further, the missing columns + * of this `Dataset` will be added at the end in the schema of the union result: * * {{{ * val df1 = Seq((1, 2, 3)).toDF("col0", "col1", "col2") @@ -1776,11 +1816,12 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { def unionByName(other: DS[T], allowMissingColumns: Boolean): DS[T] /** - * Returns a new Dataset containing rows only in both this Dataset and another Dataset. - * This is equivalent to `INTERSECT` in SQL. + * Returns a new Dataset containing rows only in both this Dataset and another Dataset. This is + * equivalent to `INTERSECT` in SQL. * - * @note Equality checking is performed directly on the encoded representation of the data - * and thus is not affected by a custom `equals` function defined on `T`. + * @note + * Equality checking is performed directly on the encoded representation of the data and thus + * is not affected by a custom `equals` function defined on `T`. * @group typedrel * @since 1.6.0 */ @@ -1788,23 +1829,24 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { /** * Returns a new Dataset containing rows only in both this Dataset and another Dataset while - * preserving the duplicates. - * This is equivalent to `INTERSECT ALL` in SQL. + * preserving the duplicates. This is equivalent to `INTERSECT ALL` in SQL. * - * @note Equality checking is performed directly on the encoded representation of the data - * and thus is not affected by a custom `equals` function defined on `T`. Also as standard - * in SQL, this function resolves columns by position (not by name). + * @note + * Equality checking is performed directly on the encoded representation of the data and thus + * is not affected by a custom `equals` function defined on `T`. Also as standard in SQL, this + * function resolves columns by position (not by name). * @group typedrel * @since 2.4.0 */ def intersectAll(other: DS[T]): DS[T] /** - * Returns a new Dataset containing rows in this Dataset but not in another Dataset. - * This is equivalent to `EXCEPT DISTINCT` in SQL. + * Returns a new Dataset containing rows in this Dataset but not in another Dataset. This is + * equivalent to `EXCEPT DISTINCT` in SQL. * - * @note Equality checking is performed directly on the encoded representation of the data - * and thus is not affected by a custom `equals` function defined on `T`. + * @note + * Equality checking is performed directly on the encoded representation of the data and thus + * is not affected by a custom `equals` function defined on `T`. * @group typedrel * @since 2.0.0 */ @@ -1812,25 +1854,28 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { /** * Returns a new Dataset containing rows in this Dataset but not in another Dataset while - * preserving the duplicates. - * This is equivalent to `EXCEPT ALL` in SQL. + * preserving the duplicates. This is equivalent to `EXCEPT ALL` in SQL. * - * @note Equality checking is performed directly on the encoded representation of the data - * and thus is not affected by a custom `equals` function defined on `T`. Also as standard - * in SQL, this function resolves columns by position (not by name). + * @note + * Equality checking is performed directly on the encoded representation of the data and thus + * is not affected by a custom `equals` function defined on `T`. Also as standard in SQL, this + * function resolves columns by position (not by name). * @group typedrel * @since 2.4.0 */ def exceptAll(other: DS[T]): DS[T] /** - * Returns a new [[Dataset]] by sampling a fraction of rows (without replacement), - * using a user-supplied seed. + * Returns a new [[Dataset]] by sampling a fraction of rows (without replacement), using a + * user-supplied seed. * - * @param fraction Fraction of rows to generate, range [0.0, 1.0]. - * @param seed Seed for sampling. - * @note This is NOT guaranteed to provide exactly the fraction of the count - * of the given [[Dataset]]. + * @param fraction + * Fraction of rows to generate, range [0.0, 1.0]. + * @param seed + * Seed for sampling. + * @note + * This is NOT guaranteed to provide exactly the fraction of the count of the given + * [[Dataset]]. * @group typedrel * @since 2.3.0 */ @@ -1839,12 +1884,14 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { } /** - * Returns a new [[Dataset]] by sampling a fraction of rows (without replacement), - * using a random seed. + * Returns a new [[Dataset]] by sampling a fraction of rows (without replacement), using a + * random seed. * - * @param fraction Fraction of rows to generate, range [0.0, 1.0]. - * @note This is NOT guaranteed to provide exactly the fraction of the count - * of the given [[Dataset]]. + * @param fraction + * Fraction of rows to generate, range [0.0, 1.0]. + * @note + * This is NOT guaranteed to provide exactly the fraction of the count of the given + * [[Dataset]]. * @group typedrel * @since 2.3.0 */ @@ -1855,11 +1902,15 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { /** * Returns a new [[Dataset]] by sampling a fraction of rows, using a user-supplied seed. * - * @param withReplacement Sample with replacement or not. - * @param fraction Fraction of rows to generate, range [0.0, 1.0]. - * @param seed Seed for sampling. - * @note This is NOT guaranteed to provide exactly the fraction of the count - * of the given [[Dataset]]. + * @param withReplacement + * Sample with replacement or not. + * @param fraction + * Fraction of rows to generate, range [0.0, 1.0]. + * @param seed + * Seed for sampling. + * @note + * This is NOT guaranteed to provide exactly the fraction of the count of the given + * [[Dataset]]. * @group typedrel * @since 1.6.0 */ @@ -1868,11 +1919,14 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { /** * Returns a new [[Dataset]] by sampling a fraction of rows, using a random seed. * - * @param withReplacement Sample with replacement or not. - * @param fraction Fraction of rows to generate, range [0.0, 1.0]. + * @param withReplacement + * Sample with replacement or not. + * @param fraction + * Fraction of rows to generate, range [0.0, 1.0]. * - * @note This is NOT guaranteed to provide exactly the fraction of the total count - * of the given [[Dataset]]. + * @note + * This is NOT guaranteed to provide exactly the fraction of the total count of the given + * [[Dataset]]. * * @group typedrel * @since 1.6.0 @@ -1884,8 +1938,10 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { /** * Randomly splits this Dataset with the provided weights. * - * @param weights weights for splits, will be normalized if they don't sum to 1. - * @param seed Seed for sampling. + * @param weights + * weights for splits, will be normalized if they don't sum to 1. + * @param seed + * Seed for sampling. * * For Java API, use [[randomSplitAsList]]. * @@ -1897,8 +1953,10 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { /** * Returns a Java list that contains randomly split Dataset with the provided weights. * - * @param weights weights for splits, will be normalized if they don't sum to 1. - * @param seed Seed for sampling. + * @param weights + * weights for splits, will be normalized if they don't sum to 1. + * @param seed + * Seed for sampling. * @group typedrel * @since 2.0.0 */ @@ -1907,16 +1965,17 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { /** * Randomly splits this Dataset with the provided weights. * - * @param weights weights for splits, will be normalized if they don't sum to 1. + * @param weights + * weights for splits, will be normalized if they don't sum to 1. * @group typedrel * @since 2.0.0 */ def randomSplit(weights: Array[Double]): Array[_ <: DS[T]] /** - * (Scala-specific) Returns a new Dataset where each row has been expanded to zero or more - * rows by the provided function. This is similar to a `LATERAL VIEW` in HiveQL. The columns of - * the input row are implicitly joined with each row that is output by the function. + * (Scala-specific) Returns a new Dataset where each row has been expanded to zero or more rows + * by the provided function. This is similar to a `LATERAL VIEW` in HiveQL. The columns of the + * input row are implicitly joined with each row that is output by the function. * * Given that this is deprecated, as an alternative, you can explode columns either using * `functions.explode()` or `flatMap()`. The following example uses these alternatives to count @@ -1941,12 +2000,13 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { * @since 2.0.0 */ @deprecated("use flatMap() or select() with functions.explode() instead", "2.0.0") - def explode[A <: Product : TypeTag](input: Column*)(f: Row => IterableOnce[A]): DS[Row] + def explode[A <: Product: TypeTag](input: Column*)(f: Row => IterableOnce[A]): DS[Row] /** - * (Scala-specific) Returns a new Dataset where a single column has been expanded to zero - * or more rows by the provided function. This is similar to a `LATERAL VIEW` in HiveQL. All - * columns of the input row are implicitly joined with each value that is output by the function. + * (Scala-specific) Returns a new Dataset where a single column has been expanded to zero or + * more rows by the provided function. This is similar to a `LATERAL VIEW` in HiveQL. All + * columns of the input row are implicitly joined with each value that is output by the + * function. * * Given that this is deprecated, as an alternative, you can explode columns either using * `functions.explode()`: @@ -1965,20 +2025,21 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { * @since 2.0.0 */ @deprecated("use flatMap() or select() with functions.explode() instead", "2.0.0") - def explode[A, B : TypeTag](inputColumn: String, outputColumn: String)(f: A => IterableOnce[B]) - : DS[Row] + def explode[A, B: TypeTag](inputColumn: String, outputColumn: String)( + f: A => IterableOnce[B]): DS[Row] /** - * Returns a new Dataset by adding a column or replacing the existing column that has - * the same name. + * Returns a new Dataset by adding a column or replacing the existing column that has the same + * name. * - * `column`'s expression must only refer to attributes supplied by this Dataset. It is an - * error to add a column that refers to some other Dataset. + * `column`'s expression must only refer to attributes supplied by this Dataset. It is an error + * to add a column that refers to some other Dataset. * - * @note this method introduces a projection internally. Therefore, calling it multiple times, - * for instance, via loops in order to add multiple columns can generate big plans which - * can cause performance issues and even `StackOverflowException`. To avoid this, - * use `select` with the multiple columns at once. + * @note + * this method introduces a projection internally. Therefore, calling it multiple times, for + * instance, via loops in order to add multiple columns can generate big plans which can cause + * performance issues and even `StackOverflowException`. To avoid this, use `select` with the + * multiple columns at once. * @group untypedrel * @since 2.0.0 */ @@ -2009,19 +2070,17 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { * @group untypedrel * @since 3.3.0 */ - def withColumns(colsMap: util.Map[String, Column]): DS[Row] = withColumns( - colsMap.asScala.toMap - ) + def withColumns(colsMap: util.Map[String, Column]): DS[Row] = withColumns(colsMap.asScala.toMap) /** - * Returns a new Dataset by adding columns or replacing the existing columns that has - * the same names. + * Returns a new Dataset by adding columns or replacing the existing columns that has the same + * names. */ protected def withColumns(colNames: Seq[String], cols: Seq[Column]): DS[Row] /** - * Returns a new Dataset with a column renamed. - * This is a no-op if schema doesn't contain existingName. + * Returns a new Dataset with a column renamed. This is a no-op if schema doesn't contain + * existingName. * * @group untypedrel * @since 2.0.0 @@ -2030,14 +2089,13 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { withColumnsRenamed(Seq(existingName), Seq(newName)) /** - * (Scala-specific) - * Returns a new Dataset with a columns renamed. - * This is a no-op if schema doesn't contain existingName. + * (Scala-specific) Returns a new Dataset with a columns renamed. This is a no-op if schema + * doesn't contain existingName. * * `colsMap` is a map of existing column name and new column name. * - * @throws org.apache.spark.sql.AnalysisException if there are duplicate names in resulting - * projection + * @throws org.apache.spark.sql.AnalysisException + * if there are duplicate names in resulting projection * @group untypedrel * @since 3.4.0 */ @@ -2048,9 +2106,8 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { } /** - * (Java-specific) - * Returns a new Dataset with a columns renamed. - * This is a no-op if schema doesn't contain existingName. + * (Java-specific) Returns a new Dataset with a columns renamed. This is a no-op if schema + * doesn't contain existingName. * * `colsMap` is a map of existing column name and new column name. * @@ -2060,9 +2117,7 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { def withColumnsRenamed(colsMap: util.Map[String, String]): DS[Row] = withColumnsRenamed(colsMap.asScala.toMap) - protected def withColumnsRenamed( - colNames: Seq[String], - newColNames: Seq[String]): DS[Row] + protected def withColumnsRenamed(colNames: Seq[String], newColNames: Seq[String]): DS[Row] /** * Returns a new Dataset by updating an existing column with metadata. @@ -2073,14 +2128,14 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { def withMetadata(columnName: String, metadata: Metadata): DS[Row] /** - * Returns a new Dataset with a column dropped. This is a no-op if schema doesn't contain - * column name. + * Returns a new Dataset with a column dropped. This is a no-op if schema doesn't contain column + * name. * * This method can only be used to drop top level columns. the colName string is treated * literally without further interpretation. * - * Note: `drop(colName)` has different semantic with `drop(col(colName))`, for example: - * 1, multi column have the same colName: + * Note: `drop(colName)` has different semantic with `drop(col(colName))`, for example: 1, multi + * column have the same colName: * {{{ * val df1 = spark.range(0, 2).withColumn("key1", lit(1)) * val df2 = spark.range(0, 2).withColumn("key2", lit(2)) @@ -2143,14 +2198,14 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { * @group untypedrel * @since 2.0.0 */ - def drop(colName: String): DS[Row] = drop(colName :: Nil : _*) + def drop(colName: String): DS[Row] = drop(colName :: Nil: _*) /** - * Returns a new Dataset with columns dropped. - * This is a no-op if schema doesn't contain column name(s). + * Returns a new Dataset with columns dropped. This is a no-op if schema doesn't contain column + * name(s). * - * This method can only be used to drop top level columns. the colName string is treated literally - * without further interpretation. + * This method can only be used to drop top level columns. the colName string is treated + * literally without further interpretation. * * @group untypedrel * @since 2.0.0 @@ -2161,25 +2216,23 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { /** * Returns a new Dataset with column dropped. * - * This method can only be used to drop top level column. - * This version of drop accepts a [[org.apache.spark.sql.Column]] rather than a name. - * This is a no-op if the Dataset doesn't have a column - * with an equivalent expression. + * This method can only be used to drop top level column. This version of drop accepts a + * [[org.apache.spark.sql.Column]] rather than a name. This is a no-op if the Dataset doesn't + * have a column with an equivalent expression. * - * Note: `drop(col(colName))` has different semantic with `drop(colName)`, - * please refer to `Dataset#drop(colName: String)`. + * Note: `drop(col(colName))` has different semantic with `drop(colName)`, please refer to + * `Dataset#drop(colName: String)`. * * @group untypedrel * @since 2.0.0 */ - def drop(col: Column): DS[Row] = drop(col, Nil : _*) + def drop(col: Column): DS[Row] = drop(col, Nil: _*) /** * Returns a new Dataset with columns dropped. * - * This method can only be used to drop top level columns. - * This is a no-op if the Dataset doesn't have a columns - * with an equivalent expression. + * This method can only be used to drop top level columns. This is a no-op if the Dataset + * doesn't have a columns with an equivalent expression. * * @group untypedrel * @since 3.4.0 @@ -2188,13 +2241,13 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { def drop(col: Column, cols: Column*): DS[Row] /** - * Returns a new Dataset that contains only the unique rows from this Dataset. - * This is an alias for `distinct`. + * Returns a new Dataset that contains only the unique rows from this Dataset. This is an alias + * for `distinct`. * * For a static batch [[Dataset]], it just drops duplicate rows. For a streaming [[Dataset]], it * will keep all data across triggers as intermediate state to drop duplicates rows. You can use - * [[withWatermark]] to limit how late the duplicate data can be and system will accordingly limit - * the state. In addition, too late data older than watermark will be dropped to avoid any + * [[withWatermark]] to limit how late the duplicate data can be and system will accordingly + * limit the state. In addition, too late data older than watermark will be dropped to avoid any * possibility of duplicates. * * @group typedrel @@ -2203,13 +2256,13 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { def dropDuplicates(): DS[T] /** - * (Scala-specific) Returns a new Dataset with duplicate rows removed, considering only - * the subset of columns. + * (Scala-specific) Returns a new Dataset with duplicate rows removed, considering only the + * subset of columns. * * For a static batch [[Dataset]], it just drops duplicate rows. For a streaming [[Dataset]], it * will keep all data across triggers as intermediate state to drop duplicates rows. You can use - * [[withWatermark]] to limit how late the duplicate data can be and system will accordingly limit - * the state. In addition, too late data older than watermark will be dropped to avoid any + * [[withWatermark]] to limit how late the duplicate data can be and system will accordingly + * limit the state. In addition, too late data older than watermark will be dropped to avoid any * possibility of duplicates. * * @group typedrel @@ -2218,13 +2271,12 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { def dropDuplicates(colNames: Seq[String]): DS[T] /** - * Returns a new Dataset with duplicate rows removed, considering only - * the subset of columns. + * Returns a new Dataset with duplicate rows removed, considering only the subset of columns. * * For a static batch [[Dataset]], it just drops duplicate rows. For a streaming [[Dataset]], it * will keep all data across triggers as intermediate state to drop duplicates rows. You can use - * [[withWatermark]] to limit how late the duplicate data can be and system will accordingly limit - * the state. In addition, too late data older than watermark will be dropped to avoid any + * [[withWatermark]] to limit how late the duplicate data can be and system will accordingly + * limit the state. In addition, too late data older than watermark will be dropped to avoid any * possibility of duplicates. * * @group typedrel @@ -2234,13 +2286,13 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { dropDuplicates(colNames.toImmutableArraySeq) /** - * Returns a new [[Dataset]] with duplicate rows removed, considering only - * the subset of columns. + * Returns a new [[Dataset]] with duplicate rows removed, considering only the subset of + * columns. * * For a static batch [[Dataset]], it just drops duplicate rows. For a streaming [[Dataset]], it * will keep all data across triggers as intermediate state to drop duplicates rows. You can use - * [[withWatermark]] to limit how late the duplicate data can be and system will accordingly limit - * the state. In addition, too late data older than watermark will be dropped to avoid any + * [[withWatermark]] to limit how late the duplicate data can be and system will accordingly + * limit the state. In addition, too late data older than watermark will be dropped to avoid any * possibility of duplicates. * * @group typedrel @@ -2258,8 +2310,8 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { * This only works with streaming [[Dataset]], and watermark for the input [[Dataset]] must be * set via [[withWatermark]]. * - * For a streaming [[Dataset]], this will keep all data across triggers as intermediate state - * to drop duplicated rows. The state will be kept to guarantee the semantic, "Events are + * For a streaming [[Dataset]], this will keep all data across triggers as intermediate state to + * drop duplicated rows. The state will be kept to guarantee the semantic, "Events are * deduplicated as long as the time distance of earliest and latest events are smaller than the * delay threshold of watermark." Users are encouraged to set the delay threshold of watermark * longer than max timestamp differences among duplicated events. @@ -2278,8 +2330,8 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { * This only works with streaming [[Dataset]], and watermark for the input [[Dataset]] must be * set via [[withWatermark]]. * - * For a streaming [[Dataset]], this will keep all data across triggers as intermediate state - * to drop duplicated rows. The state will be kept to guarantee the semantic, "Events are + * For a streaming [[Dataset]], this will keep all data across triggers as intermediate state to + * drop duplicated rows. The state will be kept to guarantee the semantic, "Events are * deduplicated as long as the time distance of earliest and latest events are smaller than the * delay threshold of watermark." Users are encouraged to set the delay threshold of watermark * longer than max timestamp differences among duplicated events. @@ -2298,8 +2350,8 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { * This only works with streaming [[Dataset]], and watermark for the input [[Dataset]] must be * set via [[withWatermark]]. * - * For a streaming [[Dataset]], this will keep all data across triggers as intermediate state - * to drop duplicated rows. The state will be kept to guarantee the semantic, "Events are + * For a streaming [[Dataset]], this will keep all data across triggers as intermediate state to + * drop duplicated rows. The state will be kept to guarantee the semantic, "Events are * deduplicated as long as the time distance of earliest and latest events are smaller than the * delay threshold of watermark." Users are encouraged to set the delay threshold of watermark * longer than max timestamp differences among duplicated events. @@ -2320,8 +2372,8 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { * This only works with streaming [[Dataset]], and watermark for the input [[Dataset]] must be * set via [[withWatermark]]. * - * For a streaming [[Dataset]], this will keep all data across triggers as intermediate state - * to drop duplicated rows. The state will be kept to guarantee the semantic, "Events are + * For a streaming [[Dataset]], this will keep all data across triggers as intermediate state to + * drop duplicated rows. The state will be kept to guarantee the semantic, "Events are * deduplicated as long as the time distance of earliest and latest events are smaller than the * delay threshold of watermark." Users are encouraged to set the delay threshold of watermark * longer than max timestamp differences among duplicated events. @@ -2360,7 +2412,8 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { * * Use [[summary]] for expanded statistics and control over which statistics to compute. * - * @param cols Columns to compute statistics on. + * @param cols + * Columns to compute statistics on. * @group action * @since 1.6.0 */ @@ -2368,20 +2421,13 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { def describe(cols: String*): DS[Row] /** - * Computes specified statistics for numeric and string columns. Available statistics are: - *
      - *
    • count
    • - *
    • mean
    • - *
    • stddev
    • - *
    • min
    • - *
    • max
    • - *
    • arbitrary approximate percentiles specified as a percentage (e.g. 75%)
    • - *
    • count_distinct
    • - *
    • approx_count_distinct
    • - *
    + * Computes specified statistics for numeric and string columns. Available statistics are:
      + *
    • count
    • mean
    • stddev
    • min
    • max
    • arbitrary + * approximate percentiles specified as a percentage (e.g. 75%)
    • count_distinct
    • + *
    • approx_count_distinct
    * - * If no statistics are given, this function computes count, mean, stddev, min, - * approximate quartiles (percentiles at 25%, 50%, and 75%), and max. + * If no statistics are given, this function computes count, mean, stddev, min, approximate + * quartiles (percentiles at 25%, 50%, and 75%), and max. * * This function is meant for exploratory data analysis, as we make no guarantee about the * backward compatibility of the schema of the resulting Dataset. If you want to @@ -2436,7 +2482,8 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { * * See also [[describe]] for basic statistics. * - * @param statistics Statistics from above list to be computed. + * @param statistics + * Statistics from above list to be computed. * @group action * @since 2.3.0 */ @@ -2446,8 +2493,9 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { /** * Returns the first `n` rows. * - * @note this method should only be used if the resulting array is expected to be small, as - * all the data is loaded into the driver's memory. + * @note + * this method should only be used if the resulting array is expected to be small, as all the + * data is loaded into the driver's memory. * @group action * @since 1.6.0 */ @@ -2485,8 +2533,8 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { def transform[U](t: DS[T] => DS[U]): DS[U] = t(this.asInstanceOf[DS[T]]) /** - * (Scala-specific) - * Returns a new Dataset that contains the result of applying `func` to each element. + * (Scala-specific) Returns a new Dataset that contains the result of applying `func` to each + * element. * * @group typedrel * @since 1.6.0 @@ -2494,8 +2542,8 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { def map[U: Encoder](func: T => U): DS[U] /** - * (Java-specific) - * Returns a new Dataset that contains the result of applying `func` to each element. + * (Java-specific) Returns a new Dataset that contains the result of applying `func` to each + * element. * * @group typedrel * @since 1.6.0 @@ -2503,8 +2551,8 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { def map[U](func: MapFunction[T, U], encoder: Encoder[U]): DS[U] /** - * (Scala-specific) - * Returns a new Dataset that contains the result of applying `func` to each partition. + * (Scala-specific) Returns a new Dataset that contains the result of applying `func` to each + * partition. * * @group typedrel * @since 1.6.0 @@ -2512,8 +2560,8 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { def mapPartitions[U: Encoder](func: Iterator[T] => Iterator[U]): DS[U] /** - * (Java-specific) - * Returns a new Dataset that contains the result of applying `f` to each partition. + * (Java-specific) Returns a new Dataset that contains the result of applying `f` to each + * partition. * * @group typedrel * @since 1.6.0 @@ -2522,9 +2570,8 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { mapPartitions(ToScalaUDF(f))(encoder) /** - * (Scala-specific) - * Returns a new Dataset by first applying a function to all elements of this Dataset, - * and then flattening the results. + * (Scala-specific) Returns a new Dataset by first applying a function to all elements of this + * Dataset, and then flattening the results. * * @group typedrel * @since 1.6.0 @@ -2533,9 +2580,8 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { mapPartitions(UDFAdaptors.flatMapToMapPartitions[T, U](func)) /** - * (Java-specific) - * Returns a new Dataset by first applying a function to all elements of this Dataset, - * and then flattening the results. + * (Java-specific) Returns a new Dataset by first applying a function to all elements of this + * Dataset, and then flattening the results. * * @group typedrel * @since 1.6.0 @@ -2555,8 +2601,7 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { } /** - * (Java-specific) - * Runs `func` on each element of this Dataset. + * (Java-specific) Runs `func` on each element of this Dataset. * * @group action * @since 1.6.0 @@ -2574,8 +2619,7 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { def foreachPartition(f: Iterator[T] => Unit): Unit /** - * (Java-specific) - * Runs `func` on each partition of this Dataset. + * (Java-specific) Runs `func` on each partition of this Dataset. * * @group action * @since 1.6.0 @@ -2587,8 +2631,8 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { /** * Returns the first `n` rows in the Dataset. * - * Running take requires moving data into the application's driver process, and doing so with - * a very large `n` can crash the driver process with OutOfMemoryError. + * Running take requires moving data into the application's driver process, and doing so with a + * very large `n` can crash the driver process with OutOfMemoryError. * * @group action * @since 1.6.0 @@ -2598,8 +2642,8 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { /** * Returns the last `n` rows in the Dataset. * - * Running tail requires moving data into the application's driver process, and doing so with - * a very large `n` can crash the driver process with OutOfMemoryError. + * Running tail requires moving data into the application's driver process, and doing so with a + * very large `n` can crash the driver process with OutOfMemoryError. * * @group action * @since 3.0.0 @@ -2609,8 +2653,8 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { /** * Returns the first `n` rows in the Dataset as a list. * - * Running take requires moving data into the application's driver process, and doing so with - * a very large `n` can crash the driver process with OutOfMemoryError. + * Running take requires moving data into the application's driver process, and doing so with a + * very large `n` can crash the driver process with OutOfMemoryError. * * @group action * @since 1.6.0 @@ -2620,8 +2664,8 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { /** * Returns an array that contains all rows in this Dataset. * - * Running collect requires moving all the data into the application's driver process, and - * doing so on a very large dataset can crash the driver process with OutOfMemoryError. + * Running collect requires moving all the data into the application's driver process, and doing + * so on a very large dataset can crash the driver process with OutOfMemoryError. * * For Java API, use [[collectAsList]]. * @@ -2633,8 +2677,8 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { /** * Returns a Java list that contains all rows in this Dataset. * - * Running collect requires moving all the data into the application's driver process, and - * doing so on a very large dataset can crash the driver process with OutOfMemoryError. + * Running collect requires moving all the data into the application's driver process, and doing + * so on a very large dataset can crash the driver process with OutOfMemoryError. * * @group action * @since 1.6.0 @@ -2646,9 +2690,10 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { * * The iterator will consume as much memory as the largest partition in this Dataset. * - * @note this results in multiple Spark jobs, and if the input Dataset is the result - * of a wide transformation (e.g. join with different partitioners), to avoid - * recomputing the input Dataset should be cached first. + * @note + * this results in multiple Spark jobs, and if the input Dataset is the result of a wide + * transformation (e.g. join with different partitioners), to avoid recomputing the input + * Dataset should be cached first. * @group action * @since 2.0.0 */ @@ -2675,8 +2720,8 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { partitionExprs: Seq[Column]): DS[T] /** - * Returns a new Dataset partitioned by the given partitioning expressions into - * `numPartitions`. The resulting Dataset is hash partitioned. + * Returns a new Dataset partitioned by the given partitioning expressions into `numPartitions`. + * The resulting Dataset is hash partitioned. * * This is the same operation as "DISTRIBUTE BY" in SQL (Hive QL). * @@ -2690,8 +2735,8 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { /** * Returns a new Dataset partitioned by the given partitioning expressions, using - * `spark.sql.shuffle.partitions` as number of partitions. - * The resulting Dataset is hash partitioned. + * `spark.sql.shuffle.partitions` as number of partitions. The resulting Dataset is hash + * partitioned. * * This is the same operation as "DISTRIBUTE BY" in SQL (Hive QL). * @@ -2703,24 +2748,19 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { repartitionByExpression(None, partitionExprs) } - protected def repartitionByRange( - numPartitions: Option[Int], - partitionExprs: Seq[Column]): DS[T] - + protected def repartitionByRange(numPartitions: Option[Int], partitionExprs: Seq[Column]): DS[T] /** - * Returns a new Dataset partitioned by the given partitioning expressions into - * `numPartitions`. The resulting Dataset is range partitioned. - * - * At least one partition-by expression must be specified. - * When no explicit sort order is specified, "ascending nulls first" is assumed. - * Note, the rows are not sorted in each partition of the resulting Dataset. + * Returns a new Dataset partitioned by the given partitioning expressions into `numPartitions`. + * The resulting Dataset is range partitioned. * + * At least one partition-by expression must be specified. When no explicit sort order is + * specified, "ascending nulls first" is assumed. Note, the rows are not sorted in each + * partition of the resulting Dataset. * - * Note that due to performance reasons this method uses sampling to estimate the ranges. - * Hence, the output may not be consistent, since sampling can return different values. - * The sample size can be controlled by the config - * `spark.sql.execution.rangeExchange.sampleSizePerPartition`. + * Note that due to performance reasons this method uses sampling to estimate the ranges. Hence, + * the output may not be consistent, since sampling can return different values. The sample size + * can be controlled by the config `spark.sql.execution.rangeExchange.sampleSizePerPartition`. * * @group typedrel * @since 2.3.0 @@ -2732,17 +2772,16 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { /** * Returns a new Dataset partitioned by the given partitioning expressions, using - * `spark.sql.shuffle.partitions` as number of partitions. - * The resulting Dataset is range partitioned. + * `spark.sql.shuffle.partitions` as number of partitions. The resulting Dataset is range + * partitioned. * - * At least one partition-by expression must be specified. - * When no explicit sort order is specified, "ascending nulls first" is assumed. - * Note, the rows are not sorted in each partition of the resulting Dataset. + * At least one partition-by expression must be specified. When no explicit sort order is + * specified, "ascending nulls first" is assumed. Note, the rows are not sorted in each + * partition of the resulting Dataset. * - * Note that due to performance reasons this method uses sampling to estimate the ranges. - * Hence, the output may not be consistent, since sampling can return different values. - * The sample size can be controlled by the config - * `spark.sql.execution.rangeExchange.sampleSizePerPartition`. + * Note that due to performance reasons this method uses sampling to estimate the ranges. Hence, + * the output may not be consistent, since sampling can return different values. The sample size + * can be controlled by the config `spark.sql.execution.rangeExchange.sampleSizePerPartition`. * * @group typedrel * @since 2.3.0 @@ -2755,16 +2794,15 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { /** * Returns a new Dataset that has exactly `numPartitions` partitions, when the fewer partitions * are requested. If a larger number of partitions is requested, it will stay at the current - * number of partitions. Similar to coalesce defined on an `RDD`, this operation results in - * a narrow dependency, e.g. if you go from 1000 partitions to 100 partitions, there will not - * be a shuffle, instead each of the 100 new partitions will claim 10 of the current partitions. + * number of partitions. Similar to coalesce defined on an `RDD`, this operation results in a + * narrow dependency, e.g. if you go from 1000 partitions to 100 partitions, there will not be a + * shuffle, instead each of the 100 new partitions will claim 10 of the current partitions. * - * However, if you're doing a drastic coalesce, e.g. to numPartitions = 1, - * this may result in your computation taking place on fewer nodes than - * you like (e.g. one node in the case of numPartitions = 1). To avoid this, - * you can call repartition. This will add a shuffle step, but means the - * current upstream partitions will be executed in parallel (per whatever - * the current partitioning is). + * However, if you're doing a drastic coalesce, e.g. to numPartitions = 1, this may result in + * your computation taking place on fewer nodes than you like (e.g. one node in the case of + * numPartitions = 1). To avoid this, you can call repartition. This will add a shuffle step, + * but means the current upstream partitions will be executed in parallel (per whatever the + * current partitioning is). * * @group typedrel * @since 1.6.0 @@ -2772,15 +2810,16 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { def coalesce(numPartitions: Int): DS[T] /** - * Returns a new Dataset that contains only the unique rows from this Dataset. - * This is an alias for `dropDuplicates`. + * Returns a new Dataset that contains only the unique rows from this Dataset. This is an alias + * for `dropDuplicates`. * - * Note that for a streaming [[Dataset]], this method returns distinct rows only once - * regardless of the output mode, which the behavior may not be same with `DISTINCT` in SQL - * against streaming [[Dataset]]. + * Note that for a streaming [[Dataset]], this method returns distinct rows only once regardless + * of the output mode, which the behavior may not be same with `DISTINCT` in SQL against + * streaming [[Dataset]]. * - * @note Equality checking is performed directly on the encoded representation of the data - * and thus is not affected by a custom `equals` function defined on `T`. + * @note + * Equality checking is performed directly on the encoded representation of the data and thus + * is not affected by a custom `equals` function defined on `T`. * @group typedrel * @since 2.0.0 */ @@ -2802,13 +2841,12 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { */ def cache(): DS[T] - /** * Persist this Dataset with the given storage level. * - * @param newLevel One of: `MEMORY_ONLY`, `MEMORY_AND_DISK`, `MEMORY_ONLY_SER`, - * `MEMORY_AND_DISK_SER`, `DISK_ONLY`, `MEMORY_ONLY_2`, - * `MEMORY_AND_DISK_2`, etc. + * @param newLevel + * One of: `MEMORY_ONLY`, `MEMORY_AND_DISK`, `MEMORY_ONLY_SER`, `MEMORY_AND_DISK_SER`, + * `DISK_ONLY`, `MEMORY_ONLY_2`, `MEMORY_AND_DISK_2`, etc. * @group basic * @since 1.6.0 */ @@ -2823,18 +2861,19 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { def storageLevel: StorageLevel /** - * Mark the Dataset as non-persistent, and remove all blocks for it from memory and disk. - * This will not un-persist any cached data that is built upon this Dataset. + * Mark the Dataset as non-persistent, and remove all blocks for it from memory and disk. This + * will not un-persist any cached data that is built upon this Dataset. * - * @param blocking Whether to block until all blocks are deleted. + * @param blocking + * Whether to block until all blocks are deleted. * @group basic * @since 1.6.0 */ def unpersist(blocking: Boolean): DS[T] /** - * Mark the Dataset as non-persistent, and remove all blocks for it from memory and disk. - * This will not un-persist any cached data that is built upon this Dataset. + * Mark the Dataset as non-persistent, and remove all blocks for it from memory and disk. This + * will not un-persist any cached data that is built upon this Dataset. * * @group basic * @since 1.6.0 @@ -2854,14 +2893,15 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { } /** - * Creates a local temporary view using the given name. The lifetime of this - * temporary view is tied to the `SparkSession` that was used to create this Dataset. + * Creates a local temporary view using the given name. The lifetime of this temporary view is + * tied to the `SparkSession` that was used to create this Dataset. * * Local temporary view is session-scoped. Its lifetime is the lifetime of the session that - * created it, i.e. it will be automatically dropped when the session terminates. It's not - * tied to any databases, i.e. we can't use `db1.view1` to reference a local temporary view. + * created it, i.e. it will be automatically dropped when the session terminates. It's not tied + * to any databases, i.e. we can't use `db1.view1` to reference a local temporary view. * - * @throws org.apache.spark.sql.AnalysisException if the view name is invalid or already exists + * @throws org.apache.spark.sql.AnalysisException + * if the view name is invalid or already exists * @group basic * @since 2.0.0 */ @@ -2870,10 +2910,9 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { createTempView(viewName, replace = false, global = false) } - /** - * Creates a local temporary view using the given name. The lifetime of this - * temporary view is tied to the `SparkSession` that was used to create this Dataset. + * Creates a local temporary view using the given name. The lifetime of this temporary view is + * tied to the `SparkSession` that was used to create this Dataset. * * @group basic * @since 2.0.0 @@ -2883,15 +2922,16 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { } /** - * Creates a global temporary view using the given name. The lifetime of this - * temporary view is tied to this Spark application. + * Creates a global temporary view using the given name. The lifetime of this temporary view is + * tied to this Spark application. * - * Global temporary view is cross-session. Its lifetime is the lifetime of the Spark application, - * i.e. it will be automatically dropped when the application terminates. It's tied to a system - * preserved database `global_temp`, and we must use the qualified name to refer a global temp - * view, e.g. `SELECT * FROM global_temp.view1`. + * Global temporary view is cross-session. Its lifetime is the lifetime of the Spark + * application, i.e. it will be automatically dropped when the application terminates. It's tied + * to a system preserved database `global_temp`, and we must use the qualified name to refer a + * global temp view, e.g. `SELECT * FROM global_temp.view1`. * - * @throws org.apache.spark.sql.AnalysisException if the view name is invalid or already exists + * @throws org.apache.spark.sql.AnalysisException + * if the view name is invalid or already exists * @group basic * @since 2.1.0 */ @@ -2904,10 +2944,10 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { * Creates or replaces a global temporary view using the given name. The lifetime of this * temporary view is tied to this Spark application. * - * Global temporary view is cross-session. Its lifetime is the lifetime of the Spark application, - * i.e. it will be automatically dropped when the application terminates. It's tied to a system - * preserved database `global_temp`, and we must use the qualified name to refer a global temp - * view, e.g. `SELECT * FROM global_temp.view1`. + * Global temporary view is cross-session. Its lifetime is the lifetime of the Spark + * application, i.e. it will be automatically dropped when the application terminates. It's tied + * to a system preserved database `global_temp`, and we must use the qualified name to refer a + * global temp view, e.g. `SELECT * FROM global_temp.view1`. * * @group basic * @since 2.2.0 @@ -2919,8 +2959,8 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { protected def createTempView(viewName: String, replace: Boolean, global: Boolean): Unit /** - * Merges a set of updates, insertions, and deletions based on a source table into - * a target table. + * Merges a set of updates, insertions, and deletions based on a source table into a target + * table. * * Scala Examples: * {{{ @@ -2972,8 +3012,9 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { /** * Returns a best-effort snapshot of the files that compose this Dataset. This method simply - * asks each constituent BaseRelation for its respective files and takes the union of all results. - * Depending on the source relations, this may not find all input files. Duplicates are removed. + * asks each constituent BaseRelation for its respective files and takes the union of all + * results. Depending on the source relations, this may not find all input files. Duplicates are + * removed. * * @group basic * @since 2.0.0 @@ -2981,14 +3022,16 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { def inputFiles: Array[String] /** - * Returns `true` when the logical query plans inside both [[Dataset]]s are equal and - * therefore return same results. + * Returns `true` when the logical query plans inside both [[Dataset]]s are equal and therefore + * return same results. * - * @note The equality comparison here is simplified by tolerating the cosmetic differences - * such as attribute names. - * @note This API can compare both [[Dataset]]s very fast but can still return `false` on - * the [[Dataset]] that return the same results, for instance, from different plans. Such - * false negative semantic can be useful when caching as an example. + * @note + * The equality comparison here is simplified by tolerating the cosmetic differences such as + * attribute names. + * @note + * This API can compare both [[Dataset]]s very fast but can still return `false` on the + * [[Dataset]] that return the same results, for instance, from different plans. Such false + * negative semantic can be useful when caching as an example. * @since 3.1.0 */ @DeveloperApi @@ -2997,8 +3040,9 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { /** * Returns a `hashCode` of the logical query plan against this [[Dataset]]. * - * @note Unlike the standard `hashCode`, the hash is calculated against the query plan - * simplified by tolerating the cosmetic differences such as attribute names. + * @note + * Unlike the standard `hashCode`, the hash is calculated against the query plan simplified by + * tolerating the cosmetic differences such as attribute names. * @since 3.1.0 */ @DeveloperApi diff --git a/sql/api/src/main/scala/org/apache/spark/sql/api/KeyValueGroupedDataset.scala b/sql/api/src/main/scala/org/apache/spark/sql/api/KeyValueGroupedDataset.scala index 5e73da2755f08..a37b2d41668e9 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/api/KeyValueGroupedDataset.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/api/KeyValueGroupedDataset.scala @@ -24,9 +24,9 @@ import org.apache.spark.sql.internal.{ToScalaUDF, UDFAdaptors} import org.apache.spark.sql.streaming.{GroupState, GroupStateTimeout, OutputMode, StatefulProcessor, StatefulProcessorWithInitialState, TimeMode} /** - * A [[Dataset]] has been logically grouped by a user specified grouping key. Users should not - * construct a [[KeyValueGroupedDataset]] directly, but should instead call `groupByKey` on - * an existing [[Dataset]]. + * A [[Dataset]] has been logically grouped by a user specified grouping key. Users should not + * construct a [[KeyValueGroupedDataset]] directly, but should instead call `groupByKey` on an + * existing [[Dataset]]. * * @since 2.0.0 */ @@ -43,8 +43,8 @@ abstract class KeyValueGroupedDataset[K, V, DS[U] <: Dataset[U, DS]] extends Ser def keyAs[L: Encoder]: KVDS[L, V] /** - * Returns a new [[KeyValueGroupedDataset]] where the given function `func` has been applied - * to the data. The grouping key is unchanged by this. + * Returns a new [[KeyValueGroupedDataset]] where the given function `func` has been applied to + * the data. The grouping key is unchanged by this. * * {{{ * // Create values grouped by key from a Dataset[(K, V)] @@ -56,8 +56,8 @@ abstract class KeyValueGroupedDataset[K, V, DS[U] <: Dataset[U, DS]] extends Ser def mapValues[W: Encoder](func: V => W): KVDS[K, W] /** - * Returns a new [[KeyValueGroupedDataset]] where the given function `func` has been applied - * to the data. The grouping key is unchanged by this. + * Returns a new [[KeyValueGroupedDataset]] where the given function `func` has been applied to + * the data. The grouping key is unchanged by this. * * {{{ * // Create Integer values grouped by String key from a Dataset> @@ -68,26 +68,23 @@ abstract class KeyValueGroupedDataset[K, V, DS[U] <: Dataset[U, DS]] extends Ser * * @since 2.1.0 */ - def mapValues[W]( - func: MapFunction[V, W], - encoder: Encoder[W]): KVDS[K, W] = { + def mapValues[W](func: MapFunction[V, W], encoder: Encoder[W]): KVDS[K, W] = { mapValues(ToScalaUDF(func))(encoder) } /** - * Returns a [[Dataset]] that contains each unique key. This is equivalent to doing mapping - * over the Dataset to extract the keys and then running a distinct operation on those. + * Returns a [[Dataset]] that contains each unique key. This is equivalent to doing mapping over + * the Dataset to extract the keys and then running a distinct operation on those. * * @since 1.6.0 */ def keys: DS[K] /** - * (Scala-specific) - * Applies the given function to each group of data. For each unique group, the function will - * be passed the group key and an iterator that contains all of the elements in the group. The - * function can return an iterator containing elements of an arbitrary type which will be returned - * as a new [[Dataset]]. + * (Scala-specific) Applies the given function to each group of data. For each unique group, the + * function will be passed the group key and an iterator that contains all of the elements in + * the group. The function can return an iterator containing elements of an arbitrary type which + * will be returned as a new [[Dataset]]. * * This function does not support partial aggregation, and as a result requires shuffling all * the data in the [[Dataset]]. If an application intends to perform an aggregation over each @@ -95,9 +92,9 @@ abstract class KeyValueGroupedDataset[K, V, DS[U] <: Dataset[U, DS]] extends Ser * `org.apache.spark.sql.expressions#Aggregator`. * * Internally, the implementation will spill to disk if any given group is too large to fit into - * memory. However, users must take care to avoid materializing the whole iterator for a group - * (for example, by calling `toList`) unless they are sure that this is possible given the memory - * constraints of their cluster. + * memory. However, users must take care to avoid materializing the whole iterator for a group + * (for example, by calling `toList`) unless they are sure that this is possible given the + * memory constraints of their cluster. * * @since 1.6.0 */ @@ -106,11 +103,10 @@ abstract class KeyValueGroupedDataset[K, V, DS[U] <: Dataset[U, DS]] extends Ser } /** - * (Java-specific) - * Applies the given function to each group of data. For each unique group, the function will - * be passed the group key and an iterator that contains all of the elements in the group. The - * function can return an iterator containing elements of an arbitrary type which will be returned - * as a new [[Dataset]]. + * (Java-specific) Applies the given function to each group of data. For each unique group, the + * function will be passed the group key and an iterator that contains all of the elements in + * the group. The function can return an iterator containing elements of an arbitrary type which + * will be returned as a new [[Dataset]]. * * This function does not support partial aggregation, and as a result requires shuffling all * the data in the [[Dataset]]. If an application intends to perform an aggregation over each @@ -118,9 +114,9 @@ abstract class KeyValueGroupedDataset[K, V, DS[U] <: Dataset[U, DS]] extends Ser * `org.apache.spark.sql.expressions#Aggregator`. * * Internally, the implementation will spill to disk if any given group is too large to fit into - * memory. However, users must take care to avoid materializing the whole iterator for a group - * (for example, by calling `toList`) unless they are sure that this is possible given the memory - * constraints of their cluster. + * memory. However, users must take care to avoid materializing the whole iterator for a group + * (for example, by calling `toList`) unless they are sure that this is possible given the + * memory constraints of their cluster. * * @since 1.6.0 */ @@ -129,11 +125,10 @@ abstract class KeyValueGroupedDataset[K, V, DS[U] <: Dataset[U, DS]] extends Ser } /** - * (Scala-specific) - * Applies the given function to each group of data. For each unique group, the function will - * be passed the group key and a sorted iterator that contains all of the elements in the group. - * The function can return an iterator containing elements of an arbitrary type which will be - * returned as a new [[Dataset]]. + * (Scala-specific) Applies the given function to each group of data. For each unique group, the + * function will be passed the group key and a sorted iterator that contains all of the elements + * in the group. The function can return an iterator containing elements of an arbitrary type + * which will be returned as a new [[Dataset]]. * * This function does not support partial aggregation, and as a result requires shuffling all * the data in the [[Dataset]]. If an application intends to perform an aggregation over each @@ -141,27 +136,26 @@ abstract class KeyValueGroupedDataset[K, V, DS[U] <: Dataset[U, DS]] extends Ser * `org.apache.spark.sql.expressions#Aggregator`. * * Internally, the implementation will spill to disk if any given group is too large to fit into - * memory. However, users must take care to avoid materializing the whole iterator for a group - * (for example, by calling `toList`) unless they are sure that this is possible given the memory - * constraints of their cluster. + * memory. However, users must take care to avoid materializing the whole iterator for a group + * (for example, by calling `toList`) unless they are sure that this is possible given the + * memory constraints of their cluster. * - * This is equivalent to [[KeyValueGroupedDataset#flatMapGroups]], except for the iterator - * to be sorted according to the given sort expressions. That sorting does not add - * computational complexity. + * This is equivalent to [[KeyValueGroupedDataset#flatMapGroups]], except for the iterator to be + * sorted according to the given sort expressions. That sorting does not add computational + * complexity. * - * @see [[org.apache.spark.sql.api.KeyValueGroupedDataset#flatMapGroups]] + * @see + * [[org.apache.spark.sql.api.KeyValueGroupedDataset#flatMapGroups]] * @since 3.4.0 */ - def flatMapSortedGroups[U: Encoder]( - sortExprs: Column*)( + def flatMapSortedGroups[U: Encoder](sortExprs: Column*)( f: (K, Iterator[V]) => IterableOnce[U]): DS[U] /** - * (Java-specific) - * Applies the given function to each group of data. For each unique group, the function will - * be passed the group key and a sorted iterator that contains all of the elements in the group. - * The function can return an iterator containing elements of an arbitrary type which will be - * returned as a new [[Dataset]]. + * (Java-specific) Applies the given function to each group of data. For each unique group, the + * function will be passed the group key and a sorted iterator that contains all of the elements + * in the group. The function can return an iterator containing elements of an arbitrary type + * which will be returned as a new [[Dataset]]. * * This function does not support partial aggregation, and as a result requires shuffling all * the data in the [[Dataset]]. If an application intends to perform an aggregation over each @@ -169,15 +163,16 @@ abstract class KeyValueGroupedDataset[K, V, DS[U] <: Dataset[U, DS]] extends Ser * `org.apache.spark.sql.expressions#Aggregator`. * * Internally, the implementation will spill to disk if any given group is too large to fit into - * memory. However, users must take care to avoid materializing the whole iterator for a group - * (for example, by calling `toList`) unless they are sure that this is possible given the memory - * constraints of their cluster. + * memory. However, users must take care to avoid materializing the whole iterator for a group + * (for example, by calling `toList`) unless they are sure that this is possible given the + * memory constraints of their cluster. * - * This is equivalent to [[KeyValueGroupedDataset#flatMapGroups]], except for the iterator - * to be sorted according to the given sort expressions. That sorting does not add - * computational complexity. + * This is equivalent to [[KeyValueGroupedDataset#flatMapGroups]], except for the iterator to be + * sorted according to the given sort expressions. That sorting does not add computational + * complexity. * - * @see [[org.apache.spark.sql.api.KeyValueGroupedDataset#flatMapGroups]] + * @see + * [[org.apache.spark.sql.api.KeyValueGroupedDataset#flatMapGroups]] * @since 3.4.0 */ def flatMapSortedGroups[U]( @@ -189,10 +184,10 @@ abstract class KeyValueGroupedDataset[K, V, DS[U] <: Dataset[U, DS]] extends Ser } /** - * (Scala-specific) - * Applies the given function to each group of data. For each unique group, the function will - * be passed the group key and an iterator that contains all of the elements in the group. The - * function can return an element of arbitrary type which will be returned as a new [[Dataset]]. + * (Scala-specific) Applies the given function to each group of data. For each unique group, the + * function will be passed the group key and an iterator that contains all of the elements in + * the group. The function can return an element of arbitrary type which will be returned as a + * new [[Dataset]]. * * This function does not support partial aggregation, and as a result requires shuffling all * the data in the [[Dataset]]. If an application intends to perform an aggregation over each @@ -200,9 +195,9 @@ abstract class KeyValueGroupedDataset[K, V, DS[U] <: Dataset[U, DS]] extends Ser * `org.apache.spark.sql.expressions#Aggregator`. * * Internally, the implementation will spill to disk if any given group is too large to fit into - * memory. However, users must take care to avoid materializing the whole iterator for a group - * (for example, by calling `toList`) unless they are sure that this is possible given the memory - * constraints of their cluster. + * memory. However, users must take care to avoid materializing the whole iterator for a group + * (for example, by calling `toList`) unless they are sure that this is possible given the + * memory constraints of their cluster. * * @since 1.6.0 */ @@ -211,10 +206,10 @@ abstract class KeyValueGroupedDataset[K, V, DS[U] <: Dataset[U, DS]] extends Ser } /** - * (Java-specific) - * Applies the given function to each group of data. For each unique group, the function will - * be passed the group key and an iterator that contains all of the elements in the group. The - * function can return an element of arbitrary type which will be returned as a new [[Dataset]]. + * (Java-specific) Applies the given function to each group of data. For each unique group, the + * function will be passed the group key and an iterator that contains all of the elements in + * the group. The function can return an element of arbitrary type which will be returned as a + * new [[Dataset]]. * * This function does not support partial aggregation, and as a result requires shuffling all * the data in the [[Dataset]]. If an application intends to perform an aggregation over each @@ -222,9 +217,9 @@ abstract class KeyValueGroupedDataset[K, V, DS[U] <: Dataset[U, DS]] extends Ser * `org.apache.spark.sql.expressions#Aggregator`. * * Internally, the implementation will spill to disk if any given group is too large to fit into - * memory. However, users must take care to avoid materializing the whole iterator for a group - * (for example, by calling `toList`) unless they are sure that this is possible given the memory - * constraints of their cluster. + * memory. However, users must take care to avoid materializing the whole iterator for a group + * (for example, by calling `toList`) unless they are sure that this is possible given the + * memory constraints of their cluster. * * @since 1.6.0 */ @@ -233,88 +228,102 @@ abstract class KeyValueGroupedDataset[K, V, DS[U] <: Dataset[U, DS]] extends Ser } /** - * (Scala-specific) - * Applies the given function to each group of data, while maintaining a user-defined per-group - * state. The result Dataset will represent the objects returned by the function. - * For a static batch Dataset, the function will be invoked once per group. For a streaming - * Dataset, the function will be invoked for each group repeatedly in every trigger, and - * updates to each group's state will be saved across invocations. - * See [[org.apache.spark.sql.streaming.GroupState]] for more details. + * (Scala-specific) Applies the given function to each group of data, while maintaining a + * user-defined per-group state. The result Dataset will represent the objects returned by the + * function. For a static batch Dataset, the function will be invoked once per group. For a + * streaming Dataset, the function will be invoked for each group repeatedly in every trigger, + * and updates to each group's state will be saved across invocations. See + * [[org.apache.spark.sql.streaming.GroupState]] for more details. * - * @tparam S The type of the user-defined state. Must be encodable to Spark SQL types. - * @tparam U The type of the output objects. Must be encodable to Spark SQL types. - * @param func Function to be called on every group. + * @tparam S + * The type of the user-defined state. Must be encodable to Spark SQL types. + * @tparam U + * The type of the output objects. Must be encodable to Spark SQL types. + * @param func + * Function to be called on every group. * - * See [[org.apache.spark.sql.Encoder]] for more details on what types are encodable to Spark SQL. + * See [[org.apache.spark.sql.Encoder]] for more details on what types are encodable to Spark + * SQL. * @since 2.2.0 */ - def mapGroupsWithState[S: Encoder, U: Encoder](func: (K, Iterator[V], GroupState[S]) => U): DS[U] + def mapGroupsWithState[S: Encoder, U: Encoder]( + func: (K, Iterator[V], GroupState[S]) => U): DS[U] /** - * (Scala-specific) - * Applies the given function to each group of data, while maintaining a user-defined per-group - * state. The result Dataset will represent the objects returned by the function. - * For a static batch Dataset, the function will be invoked once per group. For a streaming - * Dataset, the function will be invoked for each group repeatedly in every trigger, and - * updates to each group's state will be saved across invocations. - * See [[org.apache.spark.sql.streaming.GroupState]] for more details. - * - * @tparam S The type of the user-defined state. Must be encodable to Spark SQL types. - * @tparam U The type of the output objects. Must be encodable to Spark SQL types. - * @param func Function to be called on every group. - * @param timeoutConf Timeout configuration for groups that do not receive data for a while. - * - * See [[org.apache.spark.sql.Encoder]] for more details on what types are encodable to Spark SQL. + * (Scala-specific) Applies the given function to each group of data, while maintaining a + * user-defined per-group state. The result Dataset will represent the objects returned by the + * function. For a static batch Dataset, the function will be invoked once per group. For a + * streaming Dataset, the function will be invoked for each group repeatedly in every trigger, + * and updates to each group's state will be saved across invocations. See + * [[org.apache.spark.sql.streaming.GroupState]] for more details. + * + * @tparam S + * The type of the user-defined state. Must be encodable to Spark SQL types. + * @tparam U + * The type of the output objects. Must be encodable to Spark SQL types. + * @param func + * Function to be called on every group. + * @param timeoutConf + * Timeout configuration for groups that do not receive data for a while. + * + * See [[org.apache.spark.sql.Encoder]] for more details on what types are encodable to Spark + * SQL. * @since 2.2.0 */ - def mapGroupsWithState[S: Encoder, U: Encoder]( - timeoutConf: GroupStateTimeout)( + def mapGroupsWithState[S: Encoder, U: Encoder](timeoutConf: GroupStateTimeout)( func: (K, Iterator[V], GroupState[S]) => U): DS[U] /** - * (Scala-specific) - * Applies the given function to each group of data, while maintaining a user-defined per-group - * state. The result Dataset will represent the objects returned by the function. - * For a static batch Dataset, the function will be invoked once per group. For a streaming - * Dataset, the function will be invoked for each group repeatedly in every trigger, and - * updates to each group's state will be saved across invocations. - * See [[org.apache.spark.sql.streaming.GroupState]] for more details. - * - * @tparam S The type of the user-defined state. Must be encodable to Spark SQL types. - * @tparam U The type of the output objects. Must be encodable to Spark SQL types. - * @param func Function to be called on every group. - * @param timeoutConf Timeout Conf, see GroupStateTimeout for more details - * @param initialState The user provided state that will be initialized when the first batch - * of data is processed in the streaming query. The user defined function - * will be called on the state data even if there are no other values in - * the group. To convert a Dataset ds of type Dataset[(K, S)] to a - * KeyValueGroupedDataset[K, S] - * do {{{ ds.groupByKey(x => x._1).mapValues(_._2) }}} - * - * See [[org.apache.spark.sql.Encoder]] for more details on what types are encodable to Spark SQL. + * (Scala-specific) Applies the given function to each group of data, while maintaining a + * user-defined per-group state. The result Dataset will represent the objects returned by the + * function. For a static batch Dataset, the function will be invoked once per group. For a + * streaming Dataset, the function will be invoked for each group repeatedly in every trigger, + * and updates to each group's state will be saved across invocations. See + * [[org.apache.spark.sql.streaming.GroupState]] for more details. + * + * @tparam S + * The type of the user-defined state. Must be encodable to Spark SQL types. + * @tparam U + * The type of the output objects. Must be encodable to Spark SQL types. + * @param func + * Function to be called on every group. + * @param timeoutConf + * Timeout Conf, see GroupStateTimeout for more details + * @param initialState + * The user provided state that will be initialized when the first batch of data is processed + * in the streaming query. The user defined function will be called on the state data even if + * there are no other values in the group. To convert a Dataset ds of type Dataset[(K, S)] to + * a KeyValueGroupedDataset[K, S] do {{{ds.groupByKey(x => x._1).mapValues(_._2)}}} + * + * See [[org.apache.spark.sql.Encoder]] for more details on what types are encodable to Spark + * SQL. * @since 3.2.0 */ def mapGroupsWithState[S: Encoder, U: Encoder]( timeoutConf: GroupStateTimeout, - initialState: KVDS[K, S])( - func: (K, Iterator[V], GroupState[S]) => U): DS[U] - - /** - * (Java-specific) - * Applies the given function to each group of data, while maintaining a user-defined per-group - * state. The result Dataset will represent the objects returned by the function. - * For a static batch Dataset, the function will be invoked once per group. For a streaming - * Dataset, the function will be invoked for each group repeatedly in every trigger, and - * updates to each group's state will be saved across invocations. - * See `GroupState` for more details. - * - * @tparam S The type of the user-defined state. Must be encodable to Spark SQL types. - * @tparam U The type of the output objects. Must be encodable to Spark SQL types. - * @param func Function to be called on every group. - * @param stateEncoder Encoder for the state type. - * @param outputEncoder Encoder for the output type. - * - * See [[org.apache.spark.sql.Encoder]] for more details on what types are encodable to Spark SQL. + initialState: KVDS[K, S])(func: (K, Iterator[V], GroupState[S]) => U): DS[U] + + /** + * (Java-specific) Applies the given function to each group of data, while maintaining a + * user-defined per-group state. The result Dataset will represent the objects returned by the + * function. For a static batch Dataset, the function will be invoked once per group. For a + * streaming Dataset, the function will be invoked for each group repeatedly in every trigger, + * and updates to each group's state will be saved across invocations. See `GroupState` for more + * details. + * + * @tparam S + * The type of the user-defined state. Must be encodable to Spark SQL types. + * @tparam U + * The type of the output objects. Must be encodable to Spark SQL types. + * @param func + * Function to be called on every group. + * @param stateEncoder + * Encoder for the state type. + * @param outputEncoder + * Encoder for the output type. + * + * See [[org.apache.spark.sql.Encoder]] for more details on what types are encodable to Spark + * SQL. * @since 2.2.0 */ def mapGroupsWithState[S, U]( @@ -325,22 +334,28 @@ abstract class KeyValueGroupedDataset[K, V, DS[U] <: Dataset[U, DS]] extends Ser } /** - * (Java-specific) - * Applies the given function to each group of data, while maintaining a user-defined per-group - * state. The result Dataset will represent the objects returned by the function. - * For a static batch Dataset, the function will be invoked once per group. For a streaming - * Dataset, the function will be invoked for each group repeatedly in every trigger, and - * updates to each group's state will be saved across invocations. - * See `GroupState` for more details. - * - * @tparam S The type of the user-defined state. Must be encodable to Spark SQL types. - * @tparam U The type of the output objects. Must be encodable to Spark SQL types. - * @param func Function to be called on every group. - * @param stateEncoder Encoder for the state type. - * @param outputEncoder Encoder for the output type. - * @param timeoutConf Timeout configuration for groups that do not receive data for a while. - * - * See [[org.apache.spark.sql.Encoder]] for more details on what types are encodable to Spark SQL. + * (Java-specific) Applies the given function to each group of data, while maintaining a + * user-defined per-group state. The result Dataset will represent the objects returned by the + * function. For a static batch Dataset, the function will be invoked once per group. For a + * streaming Dataset, the function will be invoked for each group repeatedly in every trigger, + * and updates to each group's state will be saved across invocations. See `GroupState` for more + * details. + * + * @tparam S + * The type of the user-defined state. Must be encodable to Spark SQL types. + * @tparam U + * The type of the output objects. Must be encodable to Spark SQL types. + * @param func + * Function to be called on every group. + * @param stateEncoder + * Encoder for the state type. + * @param outputEncoder + * Encoder for the output type. + * @param timeoutConf + * Timeout configuration for groups that do not receive data for a while. + * + * See [[org.apache.spark.sql.Encoder]] for more details on what types are encodable to Spark + * SQL. * @since 2.2.0 */ def mapGroupsWithState[S, U]( @@ -352,26 +367,32 @@ abstract class KeyValueGroupedDataset[K, V, DS[U] <: Dataset[U, DS]] extends Ser } /** - * (Java-specific) - * Applies the given function to each group of data, while maintaining a user-defined per-group - * state. The result Dataset will represent the objects returned by the function. - * For a static batch Dataset, the function will be invoked once per group. For a streaming - * Dataset, the function will be invoked for each group repeatedly in every trigger, and - * updates to each group's state will be saved across invocations. - * See `GroupState` for more details. - * - * @tparam S The type of the user-defined state. Must be encodable to Spark SQL types. - * @tparam U The type of the output objects. Must be encodable to Spark SQL types. - * @param func Function to be called on every group. - * @param stateEncoder Encoder for the state type. - * @param outputEncoder Encoder for the output type. - * @param timeoutConf Timeout configuration for groups that do not receive data for a while. - * @param initialState The user provided state that will be initialized when the first batch - * of data is processed in the streaming query. The user defined function - * will be called on the state data even if there are no other values in - * the group. - * - * See [[org.apache.spark.sql.Encoder]] for more details on what types are encodable to Spark SQL. + * (Java-specific) Applies the given function to each group of data, while maintaining a + * user-defined per-group state. The result Dataset will represent the objects returned by the + * function. For a static batch Dataset, the function will be invoked once per group. For a + * streaming Dataset, the function will be invoked for each group repeatedly in every trigger, + * and updates to each group's state will be saved across invocations. See `GroupState` for more + * details. + * + * @tparam S + * The type of the user-defined state. Must be encodable to Spark SQL types. + * @tparam U + * The type of the output objects. Must be encodable to Spark SQL types. + * @param func + * Function to be called on every group. + * @param stateEncoder + * Encoder for the state type. + * @param outputEncoder + * Encoder for the output type. + * @param timeoutConf + * Timeout configuration for groups that do not receive data for a while. + * @param initialState + * The user provided state that will be initialized when the first batch of data is processed + * in the streaming query. The user defined function will be called on the state data even if + * there are no other values in the group. + * + * See [[org.apache.spark.sql.Encoder]] for more details on what types are encodable to Spark + * SQL. * @since 3.2.0 */ def mapGroupsWithState[S, U]( @@ -385,75 +406,89 @@ abstract class KeyValueGroupedDataset[K, V, DS[U] <: Dataset[U, DS]] extends Ser } /** - * (Scala-specific) - * Applies the given function to each group of data, while maintaining a user-defined per-group - * state. The result Dataset will represent the objects returned by the function. - * For a static batch Dataset, the function will be invoked once per group. For a streaming - * Dataset, the function will be invoked for each group repeatedly in every trigger, and - * updates to each group's state will be saved across invocations. - * See `GroupState` for more details. - * - * @tparam S The type of the user-defined state. Must be encodable to Spark SQL types. - * @tparam U The type of the output objects. Must be encodable to Spark SQL types. - * @param func Function to be called on every group. - * @param outputMode The output mode of the function. - * @param timeoutConf Timeout configuration for groups that do not receive data for a while. - * - * See [[org.apache.spark.sql.Encoder]] for more details on what types are encodable to Spark SQL. + * (Scala-specific) Applies the given function to each group of data, while maintaining a + * user-defined per-group state. The result Dataset will represent the objects returned by the + * function. For a static batch Dataset, the function will be invoked once per group. For a + * streaming Dataset, the function will be invoked for each group repeatedly in every trigger, + * and updates to each group's state will be saved across invocations. See `GroupState` for more + * details. + * + * @tparam S + * The type of the user-defined state. Must be encodable to Spark SQL types. + * @tparam U + * The type of the output objects. Must be encodable to Spark SQL types. + * @param func + * Function to be called on every group. + * @param outputMode + * The output mode of the function. + * @param timeoutConf + * Timeout configuration for groups that do not receive data for a while. + * + * See [[org.apache.spark.sql.Encoder]] for more details on what types are encodable to Spark + * SQL. * @since 2.2.0 */ def flatMapGroupsWithState[S: Encoder, U: Encoder]( outputMode: OutputMode, - timeoutConf: GroupStateTimeout)( - func: (K, Iterator[V], GroupState[S]) => Iterator[U]): DS[U] - - /** - * (Scala-specific) - * Applies the given function to each group of data, while maintaining a user-defined per-group - * state. The result Dataset will represent the objects returned by the function. - * For a static batch Dataset, the function will be invoked once per group. For a streaming - * Dataset, the function will be invoked for each group repeatedly in every trigger, and - * updates to each group's state will be saved across invocations. - * See `GroupState` for more details. - * - * @tparam S The type of the user-defined state. Must be encodable to Spark SQL types. - * @tparam U The type of the output objects. Must be encodable to Spark SQL types. - * @param func Function to be called on every group. - * @param outputMode The output mode of the function. - * @param timeoutConf Timeout configuration for groups that do not receive data for a while. - * @param initialState The user provided state that will be initialized when the first batch - * of data is processed in the streaming query. The user defined function - * will be called on the state data even if there are no other values in - * the group. To covert a Dataset `ds` of type of type `Dataset[(K, S)]` - * to a `KeyValueGroupedDataset[K, S]`, use - * {{{ ds.groupByKey(x => x._1).mapValues(_._2) }}} - * See [[org.apache.spark.sql.Encoder]] for more details on what types are encodable to Spark SQL. + timeoutConf: GroupStateTimeout)(func: (K, Iterator[V], GroupState[S]) => Iterator[U]): DS[U] + + /** + * (Scala-specific) Applies the given function to each group of data, while maintaining a + * user-defined per-group state. The result Dataset will represent the objects returned by the + * function. For a static batch Dataset, the function will be invoked once per group. For a + * streaming Dataset, the function will be invoked for each group repeatedly in every trigger, + * and updates to each group's state will be saved across invocations. See `GroupState` for more + * details. + * + * @tparam S + * The type of the user-defined state. Must be encodable to Spark SQL types. + * @tparam U + * The type of the output objects. Must be encodable to Spark SQL types. + * @param func + * Function to be called on every group. + * @param outputMode + * The output mode of the function. + * @param timeoutConf + * Timeout configuration for groups that do not receive data for a while. + * @param initialState + * The user provided state that will be initialized when the first batch of data is processed + * in the streaming query. The user defined function will be called on the state data even if + * there are no other values in the group. To covert a Dataset `ds` of type of type + * `Dataset[(K, S)]` to a `KeyValueGroupedDataset[K, S]`, use + * {{{ds.groupByKey(x => x._1).mapValues(_._2)}}} See [[org.apache.spark.sql.Encoder]] for + * more details on what types are encodable to Spark SQL. * @since 3.2.0 */ def flatMapGroupsWithState[S: Encoder, U: Encoder]( outputMode: OutputMode, timeoutConf: GroupStateTimeout, - initialState: KVDS[K, S])( - func: (K, Iterator[V], GroupState[S]) => Iterator[U]): DS[U] - - /** - * (Java-specific) - * Applies the given function to each group of data, while maintaining a user-defined per-group - * state. The result Dataset will represent the objects returned by the function. - * For a static batch Dataset, the function will be invoked once per group. For a streaming - * Dataset, the function will be invoked for each group repeatedly in every trigger, and - * updates to each group's state will be saved across invocations. - * See `GroupState` for more details. - * - * @tparam S The type of the user-defined state. Must be encodable to Spark SQL types. - * @tparam U The type of the output objects. Must be encodable to Spark SQL types. - * @param func Function to be called on every group. - * @param outputMode The output mode of the function. - * @param stateEncoder Encoder for the state type. - * @param outputEncoder Encoder for the output type. - * @param timeoutConf Timeout configuration for groups that do not receive data for a while. - * - * See [[org.apache.spark.sql.Encoder]] for more details on what types are encodable to Spark SQL. + initialState: KVDS[K, S])(func: (K, Iterator[V], GroupState[S]) => Iterator[U]): DS[U] + + /** + * (Java-specific) Applies the given function to each group of data, while maintaining a + * user-defined per-group state. The result Dataset will represent the objects returned by the + * function. For a static batch Dataset, the function will be invoked once per group. For a + * streaming Dataset, the function will be invoked for each group repeatedly in every trigger, + * and updates to each group's state will be saved across invocations. See `GroupState` for more + * details. + * + * @tparam S + * The type of the user-defined state. Must be encodable to Spark SQL types. + * @tparam U + * The type of the output objects. Must be encodable to Spark SQL types. + * @param func + * Function to be called on every group. + * @param outputMode + * The output mode of the function. + * @param stateEncoder + * Encoder for the state type. + * @param outputEncoder + * Encoder for the output type. + * @param timeoutConf + * Timeout configuration for groups that do not receive data for a while. + * + * See [[org.apache.spark.sql.Encoder]] for more details on what types are encodable to Spark + * SQL. * @since 2.2.0 */ def flatMapGroupsWithState[S, U]( @@ -467,29 +502,36 @@ abstract class KeyValueGroupedDataset[K, V, DS[U] <: Dataset[U, DS]] extends Ser } /** - * (Java-specific) - * Applies the given function to each group of data, while maintaining a user-defined per-group - * state. The result Dataset will represent the objects returned by the function. - * For a static batch Dataset, the function will be invoked once per group. For a streaming - * Dataset, the function will be invoked for each group repeatedly in every trigger, and - * updates to each group's state will be saved across invocations. - * See `GroupState` for more details. - * - * @tparam S The type of the user-defined state. Must be encodable to Spark SQL types. - * @tparam U The type of the output objects. Must be encodable to Spark SQL types. - * @param func Function to be called on every group. - * @param outputMode The output mode of the function. - * @param stateEncoder Encoder for the state type. - * @param outputEncoder Encoder for the output type. - * @param timeoutConf Timeout configuration for groups that do not receive data for a while. - * @param initialState The user provided state that will be initialized when the first batch - * of data is processed in the streaming query. The user defined function - * will be called on the state data even if there are no other values in - * the group. To covert a Dataset `ds` of type of type `Dataset[(K, S)]` - * to a `KeyValueGroupedDataset[K, S]`, use - * {{{ ds.groupByKey(x => x._1).mapValues(_._2) }}} - * - * See [[org.apache.spark.sql.Encoder]] for more details on what types are encodable to Spark SQL. + * (Java-specific) Applies the given function to each group of data, while maintaining a + * user-defined per-group state. The result Dataset will represent the objects returned by the + * function. For a static batch Dataset, the function will be invoked once per group. For a + * streaming Dataset, the function will be invoked for each group repeatedly in every trigger, + * and updates to each group's state will be saved across invocations. See `GroupState` for more + * details. + * + * @tparam S + * The type of the user-defined state. Must be encodable to Spark SQL types. + * @tparam U + * The type of the output objects. Must be encodable to Spark SQL types. + * @param func + * Function to be called on every group. + * @param outputMode + * The output mode of the function. + * @param stateEncoder + * Encoder for the state type. + * @param outputEncoder + * Encoder for the output type. + * @param timeoutConf + * Timeout configuration for groups that do not receive data for a while. + * @param initialState + * The user provided state that will be initialized when the first batch of data is processed + * in the streaming query. The user defined function will be called on the state data even if + * there are no other values in the group. To covert a Dataset `ds` of type of type + * `Dataset[(K, S)]` to a `KeyValueGroupedDataset[K, S]`, use + * {{{ds.groupByKey(x => x._1).mapValues(_._2)}}} + * + * See [[org.apache.spark.sql.Encoder]] for more details on what types are encodable to Spark + * SQL. * @since 3.2.0 */ def flatMapGroupsWithState[S, U]( @@ -499,32 +541,29 @@ abstract class KeyValueGroupedDataset[K, V, DS[U] <: Dataset[U, DS]] extends Ser outputEncoder: Encoder[U], timeoutConf: GroupStateTimeout, initialState: KVDS[K, S]): DS[U] = { - flatMapGroupsWithState[S, U]( - outputMode, - timeoutConf, - initialState)( - ToScalaUDF(func))( + flatMapGroupsWithState[S, U](outputMode, timeoutConf, initialState)(ToScalaUDF(func))( stateEncoder, outputEncoder) } - /** - * (Scala-specific) - * Invokes methods defined in the stateful processor used in arbitrary state API v2. - * We allow the user to act on per-group set of input rows along with keyed state and the - * user can choose to output/return 0 or more rows. - * For a streaming dataframe, we will repeatedly invoke the interface methods for new rows - * in each trigger and the user's state/state variables will be stored persistently across - * invocations. + * (Scala-specific) Invokes methods defined in the stateful processor used in arbitrary state + * API v2. We allow the user to act on per-group set of input rows along with keyed state and + * the user can choose to output/return 0 or more rows. For a streaming dataframe, we will + * repeatedly invoke the interface methods for new rows in each trigger and the user's + * state/state variables will be stored persistently across invocations. * - * @tparam U The type of the output objects. Must be encodable to Spark SQL types. - * @param statefulProcessor Instance of statefulProcessor whose functions will be invoked - * by the operator. - * @param timeMode The time mode semantics of the stateful processor for timers and TTL. - * @param outputMode The output mode of the stateful processor. + * @tparam U + * The type of the output objects. Must be encodable to Spark SQL types. + * @param statefulProcessor + * Instance of statefulProcessor whose functions will be invoked by the operator. + * @param timeMode + * The time mode semantics of the stateful processor for timers and TTL. + * @param outputMode + * The output mode of the stateful processor. * - * See [[org.apache.spark.sql.Encoder]] for more details on what types are encodable to Spark SQL. + * See [[org.apache.spark.sql.Encoder]] for more details on what types are encodable to Spark + * SQL. */ private[sql] def transformWithState[U: Encoder]( statefulProcessor: StatefulProcessor[K, V, U], @@ -532,27 +571,28 @@ abstract class KeyValueGroupedDataset[K, V, DS[U] <: Dataset[U, DS]] extends Ser outputMode: OutputMode): DS[U] /** - * (Scala-specific) - * Invokes methods defined in the stateful processor used in arbitrary state API v2. - * We allow the user to act on per-group set of input rows along with keyed state and the - * user can choose to output/return 0 or more rows. - * For a streaming dataframe, we will repeatedly invoke the interface methods for new rows - * in each trigger and the user's state/state variables will be stored persistently across - * invocations. + * (Scala-specific) Invokes methods defined in the stateful processor used in arbitrary state + * API v2. We allow the user to act on per-group set of input rows along with keyed state and + * the user can choose to output/return 0 or more rows. For a streaming dataframe, we will + * repeatedly invoke the interface methods for new rows in each trigger and the user's + * state/state variables will be stored persistently across invocations. * - * Downstream operators would use specified eventTimeColumnName to calculate watermark. - * Note that TimeMode is set to EventTime to ensure correct flow of watermark. + * Downstream operators would use specified eventTimeColumnName to calculate watermark. Note + * that TimeMode is set to EventTime to ensure correct flow of watermark. * - * @tparam U The type of the output objects. Must be encodable to Spark SQL types. - * @param statefulProcessor Instance of statefulProcessor whose functions will - * be invoked by the operator. - * @param eventTimeColumnName eventTime column in the output dataset. Any operations after - * transformWithState will use the new eventTimeColumn. The user - * needs to ensure that the eventTime for emitted output adheres to - * the watermark boundary, otherwise streaming query will fail. - * @param outputMode The output mode of the stateful processor. + * @tparam U + * The type of the output objects. Must be encodable to Spark SQL types. + * @param statefulProcessor + * Instance of statefulProcessor whose functions will be invoked by the operator. + * @param eventTimeColumnName + * eventTime column in the output dataset. Any operations after transformWithState will use + * the new eventTimeColumn. The user needs to ensure that the eventTime for emitted output + * adheres to the watermark boundary, otherwise streaming query will fail. + * @param outputMode + * The output mode of the stateful processor. * - * See [[org.apache.spark.sql.Encoder]] for more details on what types are encodable to Spark SQL. + * See [[org.apache.spark.sql.Encoder]] for more details on what types are encodable to Spark + * SQL. */ private[sql] def transformWithState[U: Encoder]( statefulProcessor: StatefulProcessor[K, V, U], @@ -560,22 +600,25 @@ abstract class KeyValueGroupedDataset[K, V, DS[U] <: Dataset[U, DS]] extends Ser outputMode: OutputMode): DS[U] /** - * (Java-specific) - * Invokes methods defined in the stateful processor used in arbitrary state API v2. - * We allow the user to act on per-group set of input rows along with keyed state and the - * user can choose to output/return 0 or more rows. - * For a streaming dataframe, we will repeatedly invoke the interface methods for new rows - * in each trigger and the user's state/state variables will be stored persistently across - * invocations. + * (Java-specific) Invokes methods defined in the stateful processor used in arbitrary state API + * v2. We allow the user to act on per-group set of input rows along with keyed state and the + * user can choose to output/return 0 or more rows. For a streaming dataframe, we will + * repeatedly invoke the interface methods for new rows in each trigger and the user's + * state/state variables will be stored persistently across invocations. * - * @tparam U The type of the output objects. Must be encodable to Spark SQL types. - * @param statefulProcessor Instance of statefulProcessor whose functions will be invoked by the - * operator. - * @param timeMode The time mode semantics of the stateful processor for timers and TTL. - * @param outputMode The output mode of the stateful processor. - * @param outputEncoder Encoder for the output type. + * @tparam U + * The type of the output objects. Must be encodable to Spark SQL types. + * @param statefulProcessor + * Instance of statefulProcessor whose functions will be invoked by the operator. + * @param timeMode + * The time mode semantics of the stateful processor for timers and TTL. + * @param outputMode + * The output mode of the stateful processor. + * @param outputEncoder + * Encoder for the output type. * - * See [[org.apache.spark.sql.Encoder]] for more details on what types are encodable to Spark SQL. + * See [[org.apache.spark.sql.Encoder]] for more details on what types are encodable to Spark + * SQL. */ private[sql] def transformWithState[U: Encoder]( statefulProcessor: StatefulProcessor[K, V, U], @@ -586,29 +629,32 @@ abstract class KeyValueGroupedDataset[K, V, DS[U] <: Dataset[U, DS]] extends Ser } /** - * (Java-specific) - * Invokes methods defined in the stateful processor used in arbitrary state API v2. - * We allow the user to act on per-group set of input rows along with keyed state and the + * (Java-specific) Invokes methods defined in the stateful processor used in arbitrary state API + * v2. We allow the user to act on per-group set of input rows along with keyed state and the * user can choose to output/return 0 or more rows. * - * For a streaming dataframe, we will repeatedly invoke the interface methods for new rows - * in each trigger and the user's state/state variables will be stored persistently across + * For a streaming dataframe, we will repeatedly invoke the interface methods for new rows in + * each trigger and the user's state/state variables will be stored persistently across * invocations. * - * Downstream operators would use specified eventTimeColumnName to calculate watermark. - * Note that TimeMode is set to EventTime to ensure correct flow of watermark. - * - * @tparam U The type of the output objects. Must be encodable to Spark SQL types. - * @param statefulProcessor Instance of statefulProcessor whose functions will be invoked by the - * operator. - * @param eventTimeColumnName eventTime column in the output dataset. Any operations after - * transformWithState will use the new eventTimeColumn. The user - * needs to ensure that the eventTime for emitted output adheres to - * the watermark boundary, otherwise streaming query will fail. - * @param outputMode The output mode of the stateful processor. - * @param outputEncoder Encoder for the output type. - * - * See [[org.apache.spark.sql.Encoder]] for more details on what types are encodable to Spark SQL. + * Downstream operators would use specified eventTimeColumnName to calculate watermark. Note + * that TimeMode is set to EventTime to ensure correct flow of watermark. + * + * @tparam U + * The type of the output objects. Must be encodable to Spark SQL types. + * @param statefulProcessor + * Instance of statefulProcessor whose functions will be invoked by the operator. + * @param eventTimeColumnName + * eventTime column in the output dataset. Any operations after transformWithState will use + * the new eventTimeColumn. The user needs to ensure that the eventTime for emitted output + * adheres to the watermark boundary, otherwise streaming query will fail. + * @param outputMode + * The output mode of the stateful processor. + * @param outputEncoder + * Encoder for the output type. + * + * See [[org.apache.spark.sql.Encoder]] for more details on what types are encodable to Spark + * SQL. */ private[sql] def transformWithState[U: Encoder]( statefulProcessor: StatefulProcessor[K, V, U], @@ -619,20 +665,25 @@ abstract class KeyValueGroupedDataset[K, V, DS[U] <: Dataset[U, DS]] extends Ser } /** - * (Scala-specific) - * Invokes methods defined in the stateful processor used in arbitrary state API v2. - * Functions as the function above, but with additional initial state. + * (Scala-specific) Invokes methods defined in the stateful processor used in arbitrary state + * API v2. Functions as the function above, but with additional initial state. * - * @tparam U The type of the output objects. Must be encodable to Spark SQL types. - * @tparam S The type of initial state objects. Must be encodable to Spark SQL types. - * @param statefulProcessor Instance of statefulProcessor whose functions will - * be invoked by the operator. - * @param timeMode The time mode semantics of the stateful processor for timers and TTL. - * @param outputMode The output mode of the stateful processor. - * @param initialState User provided initial state that will be used to initiate state for - * the query in the first batch. + * @tparam U + * The type of the output objects. Must be encodable to Spark SQL types. + * @tparam S + * The type of initial state objects. Must be encodable to Spark SQL types. + * @param statefulProcessor + * Instance of statefulProcessor whose functions will be invoked by the operator. + * @param timeMode + * The time mode semantics of the stateful processor for timers and TTL. + * @param outputMode + * The output mode of the stateful processor. + * @param initialState + * User provided initial state that will be used to initiate state for the query in the first + * batch. * - * See [[org.apache.spark.sql.Encoder]] for more details on what types are encodable to Spark SQL. + * See [[org.apache.spark.sql.Encoder]] for more details on what types are encodable to Spark + * SQL. */ private[sql] def transformWithState[U: Encoder, S: Encoder]( statefulProcessor: StatefulProcessorWithInitialState[K, V, U, S], @@ -641,27 +692,31 @@ abstract class KeyValueGroupedDataset[K, V, DS[U] <: Dataset[U, DS]] extends Ser initialState: KVDS[K, S]): DS[U] /** - * (Scala-specific) - * Invokes methods defined in the stateful processor used in arbitrary state API v2. - * Functions as the function above, but with additional eventTimeColumnName for output. + * (Scala-specific) Invokes methods defined in the stateful processor used in arbitrary state + * API v2. Functions as the function above, but with additional eventTimeColumnName for output. * - * @tparam U The type of the output objects. Must be encodable to Spark SQL types. - * @tparam S The type of initial state objects. Must be encodable to Spark SQL types. + * @tparam U + * The type of the output objects. Must be encodable to Spark SQL types. + * @tparam S + * The type of initial state objects. Must be encodable to Spark SQL types. * - * Downstream operators would use specified eventTimeColumnName to calculate watermark. - * Note that TimeMode is set to EventTime to ensure correct flow of watermark. + * Downstream operators would use specified eventTimeColumnName to calculate watermark. Note + * that TimeMode is set to EventTime to ensure correct flow of watermark. * - * @param statefulProcessor Instance of statefulProcessor whose functions will - * be invoked by the operator. - * @param eventTimeColumnName eventTime column in the output dataset. Any operations after - * transformWithState will use the new eventTimeColumn. The user - * needs to ensure that the eventTime for emitted output adheres to - * the watermark boundary, otherwise streaming query will fail. - * @param outputMode The output mode of the stateful processor. - * @param initialState User provided initial state that will be used to initiate state for - * the query in the first batch. + * @param statefulProcessor + * Instance of statefulProcessor whose functions will be invoked by the operator. + * @param eventTimeColumnName + * eventTime column in the output dataset. Any operations after transformWithState will use + * the new eventTimeColumn. The user needs to ensure that the eventTime for emitted output + * adheres to the watermark boundary, otherwise streaming query will fail. + * @param outputMode + * The output mode of the stateful processor. + * @param initialState + * User provided initial state that will be used to initiate state for the query in the first + * batch. * - * See [[org.apache.spark.sql.Encoder]] for more details on what types are encodable to Spark SQL. + * See [[org.apache.spark.sql.Encoder]] for more details on what types are encodable to Spark + * SQL. */ private[sql] def transformWithState[U: Encoder, S: Encoder]( statefulProcessor: StatefulProcessorWithInitialState[K, V, U, S], @@ -670,23 +725,30 @@ abstract class KeyValueGroupedDataset[K, V, DS[U] <: Dataset[U, DS]] extends Ser initialState: KVDS[K, S]): DS[U] /** - * (Java-specific) - * Invokes methods defined in the stateful processor used in arbitrary state API v2. - * Functions as the function above, but with additional initialStateEncoder for state encoding. - * - * @tparam U The type of the output objects. Must be encodable to Spark SQL types. - * @tparam S The type of initial state objects. Must be encodable to Spark SQL types. - * @param statefulProcessor Instance of statefulProcessor whose functions will - * be invoked by the operator. - * @param timeMode The time mode semantics of the stateful processor for - * timers and TTL. - * @param outputMode The output mode of the stateful processor. - * @param initialState User provided initial state that will be used to initiate state for - * the query in the first batch. - * @param outputEncoder Encoder for the output type. - * @param initialStateEncoder Encoder for the initial state type. - * - * See [[org.apache.spark.sql.Encoder]] for more details on what types are encodable to Spark SQL. + * (Java-specific) Invokes methods defined in the stateful processor used in arbitrary state API + * v2. Functions as the function above, but with additional initialStateEncoder for state + * encoding. + * + * @tparam U + * The type of the output objects. Must be encodable to Spark SQL types. + * @tparam S + * The type of initial state objects. Must be encodable to Spark SQL types. + * @param statefulProcessor + * Instance of statefulProcessor whose functions will be invoked by the operator. + * @param timeMode + * The time mode semantics of the stateful processor for timers and TTL. + * @param outputMode + * The output mode of the stateful processor. + * @param initialState + * User provided initial state that will be used to initiate state for the query in the first + * batch. + * @param outputEncoder + * Encoder for the output type. + * @param initialStateEncoder + * Encoder for the initial state type. + * + * See [[org.apache.spark.sql.Encoder]] for more details on what types are encodable to Spark + * SQL. */ private[sql] def transformWithState[U: Encoder, S: Encoder]( statefulProcessor: StatefulProcessorWithInitialState[K, V, U, S], @@ -695,33 +757,40 @@ abstract class KeyValueGroupedDataset[K, V, DS[U] <: Dataset[U, DS]] extends Ser initialState: KVDS[K, S], outputEncoder: Encoder[U], initialStateEncoder: Encoder[S]): DS[U] = { - transformWithState(statefulProcessor, timeMode, - outputMode, initialState)(outputEncoder, initialStateEncoder) + transformWithState(statefulProcessor, timeMode, outputMode, initialState)( + outputEncoder, + initialStateEncoder) } /** - * (Java-specific) - * Invokes methods defined in the stateful processor used in arbitrary state API v2. - * Functions as the function above, but with additional eventTimeColumnName for output. - * - * Downstream operators would use specified eventTimeColumnName to calculate watermark. - * Note that TimeMode is set to EventTime to ensure correct flow of watermark. - * - * @tparam U The type of the output objects. Must be encodable to Spark SQL types. - * @tparam S The type of initial state objects. Must be encodable to Spark SQL types. - * @param statefulProcessor Instance of statefulProcessor whose functions will - * be invoked by the operator. - * @param outputMode The output mode of the stateful processor. - * @param initialState User provided initial state that will be used to initiate state for - * the query in the first batch. - * @param eventTimeColumnName event column in the output dataset. Any operations after - * transformWithState will use the new eventTimeColumn. The user - * needs to ensure that the eventTime for emitted output adheres to - * the watermark boundary, otherwise streaming query will fail. - * @param outputEncoder Encoder for the output type. - * @param initialStateEncoder Encoder for the initial state type. - * - * See [[org.apache.spark.sql.Encoder]] for more details on what types are encodable to Spark SQL. + * (Java-specific) Invokes methods defined in the stateful processor used in arbitrary state API + * v2. Functions as the function above, but with additional eventTimeColumnName for output. + * + * Downstream operators would use specified eventTimeColumnName to calculate watermark. Note + * that TimeMode is set to EventTime to ensure correct flow of watermark. + * + * @tparam U + * The type of the output objects. Must be encodable to Spark SQL types. + * @tparam S + * The type of initial state objects. Must be encodable to Spark SQL types. + * @param statefulProcessor + * Instance of statefulProcessor whose functions will be invoked by the operator. + * @param outputMode + * The output mode of the stateful processor. + * @param initialState + * User provided initial state that will be used to initiate state for the query in the first + * batch. + * @param eventTimeColumnName + * event column in the output dataset. Any operations after transformWithState will use the + * new eventTimeColumn. The user needs to ensure that the eventTime for emitted output adheres + * to the watermark boundary, otherwise streaming query will fail. + * @param outputEncoder + * Encoder for the output type. + * @param initialStateEncoder + * Encoder for the initial state type. + * + * See [[org.apache.spark.sql.Encoder]] for more details on what types are encodable to Spark + * SQL. */ private[sql] def transformWithState[U: Encoder, S: Encoder]( statefulProcessor: StatefulProcessorWithInitialState[K, V, U, S], @@ -730,22 +799,24 @@ abstract class KeyValueGroupedDataset[K, V, DS[U] <: Dataset[U, DS]] extends Ser eventTimeColumnName: String, outputEncoder: Encoder[U], initialStateEncoder: Encoder[S]): DS[U] = { - transformWithState(statefulProcessor, eventTimeColumnName, - outputMode, initialState)(outputEncoder, initialStateEncoder) + transformWithState(statefulProcessor, eventTimeColumnName, outputMode, initialState)( + outputEncoder, + initialStateEncoder) } + /** - * (Scala-specific) - * Reduces the elements of each group of data using the specified binary function. - * The given function must be commutative and associative or the result may be non-deterministic. + * (Scala-specific) Reduces the elements of each group of data using the specified binary + * function. The given function must be commutative and associative or the result may be + * non-deterministic. * * @since 1.6.0 */ def reduceGroups(f: (V, V) => V): DS[(K, V)] /** - * (Java-specific) - * Reduces the elements of each group of data using the specified binary function. - * The given function must be commutative and associative or the result may be non-deterministic. + * (Java-specific) Reduces the elements of each group of data using the specified binary + * function. The given function must be commutative and associative or the result may be + * non-deterministic. * * @since 1.6.0 */ @@ -754,15 +825,15 @@ abstract class KeyValueGroupedDataset[K, V, DS[U] <: Dataset[U, DS]] extends Ser } /** - * Internal helper function for building typed aggregations that return tuples. For simplicity + * Internal helper function for building typed aggregations that return tuples. For simplicity * and code reuse, we do this without the help of the type system and then use helper functions * that cast appropriately for the user facing interface. */ protected def aggUntyped(columns: TypedColumn[_, _]*): DS[_] /** - * Computes the given aggregation, returning a [[Dataset]] of tuples for each unique key - * and the result of computing this aggregation over all elements in the group. + * Computes the given aggregation, returning a [[Dataset]] of tuples for each unique key and the + * result of computing this aggregation over all elements in the group. * * @since 1.6.0 */ @@ -770,8 +841,8 @@ abstract class KeyValueGroupedDataset[K, V, DS[U] <: Dataset[U, DS]] extends Ser aggUntyped(col1).asInstanceOf[DS[(K, U1)]] /** - * Computes the given aggregations, returning a [[Dataset]] of tuples for each unique key - * and the result of computing these aggregations over all elements in the group. + * Computes the given aggregations, returning a [[Dataset]] of tuples for each unique key and + * the result of computing these aggregations over all elements in the group. * * @since 1.6.0 */ @@ -779,8 +850,8 @@ abstract class KeyValueGroupedDataset[K, V, DS[U] <: Dataset[U, DS]] extends Ser aggUntyped(col1, col2).asInstanceOf[DS[(K, U1, U2)]] /** - * Computes the given aggregations, returning a [[Dataset]] of tuples for each unique key - * and the result of computing these aggregations over all elements in the group. + * Computes the given aggregations, returning a [[Dataset]] of tuples for each unique key and + * the result of computing these aggregations over all elements in the group. * * @since 1.6.0 */ @@ -791,8 +862,8 @@ abstract class KeyValueGroupedDataset[K, V, DS[U] <: Dataset[U, DS]] extends Ser aggUntyped(col1, col2, col3).asInstanceOf[DS[(K, U1, U2, U3)]] /** - * Computes the given aggregations, returning a [[Dataset]] of tuples for each unique key - * and the result of computing these aggregations over all elements in the group. + * Computes the given aggregations, returning a [[Dataset]] of tuples for each unique key and + * the result of computing these aggregations over all elements in the group. * * @since 1.6.0 */ @@ -804,8 +875,8 @@ abstract class KeyValueGroupedDataset[K, V, DS[U] <: Dataset[U, DS]] extends Ser aggUntyped(col1, col2, col3, col4).asInstanceOf[DS[(K, U1, U2, U3, U4)]] /** - * Computes the given aggregations, returning a [[Dataset]] of tuples for each unique key - * and the result of computing these aggregations over all elements in the group. + * Computes the given aggregations, returning a [[Dataset]] of tuples for each unique key and + * the result of computing these aggregations over all elements in the group. * * @since 3.0.0 */ @@ -818,8 +889,8 @@ abstract class KeyValueGroupedDataset[K, V, DS[U] <: Dataset[U, DS]] extends Ser aggUntyped(col1, col2, col3, col4, col5).asInstanceOf[DS[(K, U1, U2, U3, U4, U5)]] /** - * Computes the given aggregations, returning a [[Dataset]] of tuples for each unique key - * and the result of computing these aggregations over all elements in the group. + * Computes the given aggregations, returning a [[Dataset]] of tuples for each unique key and + * the result of computing these aggregations over all elements in the group. * * @since 3.0.0 */ @@ -834,8 +905,8 @@ abstract class KeyValueGroupedDataset[K, V, DS[U] <: Dataset[U, DS]] extends Ser .asInstanceOf[DS[(K, U1, U2, U3, U4, U5, U6)]] /** - * Computes the given aggregations, returning a [[Dataset]] of tuples for each unique key - * and the result of computing these aggregations over all elements in the group. + * Computes the given aggregations, returning a [[Dataset]] of tuples for each unique key and + * the result of computing these aggregations over all elements in the group. * * @since 3.0.0 */ @@ -851,8 +922,8 @@ abstract class KeyValueGroupedDataset[K, V, DS[U] <: Dataset[U, DS]] extends Ser .asInstanceOf[DS[(K, U1, U2, U3, U4, U5, U6, U7)]] /** - * Computes the given aggregations, returning a [[Dataset]] of tuples for each unique key - * and the result of computing these aggregations over all elements in the group. + * Computes the given aggregations, returning a [[Dataset]] of tuples for each unique key and + * the result of computing these aggregations over all elements in the group. * * @since 3.0.0 */ @@ -869,34 +940,31 @@ abstract class KeyValueGroupedDataset[K, V, DS[U] <: Dataset[U, DS]] extends Ser .asInstanceOf[DS[(K, U1, U2, U3, U4, U5, U6, U7, U8)]] /** - * Returns a [[Dataset]] that contains a tuple with each key and the number of items present - * for that key. + * Returns a [[Dataset]] that contains a tuple with each key and the number of items present for + * that key. * * @since 1.6.0 */ def count(): DS[(K, Long)] = agg(cnt(lit(1)).as(PrimitiveLongEncoder)) /** - * (Scala-specific) - * Applies the given function to each cogrouped data. For each unique group, the function will - * be passed the grouping key and 2 iterators containing all elements in the group from - * [[Dataset]] `this` and `other`. The function can return an iterator containing elements of an - * arbitrary type which will be returned as a new [[Dataset]]. + * (Scala-specific) Applies the given function to each cogrouped data. For each unique group, + * the function will be passed the grouping key and 2 iterators containing all elements in the + * group from [[Dataset]] `this` and `other`. The function can return an iterator containing + * elements of an arbitrary type which will be returned as a new [[Dataset]]. * * @since 1.6.0 */ - def cogroup[U, R: Encoder]( - other: KVDS[K, U])( + def cogroup[U, R: Encoder](other: KVDS[K, U])( f: (K, Iterator[V], Iterator[U]) => IterableOnce[R]): DS[R] = { cogroupSorted(other)(Nil: _*)(Nil: _*)(f) } /** - * (Java-specific) - * Applies the given function to each cogrouped data. For each unique group, the function will - * be passed the grouping key and 2 iterators containing all elements in the group from - * [[Dataset]] `this` and `other`. The function can return an iterator containing elements of an - * arbitrary type which will be returned as a new [[Dataset]]. + * (Java-specific) Applies the given function to each cogrouped data. For each unique group, the + * function will be passed the grouping key and 2 iterators containing all elements in the group + * from [[Dataset]] `this` and `other`. The function can return an iterator containing elements + * of an arbitrary type which will be returned as a new [[Dataset]]. * * @since 1.6.0 */ @@ -908,37 +976,36 @@ abstract class KeyValueGroupedDataset[K, V, DS[U] <: Dataset[U, DS]] extends Ser } /** - * (Scala-specific) - * Applies the given function to each sorted cogrouped data. For each unique group, the function - * will be passed the grouping key and 2 sorted iterators containing all elements in the group - * from [[Dataset]] `this` and `other`. The function can return an iterator containing elements - * of an arbitrary type which will be returned as a new [[Dataset]]. + * (Scala-specific) Applies the given function to each sorted cogrouped data. For each unique + * group, the function will be passed the grouping key and 2 sorted iterators containing all + * elements in the group from [[Dataset]] `this` and `other`. The function can return an + * iterator containing elements of an arbitrary type which will be returned as a new + * [[Dataset]]. * - * This is equivalent to [[KeyValueGroupedDataset#cogroup]], except for the iterators - * to be sorted according to the given sort expressions. That sorting does not add - * computational complexity. + * This is equivalent to [[KeyValueGroupedDataset#cogroup]], except for the iterators to be + * sorted according to the given sort expressions. That sorting does not add computational + * complexity. * - * @see [[org.apache.spark.sql.api.KeyValueGroupedDataset#cogroup]] + * @see + * [[org.apache.spark.sql.api.KeyValueGroupedDataset#cogroup]] * @since 3.4.0 */ - def cogroupSorted[U, R : Encoder]( - other: KVDS[K, U])( - thisSortExprs: Column*)( - otherSortExprs: Column*)( - f: (K, Iterator[V], Iterator[U]) => IterableOnce[R]): DS[R] + def cogroupSorted[U, R: Encoder](other: KVDS[K, U])(thisSortExprs: Column*)( + otherSortExprs: Column*)(f: (K, Iterator[V], Iterator[U]) => IterableOnce[R]): DS[R] /** - * (Java-specific) - * Applies the given function to each sorted cogrouped data. For each unique group, the function - * will be passed the grouping key and 2 sorted iterators containing all elements in the group - * from [[Dataset]] `this` and `other`. The function can return an iterator containing elements - * of an arbitrary type which will be returned as a new [[Dataset]]. + * (Java-specific) Applies the given function to each sorted cogrouped data. For each unique + * group, the function will be passed the grouping key and 2 sorted iterators containing all + * elements in the group from [[Dataset]] `this` and `other`. The function can return an + * iterator containing elements of an arbitrary type which will be returned as a new + * [[Dataset]]. * - * This is equivalent to [[KeyValueGroupedDataset#cogroup]], except for the iterators - * to be sorted according to the given sort expressions. That sorting does not add - * computational complexity. + * This is equivalent to [[KeyValueGroupedDataset#cogroup]], except for the iterators to be + * sorted according to the given sort expressions. That sorting does not add computational + * complexity. * - * @see [[org.apache.spark.sql.api.KeyValueGroupedDataset#cogroup]] + * @see + * [[org.apache.spark.sql.api.KeyValueGroupedDataset#cogroup]] * @since 3.4.0 */ def cogroupSorted[U, R]( @@ -948,8 +1015,7 @@ abstract class KeyValueGroupedDataset[K, V, DS[U] <: Dataset[U, DS]] extends Ser f: CoGroupFunction[K, V, U, R], encoder: Encoder[R]): DS[R] = { import org.apache.spark.util.ArrayImplicits._ - cogroupSorted(other)( - thisSortExprs.toImmutableArraySeq: _*)(otherSortExprs.toImmutableArraySeq: _*)( - ToScalaUDF(f))(encoder) + cogroupSorted(other)(thisSortExprs.toImmutableArraySeq: _*)( + otherSortExprs.toImmutableArraySeq: _*)(ToScalaUDF(f))(encoder) } } diff --git a/sql/api/src/main/scala/org/apache/spark/sql/api/RelationalGroupedDataset.scala b/sql/api/src/main/scala/org/apache/spark/sql/api/RelationalGroupedDataset.scala index 35d6d130ae083..7dd5f46beb316 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/api/RelationalGroupedDataset.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/api/RelationalGroupedDataset.scala @@ -30,7 +30,8 @@ import org.apache.spark.sql.{functions, Column, Encoder, Row} * The main method is the `agg` function, which has multiple variants. This class also contains * some first-order statistics such as `mean`, `sum` for convenience. * - * @note This class was named `GroupedData` in Spark 1.x. + * @note + * This class was named `GroupedData` in Spark 1.x. * @since 2.0.0 */ @Stable @@ -66,16 +67,16 @@ abstract class RelationalGroupedDataset[DS[U] <: Dataset[U, DS]] { } /** - * Returns a `KeyValueGroupedDataset` where the data is grouped by the grouping expressions - * of current `RelationalGroupedDataset`. + * Returns a `KeyValueGroupedDataset` where the data is grouped by the grouping expressions of + * current `RelationalGroupedDataset`. * * @since 3.0.0 */ def as[K: Encoder, T: Encoder]: KeyValueGroupedDataset[K, T, DS] /** - * (Scala-specific) Compute aggregates by specifying the column names and - * aggregate methods. The resulting `DataFrame` will also contain the grouping columns. + * (Scala-specific) Compute aggregates by specifying the column names and aggregate methods. The + * resulting `DataFrame` will also contain the grouping columns. * * The available aggregate methods are `avg`, `max`, `min`, `sum`, `count`. * {{{ @@ -92,8 +93,8 @@ abstract class RelationalGroupedDataset[DS[U] <: Dataset[U, DS]] { toDF((aggExpr +: aggExprs).map(toAggCol)) /** - * (Scala-specific) Compute aggregates by specifying a map from column name to - * aggregate methods. The resulting `DataFrame` will also contain the grouping columns. + * (Scala-specific) Compute aggregates by specifying a map from column name to aggregate + * methods. The resulting `DataFrame` will also contain the grouping columns. * * The available aggregate methods are `avg`, `max`, `min`, `sum`, `count`. * {{{ @@ -109,8 +110,8 @@ abstract class RelationalGroupedDataset[DS[U] <: Dataset[U, DS]] { def agg(exprs: Map[String, String]): DS[Row] = toDF(exprs.map(toAggCol).toSeq) /** - * (Java-specific) Compute aggregates by specifying a map from column name to - * aggregate methods. The resulting `DataFrame` will also contain the grouping columns. + * (Java-specific) Compute aggregates by specifying a map from column name to aggregate methods. + * The resulting `DataFrame` will also contain the grouping columns. * * The available aggregate methods are `avg`, `max`, `min`, `sum`, `count`. * {{{ @@ -160,28 +161,27 @@ abstract class RelationalGroupedDataset[DS[U] <: Dataset[U, DS]] { def agg(expr: Column, exprs: Column*): DS[Row] = toDF(expr +: exprs) /** - * Count the number of rows for each group. - * The resulting `DataFrame` will also contain the grouping columns. + * Count the number of rows for each group. The resulting `DataFrame` will also contain the + * grouping columns. * * @since 1.3.0 */ def count(): DS[Row] = toDF(functions.count(functions.lit(1)).as("count") :: Nil) /** - * Compute the average value for each numeric columns for each group. This is an alias for `avg`. - * The resulting `DataFrame` will also contain the grouping columns. - * When specified columns are given, only compute the average values for them. + * Compute the average value for each numeric columns for each group. This is an alias for + * `avg`. The resulting `DataFrame` will also contain the grouping columns. When specified + * columns are given, only compute the average values for them. * * @since 1.3.0 */ @scala.annotation.varargs def mean(colNames: String*): DS[Row] = aggregateNumericColumns(colNames, functions.avg) - /** - * Compute the max value for each numeric columns for each group. - * The resulting `DataFrame` will also contain the grouping columns. - * When specified columns are given, only compute the max values for them. + * Compute the max value for each numeric columns for each group. The resulting `DataFrame` will + * also contain the grouping columns. When specified columns are given, only compute the max + * values for them. * * @since 1.3.0 */ @@ -189,9 +189,9 @@ abstract class RelationalGroupedDataset[DS[U] <: Dataset[U, DS]] { def max(colNames: String*): DS[Row] = aggregateNumericColumns(colNames, functions.max) /** - * Compute the mean value for each numeric columns for each group. - * The resulting `DataFrame` will also contain the grouping columns. - * When specified columns are given, only compute the mean values for them. + * Compute the mean value for each numeric columns for each group. The resulting `DataFrame` + * will also contain the grouping columns. When specified columns are given, only compute the + * mean values for them. * * @since 1.3.0 */ @@ -199,9 +199,9 @@ abstract class RelationalGroupedDataset[DS[U] <: Dataset[U, DS]] { def avg(colNames: String*): DS[Row] = aggregateNumericColumns(colNames, functions.avg) /** - * Compute the min value for each numeric column for each group. - * The resulting `DataFrame` will also contain the grouping columns. - * When specified columns are given, only compute the min values for them. + * Compute the min value for each numeric column for each group. The resulting `DataFrame` will + * also contain the grouping columns. When specified columns are given, only compute the min + * values for them. * * @since 1.3.0 */ @@ -209,9 +209,9 @@ abstract class RelationalGroupedDataset[DS[U] <: Dataset[U, DS]] { def min(colNames: String*): DS[Row] = aggregateNumericColumns(colNames, functions.min) /** - * Compute the sum for each numeric columns for each group. - * The resulting `DataFrame` will also contain the grouping columns. - * When specified columns are given, only compute the sum for them. + * Compute the sum for each numeric columns for each group. The resulting `DataFrame` will also + * contain the grouping columns. When specified columns are given, only compute the sum for + * them. * * @since 1.3.0 */ @@ -221,27 +221,29 @@ abstract class RelationalGroupedDataset[DS[U] <: Dataset[U, DS]] { /** * Pivots a column of the current `DataFrame` and performs the specified aggregation. * - * Spark will eagerly compute the distinct values in `pivotColumn` so it can determine - * the resulting schema of the transformation. To avoid any eager computations, provide an - * explicit list of values via `pivot(pivotColumn: String, values: Seq[Any])`. + * Spark will eagerly compute the distinct values in `pivotColumn` so it can determine the + * resulting schema of the transformation. To avoid any eager computations, provide an explicit + * list of values via `pivot(pivotColumn: String, values: Seq[Any])`. * * {{{ * // Compute the sum of earnings for each year by course with each course as a separate column * df.groupBy("year").pivot("course").sum("earnings") * }}} * - * @see `org.apache.spark.sql.Dataset.unpivot` for the reverse operation, - * except for the aggregation. - * @param pivotColumn Name of the column to pivot. + * @see + * `org.apache.spark.sql.Dataset.unpivot` for the reverse operation, except for the + * aggregation. + * @param pivotColumn + * Name of the column to pivot. * @since 1.6.0 */ def pivot(pivotColumn: String): RGD = pivot(df.col(pivotColumn)) /** - * Pivots a column of the current `DataFrame` and performs the specified aggregation. - * There are two versions of pivot function: one that requires the caller to specify the list - * of distinct values to pivot on, and one that does not. The latter is more concise but less - * efficient, because Spark needs to first compute the list of distinct values internally. + * Pivots a column of the current `DataFrame` and performs the specified aggregation. There are + * two versions of pivot function: one that requires the caller to specify the list of distinct + * values to pivot on, and one that does not. The latter is more concise but less efficient, + * because Spark needs to first compute the list of distinct values internally. * * {{{ * // Compute the sum of earnings for each year by course with each course as a separate column @@ -260,10 +262,13 @@ abstract class RelationalGroupedDataset[DS[U] <: Dataset[U, DS]] { * .agg(sum($"earnings")) * }}} * - * @see `org.apache.spark.sql.Dataset.unpivot` for the reverse operation, - * except for the aggregation. - * @param pivotColumn Name of the column to pivot. - * @param values List of values that will be translated to columns in the output DataFrame. + * @see + * `org.apache.spark.sql.Dataset.unpivot` for the reverse operation, except for the + * aggregation. + * @param pivotColumn + * Name of the column to pivot. + * @param values + * List of values that will be translated to columns in the output DataFrame. * @since 1.6.0 */ def pivot(pivotColumn: String, values: Seq[Any]): RGD = @@ -273,8 +278,8 @@ abstract class RelationalGroupedDataset[DS[U] <: Dataset[U, DS]] { * (Java-specific) Pivots a column of the current `DataFrame` and performs the specified * aggregation. * - * There are two versions of pivot function: one that requires the caller to specify the list - * of distinct values to pivot on, and one that does not. The latter is more concise but less + * There are two versions of pivot function: one that requires the caller to specify the list of + * distinct values to pivot on, and one that does not. The latter is more concise but less * efficient, because Spark needs to first compute the list of distinct values internally. * * {{{ @@ -285,10 +290,13 @@ abstract class RelationalGroupedDataset[DS[U] <: Dataset[U, DS]] { * df.groupBy("year").pivot("course").sum("earnings"); * }}} * - * @see `org.apache.spark.sql.Dataset.unpivot` for the reverse operation, - * except for the aggregation. - * @param pivotColumn Name of the column to pivot. - * @param values List of values that will be translated to columns in the output DataFrame. + * @see + * `org.apache.spark.sql.Dataset.unpivot` for the reverse operation, except for the + * aggregation. + * @param pivotColumn + * Name of the column to pivot. + * @param values + * List of values that will be translated to columns in the output DataFrame. * @since 1.6.0 */ def pivot(pivotColumn: String, values: util.List[Any]): RGD = @@ -296,13 +304,16 @@ abstract class RelationalGroupedDataset[DS[U] <: Dataset[U, DS]] { /** * (Java-specific) Pivots a column of the current `DataFrame` and performs the specified - * aggregation. This is an overloaded version of the `pivot` method with `pivotColumn` of - * the `String` type. - * - * @see `org.apache.spark.sql.Dataset.unpivot` for the reverse operation, - * except for the aggregation. - * @param pivotColumn the column to pivot. - * @param values List of values that will be translated to columns in the output DataFrame. + * aggregation. This is an overloaded version of the `pivot` method with `pivotColumn` of the + * `String` type. + * + * @see + * `org.apache.spark.sql.Dataset.unpivot` for the reverse operation, except for the + * aggregation. + * @param pivotColumn + * the column to pivot. + * @param values + * List of values that will be translated to columns in the output DataFrame. * @since 2.4.0 */ def pivot(pivotColumn: Column, values: util.List[Any]): RGD = @@ -311,35 +322,40 @@ abstract class RelationalGroupedDataset[DS[U] <: Dataset[U, DS]] { /** * Pivots a column of the current `DataFrame` and performs the specified aggregation. * - * Spark will eagerly compute the distinct values in `pivotColumn` so it can determine - * the resulting schema of the transformation. To avoid any eager computations, provide an - * explicit list of values via `pivot(pivotColumn: Column, values: Seq[Any])`. + * Spark will eagerly compute the distinct values in `pivotColumn` so it can determine the + * resulting schema of the transformation. To avoid any eager computations, provide an explicit + * list of values via `pivot(pivotColumn: Column, values: Seq[Any])`. * * {{{ * // Compute the sum of earnings for each year by course with each course as a separate column * df.groupBy($"year").pivot($"course").sum($"earnings"); * }}} * - * @see `org.apache.spark.sql.Dataset.unpivot` for the reverse operation, - * except for the aggregation. - * @param pivotColumn he column to pivot. + * @see + * `org.apache.spark.sql.Dataset.unpivot` for the reverse operation, except for the + * aggregation. + * @param pivotColumn + * he column to pivot. * @since 2.4.0 */ def pivot(pivotColumn: Column): RGD /** - * Pivots a column of the current `DataFrame` and performs the specified aggregation. - * This is an overloaded version of the `pivot` method with `pivotColumn` of the `String` type. + * Pivots a column of the current `DataFrame` and performs the specified aggregation. This is an + * overloaded version of the `pivot` method with `pivotColumn` of the `String` type. * * {{{ * // Compute the sum of earnings for each year by course with each course as a separate column * df.groupBy($"year").pivot($"course", Seq("dotNET", "Java")).sum($"earnings") * }}} * - * @see `org.apache.spark.sql.Dataset.unpivot` for the reverse operation, - * except for the aggregation. - * @param pivotColumn the column to pivot. - * @param values List of values that will be translated to columns in the output DataFrame. + * @see + * `org.apache.spark.sql.Dataset.unpivot` for the reverse operation, except for the + * aggregation. + * @param pivotColumn + * the column to pivot. + * @param values + * List of values that will be translated to columns in the output DataFrame. * @since 2.4.0 */ def pivot(pivotColumn: Column, values: Seq[Any]): RGD diff --git a/sql/api/src/main/scala/org/apache/spark/sql/api/SparkSession.scala b/sql/api/src/main/scala/org/apache/spark/sql/api/SparkSession.scala index 633a54e32bc86..ef7c33d95f614 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/api/SparkSession.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/api/SparkSession.scala @@ -32,8 +32,8 @@ import org.apache.spark.sql.types.StructType /** * The entry point to programming Spark with the Dataset and DataFrame API. * - * In environments that this has been created upfront (e.g. REPL, notebooks), use the builder - * to get an existing session: + * In environments that this has been created upfront (e.g. REPL, notebooks), use the builder to + * get an existing session: * * {{{ * SparkSession.builder().getOrCreate() @@ -50,6 +50,7 @@ import org.apache.spark.sql.types.StructType * }}} */ abstract class SparkSession[DS[U] <: Dataset[U, DS]] extends Serializable with Closeable { + /** * The version of Spark on which this application is running. * @@ -72,21 +73,23 @@ abstract class SparkSession[DS[U] <: Dataset[U, DS]] extends Serializable with C * DataTypes.StringType); * }}} * - * @note The user-defined functions must be deterministic. Due to optimization, - * duplicate invocations may be eliminated or the function may even be invoked more times - * than it is present in the query. + * @note + * The user-defined functions must be deterministic. Due to optimization, duplicate + * invocations may be eliminated or the function may even be invoked more times than it is + * present in the query. * @since 2.0.0 */ def udf: UDFRegistration /** - * Start a new session with isolated SQL configurations, temporary tables, registered - * functions are isolated, but sharing the underlying `SparkContext` and cached data. - * - * @note Other than the `SparkContext`, all shared state is initialized lazily. - * This method will force the initialization of the shared state to ensure that parent - * and child sessions are set up with the same shared state. If the underlying catalog - * implementation is Hive, this will initialize the metastore, which may take some time. + * Start a new session with isolated SQL configurations, temporary tables, registered functions + * are isolated, but sharing the underlying `SparkContext` and cached data. + * + * @note + * Other than the `SparkContext`, all shared state is initialized lazily. This method will + * force the initialization of the shared state to ensure that parent and child sessions are + * set up with the same shared state. If the underlying catalog implementation is Hive, this + * will initialize the metastore, which may take some time. * @since 2.0.0 */ def newSession(): SparkSession[DS] @@ -108,14 +111,13 @@ abstract class SparkSession[DS[U] <: Dataset[U, DS]] extends Serializable with C * * @since 2.0.0 */ - def createDataFrame[A <: Product : TypeTag](data: Seq[A]): DS[Row] + def createDataFrame[A <: Product: TypeTag](data: Seq[A]): DS[Row] /** - * :: DeveloperApi :: - * Creates a `DataFrame` from a `java.util.List` containing [[org.apache.spark.sql.Row]]s using - * the given schema.It is important to make sure that the structure of every - * [[org.apache.spark.sql.Row]] of the provided List matches the provided schema. Otherwise, - * there will be runtime exception. + * :: DeveloperApi :: Creates a `DataFrame` from a `java.util.List` containing + * [[org.apache.spark.sql.Row]]s using the given schema.It is important to make sure that the + * structure of every [[org.apache.spark.sql.Row]] of the provided List matches the provided + * schema. Otherwise, there will be runtime exception. * * @since 2.0.0 */ @@ -125,8 +127,8 @@ abstract class SparkSession[DS[U] <: Dataset[U, DS]] extends Serializable with C /** * Applies a schema to a List of Java Beans. * - * WARNING: Since there is no guaranteed ordering for fields in a Java Bean, - * SELECT * queries will return the columns in an undefined order. + * WARNING: Since there is no guaranteed ordering for fields in a Java Bean, SELECT * queries + * will return the columns in an undefined order. * * @since 1.6.0 */ @@ -145,11 +147,11 @@ abstract class SparkSession[DS[U] <: Dataset[U, DS]] extends Serializable with C /** * Creates a [[Dataset]] from a local Seq of data of a given type. This method requires an - * encoder (to convert a JVM object of type `T` to and from the internal Spark SQL representation) - * that is generally created automatically through implicits from a `SparkSession`, or can be - * created explicitly by calling static methods on `Encoders`. + * encoder (to convert a JVM object of type `T` to and from the internal Spark SQL + * representation) that is generally created automatically through implicits from a + * `SparkSession`, or can be created explicitly by calling static methods on `Encoders`. * - * == Example == + * ==Example== * * {{{ * @@ -174,11 +176,11 @@ abstract class SparkSession[DS[U] <: Dataset[U, DS]] extends Serializable with C /** * Creates a [[Dataset]] from a `java.util.List` of a given type. This method requires an - * encoder (to convert a JVM object of type `T` to and from the internal Spark SQL representation) - * that is generally created automatically through implicits from a `SparkSession`, or can be - * created explicitly by calling static methods on `Encoders`. + * encoder (to convert a JVM object of type `T` to and from the internal Spark SQL + * representation) that is generally created automatically through implicits from a + * `SparkSession`, or can be created explicitly by calling static methods on `Encoders`. * - * == Java Example == + * ==Java Example== * * {{{ * List data = Arrays.asList("hello", "world"); @@ -190,33 +192,32 @@ abstract class SparkSession[DS[U] <: Dataset[U, DS]] extends Serializable with C def createDataset[T: Encoder](data: util.List[T]): DS[T] /** - * Creates a [[Dataset]] with a single `LongType` column named `id`, containing elements - * in a range from 0 to `end` (exclusive) with step value 1. + * Creates a [[Dataset]] with a single `LongType` column named `id`, containing elements in a + * range from 0 to `end` (exclusive) with step value 1. * * @since 2.0.0 */ def range(end: Long): DS[lang.Long] /** - * Creates a [[Dataset]] with a single `LongType` column named `id`, containing elements - * in a range from `start` to `end` (exclusive) with step value 1. + * Creates a [[Dataset]] with a single `LongType` column named `id`, containing elements in a + * range from `start` to `end` (exclusive) with step value 1. * * @since 2.0.0 */ def range(start: Long, end: Long): DS[lang.Long] /** - * Creates a [[Dataset]] with a single `LongType` column named `id`, containing elements - * in a range from `start` to `end` (exclusive) with a step value. + * Creates a [[Dataset]] with a single `LongType` column named `id`, containing elements in a + * range from `start` to `end` (exclusive) with a step value. * * @since 2.0.0 */ def range(start: Long, end: Long, step: Long): DS[lang.Long] /** - * Creates a [[Dataset]] with a single `LongType` column named `id`, containing elements - * in a range from `start` to `end` (exclusive) with a step value, with partition number - * specified. + * Creates a [[Dataset]] with a single `LongType` column named `id`, containing elements in a + * range from `start` to `end` (exclusive) with a step value, with partition number specified. * * @since 2.0.0 */ @@ -228,15 +229,15 @@ abstract class SparkSession[DS[U] <: Dataset[U, DS]] extends Serializable with C /** * Returns the specified table/view as a `DataFrame`. If it's a table, it must support batch - * reading and the returned DataFrame is the batch scan query plan of this table. If it's a view, - * the returned DataFrame is simply the query plan of the view, which can either be a batch or - * streaming query plan. - * - * @param tableName is either a qualified or unqualified name that designates a table or view. - * If a database is specified, it identifies the table/view from the database. - * Otherwise, it first attempts to find a temporary view with the given name - * and then match the table/view from the current database. - * Note that, the global temporary view database is also valid here. + * reading and the returned DataFrame is the batch scan query plan of this table. If it's a + * view, the returned DataFrame is simply the query plan of the view, which can either be a + * batch or streaming query plan. + * + * @param tableName + * is either a qualified or unqualified name that designates a table or view. If a database is + * specified, it identifies the table/view from the database. Otherwise, it first attempts to + * find a temporary view with the given name and then match the table/view from the current + * database. Note that, the global temporary view database is also valid here. * @since 2.0.0 */ def table(tableName: String): DS[Row] @@ -246,58 +247,54 @@ abstract class SparkSession[DS[U] <: Dataset[U, DS]] extends Serializable with C * ----------------- */ /** - * Executes a SQL query substituting positional parameters by the given arguments, - * returning the result as a `DataFrame`. - * This API eagerly runs DDL/DML commands, but not for SELECT queries. - * - * @param sqlText A SQL statement with positional parameters to execute. - * @param args An array of Java/Scala objects that can be converted to - * SQL literal expressions. See - * - * Supported Data Types for supported value types in Scala/Java. - * For example, 1, "Steven", LocalDate.of(2023, 4, 2). - * A value can be also a `Column` of a literal or collection constructor functions - * such as `map()`, `array()`, `struct()`, in that case it is taken as is. + * Executes a SQL query substituting positional parameters by the given arguments, returning the + * result as a `DataFrame`. This API eagerly runs DDL/DML commands, but not for SELECT queries. + * + * @param sqlText + * A SQL statement with positional parameters to execute. + * @param args + * An array of Java/Scala objects that can be converted to SQL literal expressions. See Supported Data + * Types for supported value types in Scala/Java. For example, 1, "Steven", + * LocalDate.of(2023, 4, 2). A value can be also a `Column` of a literal or collection + * constructor functions such as `map()`, `array()`, `struct()`, in that case it is taken as + * is. * @since 3.5.0 */ @Experimental def sql(sqlText: String, args: Array[_]): DS[Row] /** - * Executes a SQL query substituting named parameters by the given arguments, - * returning the result as a `DataFrame`. - * This API eagerly runs DDL/DML commands, but not for SELECT queries. - * - * @param sqlText A SQL statement with named parameters to execute. - * @param args A map of parameter names to Java/Scala objects that can be converted to - * SQL literal expressions. See - * - * Supported Data Types for supported value types in Scala/Java. - * For example, map keys: "rank", "name", "birthdate"; - * map values: 1, "Steven", LocalDate.of(2023, 4, 2). - * Map value can be also a `Column` of a literal or collection constructor - * functions such as `map()`, `array()`, `struct()`, in that case it is taken - * as is. + * Executes a SQL query substituting named parameters by the given arguments, returning the + * result as a `DataFrame`. This API eagerly runs DDL/DML commands, but not for SELECT queries. + * + * @param sqlText + * A SQL statement with named parameters to execute. + * @param args + * A map of parameter names to Java/Scala objects that can be converted to SQL literal + * expressions. See + * Supported Data Types for supported value types in Scala/Java. For example, map keys: + * "rank", "name", "birthdate"; map values: 1, "Steven", LocalDate.of(2023, 4, 2). Map value + * can be also a `Column` of a literal or collection constructor functions such as `map()`, + * `array()`, `struct()`, in that case it is taken as is. * @since 3.4.0 */ @Experimental def sql(sqlText: String, args: Map[String, Any]): DS[Row] /** - * Executes a SQL query substituting named parameters by the given arguments, - * returning the result as a `DataFrame`. - * This API eagerly runs DDL/DML commands, but not for SELECT queries. - * - * @param sqlText A SQL statement with named parameters to execute. - * @param args A map of parameter names to Java/Scala objects that can be converted to - * SQL literal expressions. See - * - * Supported Data Types for supported value types in Scala/Java. - * For example, map keys: "rank", "name", "birthdate"; - * map values: 1, "Steven", LocalDate.of(2023, 4, 2). - * Map value can be also a `Column` of a literal or collection constructor - * functions such as `map()`, `array()`, `struct()`, in that case it is taken - * as is. + * Executes a SQL query substituting named parameters by the given arguments, returning the + * result as a `DataFrame`. This API eagerly runs DDL/DML commands, but not for SELECT queries. + * + * @param sqlText + * A SQL statement with named parameters to execute. + * @param args + * A map of parameter names to Java/Scala objects that can be converted to SQL literal + * expressions. See + * Supported Data Types for supported value types in Scala/Java. For example, map keys: + * "rank", "name", "birthdate"; map values: 1, "Steven", LocalDate.of(2023, 4, 2). Map value + * can be also a `Column` of a literal or collection constructor functions such as `map()`, + * `array()`, `struct()`, in that case it is taken as is. * @since 3.4.0 */ @Experimental @@ -306,8 +303,8 @@ abstract class SparkSession[DS[U] <: Dataset[U, DS]] extends Serializable with C } /** - * Executes a SQL query using Spark, returning the result as a `DataFrame`. - * This API eagerly runs DDL/DML commands, but not for SELECT queries. + * Executes a SQL query using Spark, returning the result as a `DataFrame`. This API eagerly + * runs DDL/DML commands, but not for SELECT queries. * * @since 2.0.0 */ diff --git a/sql/api/src/main/scala/org/apache/spark/sql/api/UDFRegistration.scala b/sql/api/src/main/scala/org/apache/spark/sql/api/UDFRegistration.scala index 4611393f0f7ec..c11e266827ff9 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/api/UDFRegistration.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/api/UDFRegistration.scala @@ -50,9 +50,12 @@ abstract class UDFRegistration { * spark.udf.register("stringLit", bar.asNonNullable()) * }}} * - * @param name the name of the UDF. - * @param udf the UDF needs to be registered. - * @return the registered UDF. + * @param name + * the name of the UDF. + * @param udf + * the UDF needs to be registered. + * @return + * the registered UDF. * * @since 2.2.0 */ @@ -117,11 +120,12 @@ abstract class UDFRegistration { | registerJavaUDF(name, ToScalaUDF(f), returnType, $i) |}""".stripMargin) } - */ + */ /** * Registers a deterministic Scala closure of 0 arguments as user-defined function (UDF). - * @tparam RT return type of UDF. + * @tparam RT + * return type of UDF. * @since 1.3.0 */ def register[RT: TypeTag](name: String, func: Function0[RT]): UserDefinedFunction = { @@ -130,200 +134,964 @@ abstract class UDFRegistration { /** * Registers a deterministic Scala closure of 1 arguments as user-defined function (UDF). - * @tparam RT return type of UDF. + * @tparam RT + * return type of UDF. * @since 1.3.0 */ - def register[RT: TypeTag, A1: TypeTag](name: String, func: Function1[A1, RT]): UserDefinedFunction = { + def register[RT: TypeTag, A1: TypeTag]( + name: String, + func: Function1[A1, RT]): UserDefinedFunction = { registerScalaUDF(name, func, implicitly[TypeTag[RT]], implicitly[TypeTag[A1]]) } /** * Registers a deterministic Scala closure of 2 arguments as user-defined function (UDF). - * @tparam RT return type of UDF. + * @tparam RT + * return type of UDF. * @since 1.3.0 */ - def register[RT: TypeTag, A1: TypeTag, A2: TypeTag](name: String, func: Function2[A1, A2, RT]): UserDefinedFunction = { - registerScalaUDF(name, func, implicitly[TypeTag[RT]], implicitly[TypeTag[A1]], implicitly[TypeTag[A2]]) + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag]( + name: String, + func: Function2[A1, A2, RT]): UserDefinedFunction = { + registerScalaUDF( + name, + func, + implicitly[TypeTag[RT]], + implicitly[TypeTag[A1]], + implicitly[TypeTag[A2]]) } /** * Registers a deterministic Scala closure of 3 arguments as user-defined function (UDF). - * @tparam RT return type of UDF. + * @tparam RT + * return type of UDF. * @since 1.3.0 */ - def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag](name: String, func: Function3[A1, A2, A3, RT]): UserDefinedFunction = { - registerScalaUDF(name, func, implicitly[TypeTag[RT]], implicitly[TypeTag[A1]], implicitly[TypeTag[A2]], implicitly[TypeTag[A3]]) + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag]( + name: String, + func: Function3[A1, A2, A3, RT]): UserDefinedFunction = { + registerScalaUDF( + name, + func, + implicitly[TypeTag[RT]], + implicitly[TypeTag[A1]], + implicitly[TypeTag[A2]], + implicitly[TypeTag[A3]]) } /** * Registers a deterministic Scala closure of 4 arguments as user-defined function (UDF). - * @tparam RT return type of UDF. + * @tparam RT + * return type of UDF. * @since 1.3.0 */ - def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag](name: String, func: Function4[A1, A2, A3, A4, RT]): UserDefinedFunction = { - registerScalaUDF(name, func, implicitly[TypeTag[RT]], implicitly[TypeTag[A1]], implicitly[TypeTag[A2]], implicitly[TypeTag[A3]], implicitly[TypeTag[A4]]) + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag]( + name: String, + func: Function4[A1, A2, A3, A4, RT]): UserDefinedFunction = { + registerScalaUDF( + name, + func, + implicitly[TypeTag[RT]], + implicitly[TypeTag[A1]], + implicitly[TypeTag[A2]], + implicitly[TypeTag[A3]], + implicitly[TypeTag[A4]]) } /** * Registers a deterministic Scala closure of 5 arguments as user-defined function (UDF). - * @tparam RT return type of UDF. + * @tparam RT + * return type of UDF. * @since 1.3.0 */ - def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag](name: String, func: Function5[A1, A2, A3, A4, A5, RT]): UserDefinedFunction = { - registerScalaUDF(name, func, implicitly[TypeTag[RT]], implicitly[TypeTag[A1]], implicitly[TypeTag[A2]], implicitly[TypeTag[A3]], implicitly[TypeTag[A4]], implicitly[TypeTag[A5]]) + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag]( + name: String, + func: Function5[A1, A2, A3, A4, A5, RT]): UserDefinedFunction = { + registerScalaUDF( + name, + func, + implicitly[TypeTag[RT]], + implicitly[TypeTag[A1]], + implicitly[TypeTag[A2]], + implicitly[TypeTag[A3]], + implicitly[TypeTag[A4]], + implicitly[TypeTag[A5]]) } /** * Registers a deterministic Scala closure of 6 arguments as user-defined function (UDF). - * @tparam RT return type of UDF. + * @tparam RT + * return type of UDF. * @since 1.3.0 */ - def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag](name: String, func: Function6[A1, A2, A3, A4, A5, A6, RT]): UserDefinedFunction = { - registerScalaUDF(name, func, implicitly[TypeTag[RT]], implicitly[TypeTag[A1]], implicitly[TypeTag[A2]], implicitly[TypeTag[A3]], implicitly[TypeTag[A4]], implicitly[TypeTag[A5]], implicitly[TypeTag[A6]]) + def register[ + RT: TypeTag, + A1: TypeTag, + A2: TypeTag, + A3: TypeTag, + A4: TypeTag, + A5: TypeTag, + A6: TypeTag]( + name: String, + func: Function6[A1, A2, A3, A4, A5, A6, RT]): UserDefinedFunction = { + registerScalaUDF( + name, + func, + implicitly[TypeTag[RT]], + implicitly[TypeTag[A1]], + implicitly[TypeTag[A2]], + implicitly[TypeTag[A3]], + implicitly[TypeTag[A4]], + implicitly[TypeTag[A5]], + implicitly[TypeTag[A6]]) } /** * Registers a deterministic Scala closure of 7 arguments as user-defined function (UDF). - * @tparam RT return type of UDF. + * @tparam RT + * return type of UDF. * @since 1.3.0 */ - def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag](name: String, func: Function7[A1, A2, A3, A4, A5, A6, A7, RT]): UserDefinedFunction = { - registerScalaUDF(name, func, implicitly[TypeTag[RT]], implicitly[TypeTag[A1]], implicitly[TypeTag[A2]], implicitly[TypeTag[A3]], implicitly[TypeTag[A4]], implicitly[TypeTag[A5]], implicitly[TypeTag[A6]], implicitly[TypeTag[A7]]) + def register[ + RT: TypeTag, + A1: TypeTag, + A2: TypeTag, + A3: TypeTag, + A4: TypeTag, + A5: TypeTag, + A6: TypeTag, + A7: TypeTag]( + name: String, + func: Function7[A1, A2, A3, A4, A5, A6, A7, RT]): UserDefinedFunction = { + registerScalaUDF( + name, + func, + implicitly[TypeTag[RT]], + implicitly[TypeTag[A1]], + implicitly[TypeTag[A2]], + implicitly[TypeTag[A3]], + implicitly[TypeTag[A4]], + implicitly[TypeTag[A5]], + implicitly[TypeTag[A6]], + implicitly[TypeTag[A7]]) } /** * Registers a deterministic Scala closure of 8 arguments as user-defined function (UDF). - * @tparam RT return type of UDF. + * @tparam RT + * return type of UDF. * @since 1.3.0 */ - def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag](name: String, func: Function8[A1, A2, A3, A4, A5, A6, A7, A8, RT]): UserDefinedFunction = { - registerScalaUDF(name, func, implicitly[TypeTag[RT]], implicitly[TypeTag[A1]], implicitly[TypeTag[A2]], implicitly[TypeTag[A3]], implicitly[TypeTag[A4]], implicitly[TypeTag[A5]], implicitly[TypeTag[A6]], implicitly[TypeTag[A7]], implicitly[TypeTag[A8]]) + def register[ + RT: TypeTag, + A1: TypeTag, + A2: TypeTag, + A3: TypeTag, + A4: TypeTag, + A5: TypeTag, + A6: TypeTag, + A7: TypeTag, + A8: TypeTag]( + name: String, + func: Function8[A1, A2, A3, A4, A5, A6, A7, A8, RT]): UserDefinedFunction = { + registerScalaUDF( + name, + func, + implicitly[TypeTag[RT]], + implicitly[TypeTag[A1]], + implicitly[TypeTag[A2]], + implicitly[TypeTag[A3]], + implicitly[TypeTag[A4]], + implicitly[TypeTag[A5]], + implicitly[TypeTag[A6]], + implicitly[TypeTag[A7]], + implicitly[TypeTag[A8]]) } /** * Registers a deterministic Scala closure of 9 arguments as user-defined function (UDF). - * @tparam RT return type of UDF. + * @tparam RT + * return type of UDF. * @since 1.3.0 */ - def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag](name: String, func: Function9[A1, A2, A3, A4, A5, A6, A7, A8, A9, RT]): UserDefinedFunction = { - registerScalaUDF(name, func, implicitly[TypeTag[RT]], implicitly[TypeTag[A1]], implicitly[TypeTag[A2]], implicitly[TypeTag[A3]], implicitly[TypeTag[A4]], implicitly[TypeTag[A5]], implicitly[TypeTag[A6]], implicitly[TypeTag[A7]], implicitly[TypeTag[A8]], implicitly[TypeTag[A9]]) + def register[ + RT: TypeTag, + A1: TypeTag, + A2: TypeTag, + A3: TypeTag, + A4: TypeTag, + A5: TypeTag, + A6: TypeTag, + A7: TypeTag, + A8: TypeTag, + A9: TypeTag]( + name: String, + func: Function9[A1, A2, A3, A4, A5, A6, A7, A8, A9, RT]): UserDefinedFunction = { + registerScalaUDF( + name, + func, + implicitly[TypeTag[RT]], + implicitly[TypeTag[A1]], + implicitly[TypeTag[A2]], + implicitly[TypeTag[A3]], + implicitly[TypeTag[A4]], + implicitly[TypeTag[A5]], + implicitly[TypeTag[A6]], + implicitly[TypeTag[A7]], + implicitly[TypeTag[A8]], + implicitly[TypeTag[A9]]) } /** * Registers a deterministic Scala closure of 10 arguments as user-defined function (UDF). - * @tparam RT return type of UDF. + * @tparam RT + * return type of UDF. * @since 1.3.0 */ - def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag](name: String, func: Function10[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, RT]): UserDefinedFunction = { - registerScalaUDF(name, func, implicitly[TypeTag[RT]], implicitly[TypeTag[A1]], implicitly[TypeTag[A2]], implicitly[TypeTag[A3]], implicitly[TypeTag[A4]], implicitly[TypeTag[A5]], implicitly[TypeTag[A6]], implicitly[TypeTag[A7]], implicitly[TypeTag[A8]], implicitly[TypeTag[A9]], implicitly[TypeTag[A10]]) + def register[ + RT: TypeTag, + A1: TypeTag, + A2: TypeTag, + A3: TypeTag, + A4: TypeTag, + A5: TypeTag, + A6: TypeTag, + A7: TypeTag, + A8: TypeTag, + A9: TypeTag, + A10: TypeTag]( + name: String, + func: Function10[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, RT]): UserDefinedFunction = { + registerScalaUDF( + name, + func, + implicitly[TypeTag[RT]], + implicitly[TypeTag[A1]], + implicitly[TypeTag[A2]], + implicitly[TypeTag[A3]], + implicitly[TypeTag[A4]], + implicitly[TypeTag[A5]], + implicitly[TypeTag[A6]], + implicitly[TypeTag[A7]], + implicitly[TypeTag[A8]], + implicitly[TypeTag[A9]], + implicitly[TypeTag[A10]]) } /** * Registers a deterministic Scala closure of 11 arguments as user-defined function (UDF). - * @tparam RT return type of UDF. + * @tparam RT + * return type of UDF. * @since 1.3.0 */ - def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag](name: String, func: Function11[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, RT]): UserDefinedFunction = { - registerScalaUDF(name, func, implicitly[TypeTag[RT]], implicitly[TypeTag[A1]], implicitly[TypeTag[A2]], implicitly[TypeTag[A3]], implicitly[TypeTag[A4]], implicitly[TypeTag[A5]], implicitly[TypeTag[A6]], implicitly[TypeTag[A7]], implicitly[TypeTag[A8]], implicitly[TypeTag[A9]], implicitly[TypeTag[A10]], implicitly[TypeTag[A11]]) + def register[ + RT: TypeTag, + A1: TypeTag, + A2: TypeTag, + A3: TypeTag, + A4: TypeTag, + A5: TypeTag, + A6: TypeTag, + A7: TypeTag, + A8: TypeTag, + A9: TypeTag, + A10: TypeTag, + A11: TypeTag]( + name: String, + func: Function11[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, RT]): UserDefinedFunction = { + registerScalaUDF( + name, + func, + implicitly[TypeTag[RT]], + implicitly[TypeTag[A1]], + implicitly[TypeTag[A2]], + implicitly[TypeTag[A3]], + implicitly[TypeTag[A4]], + implicitly[TypeTag[A5]], + implicitly[TypeTag[A6]], + implicitly[TypeTag[A7]], + implicitly[TypeTag[A8]], + implicitly[TypeTag[A9]], + implicitly[TypeTag[A10]], + implicitly[TypeTag[A11]]) } /** * Registers a deterministic Scala closure of 12 arguments as user-defined function (UDF). - * @tparam RT return type of UDF. + * @tparam RT + * return type of UDF. * @since 1.3.0 */ - def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag](name: String, func: Function12[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, RT]): UserDefinedFunction = { - registerScalaUDF(name, func, implicitly[TypeTag[RT]], implicitly[TypeTag[A1]], implicitly[TypeTag[A2]], implicitly[TypeTag[A3]], implicitly[TypeTag[A4]], implicitly[TypeTag[A5]], implicitly[TypeTag[A6]], implicitly[TypeTag[A7]], implicitly[TypeTag[A8]], implicitly[TypeTag[A9]], implicitly[TypeTag[A10]], implicitly[TypeTag[A11]], implicitly[TypeTag[A12]]) + def register[ + RT: TypeTag, + A1: TypeTag, + A2: TypeTag, + A3: TypeTag, + A4: TypeTag, + A5: TypeTag, + A6: TypeTag, + A7: TypeTag, + A8: TypeTag, + A9: TypeTag, + A10: TypeTag, + A11: TypeTag, + A12: TypeTag]( + name: String, + func: Function12[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, RT]) + : UserDefinedFunction = { + registerScalaUDF( + name, + func, + implicitly[TypeTag[RT]], + implicitly[TypeTag[A1]], + implicitly[TypeTag[A2]], + implicitly[TypeTag[A3]], + implicitly[TypeTag[A4]], + implicitly[TypeTag[A5]], + implicitly[TypeTag[A6]], + implicitly[TypeTag[A7]], + implicitly[TypeTag[A8]], + implicitly[TypeTag[A9]], + implicitly[TypeTag[A10]], + implicitly[TypeTag[A11]], + implicitly[TypeTag[A12]]) } /** * Registers a deterministic Scala closure of 13 arguments as user-defined function (UDF). - * @tparam RT return type of UDF. + * @tparam RT + * return type of UDF. * @since 1.3.0 */ - def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag](name: String, func: Function13[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, RT]): UserDefinedFunction = { - registerScalaUDF(name, func, implicitly[TypeTag[RT]], implicitly[TypeTag[A1]], implicitly[TypeTag[A2]], implicitly[TypeTag[A3]], implicitly[TypeTag[A4]], implicitly[TypeTag[A5]], implicitly[TypeTag[A6]], implicitly[TypeTag[A7]], implicitly[TypeTag[A8]], implicitly[TypeTag[A9]], implicitly[TypeTag[A10]], implicitly[TypeTag[A11]], implicitly[TypeTag[A12]], implicitly[TypeTag[A13]]) + def register[ + RT: TypeTag, + A1: TypeTag, + A2: TypeTag, + A3: TypeTag, + A4: TypeTag, + A5: TypeTag, + A6: TypeTag, + A7: TypeTag, + A8: TypeTag, + A9: TypeTag, + A10: TypeTag, + A11: TypeTag, + A12: TypeTag, + A13: TypeTag]( + name: String, + func: Function13[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, RT]) + : UserDefinedFunction = { + registerScalaUDF( + name, + func, + implicitly[TypeTag[RT]], + implicitly[TypeTag[A1]], + implicitly[TypeTag[A2]], + implicitly[TypeTag[A3]], + implicitly[TypeTag[A4]], + implicitly[TypeTag[A5]], + implicitly[TypeTag[A6]], + implicitly[TypeTag[A7]], + implicitly[TypeTag[A8]], + implicitly[TypeTag[A9]], + implicitly[TypeTag[A10]], + implicitly[TypeTag[A11]], + implicitly[TypeTag[A12]], + implicitly[TypeTag[A13]]) } /** * Registers a deterministic Scala closure of 14 arguments as user-defined function (UDF). - * @tparam RT return type of UDF. + * @tparam RT + * return type of UDF. * @since 1.3.0 */ - def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag](name: String, func: Function14[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, RT]): UserDefinedFunction = { - registerScalaUDF(name, func, implicitly[TypeTag[RT]], implicitly[TypeTag[A1]], implicitly[TypeTag[A2]], implicitly[TypeTag[A3]], implicitly[TypeTag[A4]], implicitly[TypeTag[A5]], implicitly[TypeTag[A6]], implicitly[TypeTag[A7]], implicitly[TypeTag[A8]], implicitly[TypeTag[A9]], implicitly[TypeTag[A10]], implicitly[TypeTag[A11]], implicitly[TypeTag[A12]], implicitly[TypeTag[A13]], implicitly[TypeTag[A14]]) + def register[ + RT: TypeTag, + A1: TypeTag, + A2: TypeTag, + A3: TypeTag, + A4: TypeTag, + A5: TypeTag, + A6: TypeTag, + A7: TypeTag, + A8: TypeTag, + A9: TypeTag, + A10: TypeTag, + A11: TypeTag, + A12: TypeTag, + A13: TypeTag, + A14: TypeTag]( + name: String, + func: Function14[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, RT]) + : UserDefinedFunction = { + registerScalaUDF( + name, + func, + implicitly[TypeTag[RT]], + implicitly[TypeTag[A1]], + implicitly[TypeTag[A2]], + implicitly[TypeTag[A3]], + implicitly[TypeTag[A4]], + implicitly[TypeTag[A5]], + implicitly[TypeTag[A6]], + implicitly[TypeTag[A7]], + implicitly[TypeTag[A8]], + implicitly[TypeTag[A9]], + implicitly[TypeTag[A10]], + implicitly[TypeTag[A11]], + implicitly[TypeTag[A12]], + implicitly[TypeTag[A13]], + implicitly[TypeTag[A14]]) } /** * Registers a deterministic Scala closure of 15 arguments as user-defined function (UDF). - * @tparam RT return type of UDF. + * @tparam RT + * return type of UDF. * @since 1.3.0 */ - def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag](name: String, func: Function15[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, RT]): UserDefinedFunction = { - registerScalaUDF(name, func, implicitly[TypeTag[RT]], implicitly[TypeTag[A1]], implicitly[TypeTag[A2]], implicitly[TypeTag[A3]], implicitly[TypeTag[A4]], implicitly[TypeTag[A5]], implicitly[TypeTag[A6]], implicitly[TypeTag[A7]], implicitly[TypeTag[A8]], implicitly[TypeTag[A9]], implicitly[TypeTag[A10]], implicitly[TypeTag[A11]], implicitly[TypeTag[A12]], implicitly[TypeTag[A13]], implicitly[TypeTag[A14]], implicitly[TypeTag[A15]]) + def register[ + RT: TypeTag, + A1: TypeTag, + A2: TypeTag, + A3: TypeTag, + A4: TypeTag, + A5: TypeTag, + A6: TypeTag, + A7: TypeTag, + A8: TypeTag, + A9: TypeTag, + A10: TypeTag, + A11: TypeTag, + A12: TypeTag, + A13: TypeTag, + A14: TypeTag, + A15: TypeTag]( + name: String, + func: Function15[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, RT]) + : UserDefinedFunction = { + registerScalaUDF( + name, + func, + implicitly[TypeTag[RT]], + implicitly[TypeTag[A1]], + implicitly[TypeTag[A2]], + implicitly[TypeTag[A3]], + implicitly[TypeTag[A4]], + implicitly[TypeTag[A5]], + implicitly[TypeTag[A6]], + implicitly[TypeTag[A7]], + implicitly[TypeTag[A8]], + implicitly[TypeTag[A9]], + implicitly[TypeTag[A10]], + implicitly[TypeTag[A11]], + implicitly[TypeTag[A12]], + implicitly[TypeTag[A13]], + implicitly[TypeTag[A14]], + implicitly[TypeTag[A15]]) } /** * Registers a deterministic Scala closure of 16 arguments as user-defined function (UDF). - * @tparam RT return type of UDF. + * @tparam RT + * return type of UDF. * @since 1.3.0 */ - def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag](name: String, func: Function16[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, RT]): UserDefinedFunction = { - registerScalaUDF(name, func, implicitly[TypeTag[RT]], implicitly[TypeTag[A1]], implicitly[TypeTag[A2]], implicitly[TypeTag[A3]], implicitly[TypeTag[A4]], implicitly[TypeTag[A5]], implicitly[TypeTag[A6]], implicitly[TypeTag[A7]], implicitly[TypeTag[A8]], implicitly[TypeTag[A9]], implicitly[TypeTag[A10]], implicitly[TypeTag[A11]], implicitly[TypeTag[A12]], implicitly[TypeTag[A13]], implicitly[TypeTag[A14]], implicitly[TypeTag[A15]], implicitly[TypeTag[A16]]) + def register[ + RT: TypeTag, + A1: TypeTag, + A2: TypeTag, + A3: TypeTag, + A4: TypeTag, + A5: TypeTag, + A6: TypeTag, + A7: TypeTag, + A8: TypeTag, + A9: TypeTag, + A10: TypeTag, + A11: TypeTag, + A12: TypeTag, + A13: TypeTag, + A14: TypeTag, + A15: TypeTag, + A16: TypeTag]( + name: String, + func: Function16[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, RT]) + : UserDefinedFunction = { + registerScalaUDF( + name, + func, + implicitly[TypeTag[RT]], + implicitly[TypeTag[A1]], + implicitly[TypeTag[A2]], + implicitly[TypeTag[A3]], + implicitly[TypeTag[A4]], + implicitly[TypeTag[A5]], + implicitly[TypeTag[A6]], + implicitly[TypeTag[A7]], + implicitly[TypeTag[A8]], + implicitly[TypeTag[A9]], + implicitly[TypeTag[A10]], + implicitly[TypeTag[A11]], + implicitly[TypeTag[A12]], + implicitly[TypeTag[A13]], + implicitly[TypeTag[A14]], + implicitly[TypeTag[A15]], + implicitly[TypeTag[A16]]) } /** * Registers a deterministic Scala closure of 17 arguments as user-defined function (UDF). - * @tparam RT return type of UDF. + * @tparam RT + * return type of UDF. * @since 1.3.0 */ - def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag](name: String, func: Function17[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, RT]): UserDefinedFunction = { - registerScalaUDF(name, func, implicitly[TypeTag[RT]], implicitly[TypeTag[A1]], implicitly[TypeTag[A2]], implicitly[TypeTag[A3]], implicitly[TypeTag[A4]], implicitly[TypeTag[A5]], implicitly[TypeTag[A6]], implicitly[TypeTag[A7]], implicitly[TypeTag[A8]], implicitly[TypeTag[A9]], implicitly[TypeTag[A10]], implicitly[TypeTag[A11]], implicitly[TypeTag[A12]], implicitly[TypeTag[A13]], implicitly[TypeTag[A14]], implicitly[TypeTag[A15]], implicitly[TypeTag[A16]], implicitly[TypeTag[A17]]) + def register[ + RT: TypeTag, + A1: TypeTag, + A2: TypeTag, + A3: TypeTag, + A4: TypeTag, + A5: TypeTag, + A6: TypeTag, + A7: TypeTag, + A8: TypeTag, + A9: TypeTag, + A10: TypeTag, + A11: TypeTag, + A12: TypeTag, + A13: TypeTag, + A14: TypeTag, + A15: TypeTag, + A16: TypeTag, + A17: TypeTag]( + name: String, + func: Function17[ + A1, + A2, + A3, + A4, + A5, + A6, + A7, + A8, + A9, + A10, + A11, + A12, + A13, + A14, + A15, + A16, + A17, + RT]): UserDefinedFunction = { + registerScalaUDF( + name, + func, + implicitly[TypeTag[RT]], + implicitly[TypeTag[A1]], + implicitly[TypeTag[A2]], + implicitly[TypeTag[A3]], + implicitly[TypeTag[A4]], + implicitly[TypeTag[A5]], + implicitly[TypeTag[A6]], + implicitly[TypeTag[A7]], + implicitly[TypeTag[A8]], + implicitly[TypeTag[A9]], + implicitly[TypeTag[A10]], + implicitly[TypeTag[A11]], + implicitly[TypeTag[A12]], + implicitly[TypeTag[A13]], + implicitly[TypeTag[A14]], + implicitly[TypeTag[A15]], + implicitly[TypeTag[A16]], + implicitly[TypeTag[A17]]) } /** * Registers a deterministic Scala closure of 18 arguments as user-defined function (UDF). - * @tparam RT return type of UDF. + * @tparam RT + * return type of UDF. * @since 1.3.0 */ - def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag](name: String, func: Function18[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, RT]): UserDefinedFunction = { - registerScalaUDF(name, func, implicitly[TypeTag[RT]], implicitly[TypeTag[A1]], implicitly[TypeTag[A2]], implicitly[TypeTag[A3]], implicitly[TypeTag[A4]], implicitly[TypeTag[A5]], implicitly[TypeTag[A6]], implicitly[TypeTag[A7]], implicitly[TypeTag[A8]], implicitly[TypeTag[A9]], implicitly[TypeTag[A10]], implicitly[TypeTag[A11]], implicitly[TypeTag[A12]], implicitly[TypeTag[A13]], implicitly[TypeTag[A14]], implicitly[TypeTag[A15]], implicitly[TypeTag[A16]], implicitly[TypeTag[A17]], implicitly[TypeTag[A18]]) + def register[ + RT: TypeTag, + A1: TypeTag, + A2: TypeTag, + A3: TypeTag, + A4: TypeTag, + A5: TypeTag, + A6: TypeTag, + A7: TypeTag, + A8: TypeTag, + A9: TypeTag, + A10: TypeTag, + A11: TypeTag, + A12: TypeTag, + A13: TypeTag, + A14: TypeTag, + A15: TypeTag, + A16: TypeTag, + A17: TypeTag, + A18: TypeTag]( + name: String, + func: Function18[ + A1, + A2, + A3, + A4, + A5, + A6, + A7, + A8, + A9, + A10, + A11, + A12, + A13, + A14, + A15, + A16, + A17, + A18, + RT]): UserDefinedFunction = { + registerScalaUDF( + name, + func, + implicitly[TypeTag[RT]], + implicitly[TypeTag[A1]], + implicitly[TypeTag[A2]], + implicitly[TypeTag[A3]], + implicitly[TypeTag[A4]], + implicitly[TypeTag[A5]], + implicitly[TypeTag[A6]], + implicitly[TypeTag[A7]], + implicitly[TypeTag[A8]], + implicitly[TypeTag[A9]], + implicitly[TypeTag[A10]], + implicitly[TypeTag[A11]], + implicitly[TypeTag[A12]], + implicitly[TypeTag[A13]], + implicitly[TypeTag[A14]], + implicitly[TypeTag[A15]], + implicitly[TypeTag[A16]], + implicitly[TypeTag[A17]], + implicitly[TypeTag[A18]]) } /** * Registers a deterministic Scala closure of 19 arguments as user-defined function (UDF). - * @tparam RT return type of UDF. + * @tparam RT + * return type of UDF. * @since 1.3.0 */ - def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag](name: String, func: Function19[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, RT]): UserDefinedFunction = { - registerScalaUDF(name, func, implicitly[TypeTag[RT]], implicitly[TypeTag[A1]], implicitly[TypeTag[A2]], implicitly[TypeTag[A3]], implicitly[TypeTag[A4]], implicitly[TypeTag[A5]], implicitly[TypeTag[A6]], implicitly[TypeTag[A7]], implicitly[TypeTag[A8]], implicitly[TypeTag[A9]], implicitly[TypeTag[A10]], implicitly[TypeTag[A11]], implicitly[TypeTag[A12]], implicitly[TypeTag[A13]], implicitly[TypeTag[A14]], implicitly[TypeTag[A15]], implicitly[TypeTag[A16]], implicitly[TypeTag[A17]], implicitly[TypeTag[A18]], implicitly[TypeTag[A19]]) + def register[ + RT: TypeTag, + A1: TypeTag, + A2: TypeTag, + A3: TypeTag, + A4: TypeTag, + A5: TypeTag, + A6: TypeTag, + A7: TypeTag, + A8: TypeTag, + A9: TypeTag, + A10: TypeTag, + A11: TypeTag, + A12: TypeTag, + A13: TypeTag, + A14: TypeTag, + A15: TypeTag, + A16: TypeTag, + A17: TypeTag, + A18: TypeTag, + A19: TypeTag]( + name: String, + func: Function19[ + A1, + A2, + A3, + A4, + A5, + A6, + A7, + A8, + A9, + A10, + A11, + A12, + A13, + A14, + A15, + A16, + A17, + A18, + A19, + RT]): UserDefinedFunction = { + registerScalaUDF( + name, + func, + implicitly[TypeTag[RT]], + implicitly[TypeTag[A1]], + implicitly[TypeTag[A2]], + implicitly[TypeTag[A3]], + implicitly[TypeTag[A4]], + implicitly[TypeTag[A5]], + implicitly[TypeTag[A6]], + implicitly[TypeTag[A7]], + implicitly[TypeTag[A8]], + implicitly[TypeTag[A9]], + implicitly[TypeTag[A10]], + implicitly[TypeTag[A11]], + implicitly[TypeTag[A12]], + implicitly[TypeTag[A13]], + implicitly[TypeTag[A14]], + implicitly[TypeTag[A15]], + implicitly[TypeTag[A16]], + implicitly[TypeTag[A17]], + implicitly[TypeTag[A18]], + implicitly[TypeTag[A19]]) } /** * Registers a deterministic Scala closure of 20 arguments as user-defined function (UDF). - * @tparam RT return type of UDF. + * @tparam RT + * return type of UDF. * @since 1.3.0 */ - def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag, A20: TypeTag](name: String, func: Function20[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, RT]): UserDefinedFunction = { - registerScalaUDF(name, func, implicitly[TypeTag[RT]], implicitly[TypeTag[A1]], implicitly[TypeTag[A2]], implicitly[TypeTag[A3]], implicitly[TypeTag[A4]], implicitly[TypeTag[A5]], implicitly[TypeTag[A6]], implicitly[TypeTag[A7]], implicitly[TypeTag[A8]], implicitly[TypeTag[A9]], implicitly[TypeTag[A10]], implicitly[TypeTag[A11]], implicitly[TypeTag[A12]], implicitly[TypeTag[A13]], implicitly[TypeTag[A14]], implicitly[TypeTag[A15]], implicitly[TypeTag[A16]], implicitly[TypeTag[A17]], implicitly[TypeTag[A18]], implicitly[TypeTag[A19]], implicitly[TypeTag[A20]]) + def register[ + RT: TypeTag, + A1: TypeTag, + A2: TypeTag, + A3: TypeTag, + A4: TypeTag, + A5: TypeTag, + A6: TypeTag, + A7: TypeTag, + A8: TypeTag, + A9: TypeTag, + A10: TypeTag, + A11: TypeTag, + A12: TypeTag, + A13: TypeTag, + A14: TypeTag, + A15: TypeTag, + A16: TypeTag, + A17: TypeTag, + A18: TypeTag, + A19: TypeTag, + A20: TypeTag]( + name: String, + func: Function20[ + A1, + A2, + A3, + A4, + A5, + A6, + A7, + A8, + A9, + A10, + A11, + A12, + A13, + A14, + A15, + A16, + A17, + A18, + A19, + A20, + RT]): UserDefinedFunction = { + registerScalaUDF( + name, + func, + implicitly[TypeTag[RT]], + implicitly[TypeTag[A1]], + implicitly[TypeTag[A2]], + implicitly[TypeTag[A3]], + implicitly[TypeTag[A4]], + implicitly[TypeTag[A5]], + implicitly[TypeTag[A6]], + implicitly[TypeTag[A7]], + implicitly[TypeTag[A8]], + implicitly[TypeTag[A9]], + implicitly[TypeTag[A10]], + implicitly[TypeTag[A11]], + implicitly[TypeTag[A12]], + implicitly[TypeTag[A13]], + implicitly[TypeTag[A14]], + implicitly[TypeTag[A15]], + implicitly[TypeTag[A16]], + implicitly[TypeTag[A17]], + implicitly[TypeTag[A18]], + implicitly[TypeTag[A19]], + implicitly[TypeTag[A20]]) } /** * Registers a deterministic Scala closure of 21 arguments as user-defined function (UDF). - * @tparam RT return type of UDF. + * @tparam RT + * return type of UDF. * @since 1.3.0 */ - def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag, A20: TypeTag, A21: TypeTag](name: String, func: Function21[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, A21, RT]): UserDefinedFunction = { - registerScalaUDF(name, func, implicitly[TypeTag[RT]], implicitly[TypeTag[A1]], implicitly[TypeTag[A2]], implicitly[TypeTag[A3]], implicitly[TypeTag[A4]], implicitly[TypeTag[A5]], implicitly[TypeTag[A6]], implicitly[TypeTag[A7]], implicitly[TypeTag[A8]], implicitly[TypeTag[A9]], implicitly[TypeTag[A10]], implicitly[TypeTag[A11]], implicitly[TypeTag[A12]], implicitly[TypeTag[A13]], implicitly[TypeTag[A14]], implicitly[TypeTag[A15]], implicitly[TypeTag[A16]], implicitly[TypeTag[A17]], implicitly[TypeTag[A18]], implicitly[TypeTag[A19]], implicitly[TypeTag[A20]], implicitly[TypeTag[A21]]) + def register[ + RT: TypeTag, + A1: TypeTag, + A2: TypeTag, + A3: TypeTag, + A4: TypeTag, + A5: TypeTag, + A6: TypeTag, + A7: TypeTag, + A8: TypeTag, + A9: TypeTag, + A10: TypeTag, + A11: TypeTag, + A12: TypeTag, + A13: TypeTag, + A14: TypeTag, + A15: TypeTag, + A16: TypeTag, + A17: TypeTag, + A18: TypeTag, + A19: TypeTag, + A20: TypeTag, + A21: TypeTag]( + name: String, + func: Function21[ + A1, + A2, + A3, + A4, + A5, + A6, + A7, + A8, + A9, + A10, + A11, + A12, + A13, + A14, + A15, + A16, + A17, + A18, + A19, + A20, + A21, + RT]): UserDefinedFunction = { + registerScalaUDF( + name, + func, + implicitly[TypeTag[RT]], + implicitly[TypeTag[A1]], + implicitly[TypeTag[A2]], + implicitly[TypeTag[A3]], + implicitly[TypeTag[A4]], + implicitly[TypeTag[A5]], + implicitly[TypeTag[A6]], + implicitly[TypeTag[A7]], + implicitly[TypeTag[A8]], + implicitly[TypeTag[A9]], + implicitly[TypeTag[A10]], + implicitly[TypeTag[A11]], + implicitly[TypeTag[A12]], + implicitly[TypeTag[A13]], + implicitly[TypeTag[A14]], + implicitly[TypeTag[A15]], + implicitly[TypeTag[A16]], + implicitly[TypeTag[A17]], + implicitly[TypeTag[A18]], + implicitly[TypeTag[A19]], + implicitly[TypeTag[A20]], + implicitly[TypeTag[A21]]) } /** * Registers a deterministic Scala closure of 22 arguments as user-defined function (UDF). - * @tparam RT return type of UDF. + * @tparam RT + * return type of UDF. * @since 1.3.0 */ - def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag, A20: TypeTag, A21: TypeTag, A22: TypeTag](name: String, func: Function22[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, A21, A22, RT]): UserDefinedFunction = { - registerScalaUDF(name, func, implicitly[TypeTag[RT]], implicitly[TypeTag[A1]], implicitly[TypeTag[A2]], implicitly[TypeTag[A3]], implicitly[TypeTag[A4]], implicitly[TypeTag[A5]], implicitly[TypeTag[A6]], implicitly[TypeTag[A7]], implicitly[TypeTag[A8]], implicitly[TypeTag[A9]], implicitly[TypeTag[A10]], implicitly[TypeTag[A11]], implicitly[TypeTag[A12]], implicitly[TypeTag[A13]], implicitly[TypeTag[A14]], implicitly[TypeTag[A15]], implicitly[TypeTag[A16]], implicitly[TypeTag[A17]], implicitly[TypeTag[A18]], implicitly[TypeTag[A19]], implicitly[TypeTag[A20]], implicitly[TypeTag[A21]], implicitly[TypeTag[A22]]) + def register[ + RT: TypeTag, + A1: TypeTag, + A2: TypeTag, + A3: TypeTag, + A4: TypeTag, + A5: TypeTag, + A6: TypeTag, + A7: TypeTag, + A8: TypeTag, + A9: TypeTag, + A10: TypeTag, + A11: TypeTag, + A12: TypeTag, + A13: TypeTag, + A14: TypeTag, + A15: TypeTag, + A16: TypeTag, + A17: TypeTag, + A18: TypeTag, + A19: TypeTag, + A20: TypeTag, + A21: TypeTag, + A22: TypeTag]( + name: String, + func: Function22[ + A1, + A2, + A3, + A4, + A5, + A6, + A7, + A8, + A9, + A10, + A11, + A12, + A13, + A14, + A15, + A16, + A17, + A18, + A19, + A20, + A21, + A22, + RT]): UserDefinedFunction = { + registerScalaUDF( + name, + func, + implicitly[TypeTag[RT]], + implicitly[TypeTag[A1]], + implicitly[TypeTag[A2]], + implicitly[TypeTag[A3]], + implicitly[TypeTag[A4]], + implicitly[TypeTag[A5]], + implicitly[TypeTag[A6]], + implicitly[TypeTag[A7]], + implicitly[TypeTag[A8]], + implicitly[TypeTag[A9]], + implicitly[TypeTag[A10]], + implicitly[TypeTag[A11]], + implicitly[TypeTag[A12]], + implicitly[TypeTag[A13]], + implicitly[TypeTag[A14]], + implicitly[TypeTag[A15]], + implicitly[TypeTag[A16]], + implicitly[TypeTag[A17]], + implicitly[TypeTag[A18]], + implicitly[TypeTag[A19]], + implicitly[TypeTag[A20]], + implicitly[TypeTag[A21]], + implicitly[TypeTag[A22]]) } ////////////////////////////////////////////////////////////////////////////////////////////// @@ -405,7 +1173,10 @@ abstract class UDFRegistration { * Register a deterministic Java UDF9 instance as user-defined function (UDF). * @since 1.3.0 */ - def register(name: String, f: UDF9[_, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { + def register( + name: String, + f: UDF9[_, _, _, _, _, _, _, _, _, _], + returnType: DataType): Unit = { registerJavaUDF(name, ToScalaUDF(f), returnType, 9) } @@ -413,7 +1184,10 @@ abstract class UDFRegistration { * Register a deterministic Java UDF10 instance as user-defined function (UDF). * @since 1.3.0 */ - def register(name: String, f: UDF10[_, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { + def register( + name: String, + f: UDF10[_, _, _, _, _, _, _, _, _, _, _], + returnType: DataType): Unit = { registerJavaUDF(name, ToScalaUDF(f), returnType, 10) } @@ -421,7 +1195,10 @@ abstract class UDFRegistration { * Register a deterministic Java UDF11 instance as user-defined function (UDF). * @since 1.3.0 */ - def register(name: String, f: UDF11[_, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { + def register( + name: String, + f: UDF11[_, _, _, _, _, _, _, _, _, _, _, _], + returnType: DataType): Unit = { registerJavaUDF(name, ToScalaUDF(f), returnType, 11) } @@ -429,7 +1206,10 @@ abstract class UDFRegistration { * Register a deterministic Java UDF12 instance as user-defined function (UDF). * @since 1.3.0 */ - def register(name: String, f: UDF12[_, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { + def register( + name: String, + f: UDF12[_, _, _, _, _, _, _, _, _, _, _, _, _], + returnType: DataType): Unit = { registerJavaUDF(name, ToScalaUDF(f), returnType, 12) } @@ -437,7 +1217,10 @@ abstract class UDFRegistration { * Register a deterministic Java UDF13 instance as user-defined function (UDF). * @since 1.3.0 */ - def register(name: String, f: UDF13[_, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { + def register( + name: String, + f: UDF13[_, _, _, _, _, _, _, _, _, _, _, _, _, _], + returnType: DataType): Unit = { registerJavaUDF(name, ToScalaUDF(f), returnType, 13) } @@ -445,7 +1228,10 @@ abstract class UDFRegistration { * Register a deterministic Java UDF14 instance as user-defined function (UDF). * @since 1.3.0 */ - def register(name: String, f: UDF14[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { + def register( + name: String, + f: UDF14[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _], + returnType: DataType): Unit = { registerJavaUDF(name, ToScalaUDF(f), returnType, 14) } @@ -453,7 +1239,10 @@ abstract class UDFRegistration { * Register a deterministic Java UDF15 instance as user-defined function (UDF). * @since 1.3.0 */ - def register(name: String, f: UDF15[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { + def register( + name: String, + f: UDF15[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], + returnType: DataType): Unit = { registerJavaUDF(name, ToScalaUDF(f), returnType, 15) } @@ -461,7 +1250,10 @@ abstract class UDFRegistration { * Register a deterministic Java UDF16 instance as user-defined function (UDF). * @since 1.3.0 */ - def register(name: String, f: UDF16[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { + def register( + name: String, + f: UDF16[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], + returnType: DataType): Unit = { registerJavaUDF(name, ToScalaUDF(f), returnType, 16) } @@ -469,7 +1261,10 @@ abstract class UDFRegistration { * Register a deterministic Java UDF17 instance as user-defined function (UDF). * @since 1.3.0 */ - def register(name: String, f: UDF17[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { + def register( + name: String, + f: UDF17[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], + returnType: DataType): Unit = { registerJavaUDF(name, ToScalaUDF(f), returnType, 17) } @@ -477,7 +1272,10 @@ abstract class UDFRegistration { * Register a deterministic Java UDF18 instance as user-defined function (UDF). * @since 1.3.0 */ - def register(name: String, f: UDF18[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { + def register( + name: String, + f: UDF18[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], + returnType: DataType): Unit = { registerJavaUDF(name, ToScalaUDF(f), returnType, 18) } @@ -485,7 +1283,10 @@ abstract class UDFRegistration { * Register a deterministic Java UDF19 instance as user-defined function (UDF). * @since 1.3.0 */ - def register(name: String, f: UDF19[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { + def register( + name: String, + f: UDF19[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], + returnType: DataType): Unit = { registerJavaUDF(name, ToScalaUDF(f), returnType, 19) } @@ -493,7 +1294,10 @@ abstract class UDFRegistration { * Register a deterministic Java UDF20 instance as user-defined function (UDF). * @since 1.3.0 */ - def register(name: String, f: UDF20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { + def register( + name: String, + f: UDF20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], + returnType: DataType): Unit = { registerJavaUDF(name, ToScalaUDF(f), returnType, 20) } @@ -501,7 +1305,10 @@ abstract class UDFRegistration { * Register a deterministic Java UDF21 instance as user-defined function (UDF). * @since 1.3.0 */ - def register(name: String, f: UDF21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { + def register( + name: String, + f: UDF21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], + returnType: DataType): Unit = { registerJavaUDF(name, ToScalaUDF(f), returnType, 21) } @@ -509,7 +1316,10 @@ abstract class UDFRegistration { * Register a deterministic Java UDF22 instance as user-defined function (UDF). * @since 1.3.0 */ - def register(name: String, f: UDF22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { + def register( + name: String, + f: UDF22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], + returnType: DataType): Unit = { registerJavaUDF(name, ToScalaUDF(f), returnType, 22) } diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/DefinedByConstructorParams.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/DefinedByConstructorParams.scala index fc6bc2095a821..efd9d457ef7d4 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/DefinedByConstructorParams.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/DefinedByConstructorParams.scala @@ -18,8 +18,7 @@ package org.apache.spark.sql.catalyst /** - * A helper trait to create [[org.apache.spark.sql.catalyst.encoders.ExpressionEncoder]]s - * for classes whose fields are entirely defined by constructor params but should not be - * case classes. + * A helper trait to create [[org.apache.spark.sql.catalyst.encoders.ExpressionEncoder]]s for + * classes whose fields are entirely defined by constructor params but should not be case classes. */ trait DefinedByConstructorParams diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala index f85e96da2be11..8d0103ca69635 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala @@ -36,10 +36,13 @@ import org.apache.spark.util.ArrayImplicits._ * Type-inference utilities for POJOs and Java collections. */ object JavaTypeInference { + /** * Infers the corresponding SQL data type of a Java type. - * @param beanType Java type - * @return (SQL data type, nullable) + * @param beanType + * Java type + * @return + * (SQL data type, nullable) */ def inferDataType(beanType: Type): (DataType, Boolean) = { val encoder = encoderFor(beanType) @@ -60,8 +63,10 @@ object JavaTypeInference { encoderFor(beanType, Set.empty).asInstanceOf[AgnosticEncoder[T]] } - private def encoderFor(t: Type, seenTypeSet: Set[Class[_]], - typeVariables: Map[TypeVariable[_], Type] = Map.empty): AgnosticEncoder[_] = t match { + private def encoderFor( + t: Type, + seenTypeSet: Set[Class[_]], + typeVariables: Map[TypeVariable[_], Type] = Map.empty): AgnosticEncoder[_] = t match { case c: Class[_] if c == java.lang.Boolean.TYPE => PrimitiveBooleanEncoder case c: Class[_] if c == java.lang.Byte.TYPE => PrimitiveByteEncoder @@ -94,14 +99,22 @@ object JavaTypeInference { case c: Class[_] if c.isEnum => JavaEnumEncoder(ClassTag(c)) case c: Class[_] if c.isAnnotationPresent(classOf[SQLUserDefinedType]) => - val udt = c.getAnnotation(classOf[SQLUserDefinedType]).udt() - .getConstructor().newInstance().asInstanceOf[UserDefinedType[Any]] + val udt = c + .getAnnotation(classOf[SQLUserDefinedType]) + .udt() + .getConstructor() + .newInstance() + .asInstanceOf[UserDefinedType[Any]] val udtClass = udt.userClass.getAnnotation(classOf[SQLUserDefinedType]).udt() UDTEncoder(udt, udtClass) case c: Class[_] if UDTRegistration.exists(c.getName) => - val udt = UDTRegistration.getUDTFor(c.getName).get.getConstructor(). - newInstance().asInstanceOf[UserDefinedType[Any]] + val udt = UDTRegistration + .getUDTFor(c.getName) + .get + .getConstructor() + .newInstance() + .asInstanceOf[UserDefinedType[Any]] UDTEncoder(udt, udt.getClass) case c: Class[_] if c.isArray => @@ -160,7 +173,8 @@ object JavaTypeInference { def getJavaBeanReadableProperties(beanClass: Class[_]): Array[PropertyDescriptor] = { val beanInfo = Introspector.getBeanInfo(beanClass) - beanInfo.getPropertyDescriptors.filterNot(_.getName == "class") + beanInfo.getPropertyDescriptors + .filterNot(_.getName == "class") .filterNot(_.getName == "declaringClass") .filter(_.getReadMethod != null) } diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index f204421b0add2..cd12cbd267cc4 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -55,9 +55,8 @@ object ScalaReflection extends ScalaReflection { import scala.collection.Map /** - * Synchronize to prevent concurrent usage of `<:<` operator. - * This operator is not thread safe in any current version of scala; i.e. - * (2.11.12, 2.12.10, 2.13.0-M5). + * Synchronize to prevent concurrent usage of `<:<` operator. This operator is not thread safe + * in any current version of scala; i.e. (2.11.12, 2.12.10, 2.13.0-M5). * * See https://github.com/scala/bug/issues/10766 */ @@ -91,11 +90,11 @@ object ScalaReflection extends ScalaReflection { /** * Workaround for [[https://github.com/scala/bug/issues/12190 Scala bug #12190]] * - * `ClassSymbol.selfType` can throw an exception in case of cyclic annotation reference - * in Java classes. A retry of this operation will succeed as the class which defines the - * cycle is now resolved. It can however expose further recursive annotation references, so - * we keep retrying until we exhaust our retry threshold. Default threshold is set to 5 - * to allow for a few level of cyclic references. + * `ClassSymbol.selfType` can throw an exception in case of cyclic annotation reference in Java + * classes. A retry of this operation will succeed as the class which defines the cycle is now + * resolved. It can however expose further recursive annotation references, so we keep retrying + * until we exhaust our retry threshold. Default threshold is set to 5 to allow for a few level + * of cyclic references. */ @tailrec private def selfType(clsSymbol: ClassSymbol, tries: Int = 5): Type = { @@ -107,7 +106,7 @@ object ScalaReflection extends ScalaReflection { // Retry on Symbols#CyclicReference if we haven't exhausted our retry limit selfType(clsSymbol, tries - 1) case Failure(e: RuntimeException) - if e.getMessage.contains("illegal cyclic reference") && tries > 1 => + if e.getMessage.contains("illegal cyclic reference") && tries > 1 => // UnPickler.unpickle wraps the original Symbols#CyclicReference exception into a runtime // exception and does not set the cause, so we inspect the message. The previous case // statement is useful for Java classes while this one is for Scala classes. @@ -131,14 +130,14 @@ object ScalaReflection extends ScalaReflection { } /** - * Returns the full class name for a type. The returned name is the canonical - * Scala name, where each component is separated by a period. It is NOT the - * Java-equivalent runtime name (no dollar signs). + * Returns the full class name for a type. The returned name is the canonical Scala name, where + * each component is separated by a period. It is NOT the Java-equivalent runtime name (no + * dollar signs). * - * In simple cases, both the Scala and Java names are the same, however when Scala - * generates constructs that do not map to a Java equivalent, such as singleton objects - * or nested classes in package objects, it uses the dollar sign ($) to create - * synthetic classes, emulating behaviour in Java bytecode. + * In simple cases, both the Scala and Java names are the same, however when Scala generates + * constructs that do not map to a Java equivalent, such as singleton objects or nested classes + * in package objects, it uses the dollar sign ($) to create synthetic classes, emulating + * behaviour in Java bytecode. */ def getClassNameFromType(tpe: `Type`): String = { erasure(tpe).dealias.typeSymbol.asClass.fullName @@ -152,20 +151,25 @@ object ScalaReflection extends ScalaReflection { case class Schema(dataType: DataType, nullable: Boolean) - /** Returns a catalyst DataType and its nullability for the given Scala Type using reflection. */ + /** + * Returns a catalyst DataType and its nullability for the given Scala Type using reflection. + */ def schemaFor[T: TypeTag]: Schema = schemaFor(localTypeOf[T]) - /** Returns a catalyst DataType and its nullability for the given Scala Type using reflection. */ + /** + * Returns a catalyst DataType and its nullability for the given Scala Type using reflection. + */ def schemaFor(tpe: `Type`): Schema = { val enc = encoderFor(tpe) Schema(enc.dataType, enc.nullable) } /** - * Finds an accessible constructor with compatible parameters. This is a more flexible search than - * the exact matching algorithm in `Class.getConstructor`. The first assignment-compatible + * Finds an accessible constructor with compatible parameters. This is a more flexible search + * than the exact matching algorithm in `Class.getConstructor`. The first assignment-compatible * matching constructor is returned if it exists. Otherwise, we check for additional compatible - * constructors defined in the companion object as `apply` methods. Otherwise, it returns `None`. + * constructors defined in the companion object as `apply` methods. Otherwise, it returns + * `None`. */ def findConstructor[T](cls: Class[T], paramTypes: Seq[Class[_]]): Option[Seq[AnyRef] => T] = { Option(ConstructorUtils.getMatchingAccessibleConstructor(cls, paramTypes: _*)) match { @@ -174,24 +178,28 @@ object ScalaReflection extends ScalaReflection { val companion = mirror.staticClass(cls.getName).companion val moduleMirror = mirror.reflectModule(companion.asModule) val applyMethods = companion.asTerm.typeSignature - .member(universe.TermName("apply")).asTerm.alternatives - applyMethods.find { method => - val params = method.typeSignature.paramLists.head - // Check that the needed params are the same length and of matching types - params.size == paramTypes.size && - params.zip(paramTypes).forall { case(ps, pc) => - ps.typeSignature.typeSymbol == mirror.classSymbol(pc) + .member(universe.TermName("apply")) + .asTerm + .alternatives + applyMethods + .find { method => + val params = method.typeSignature.paramLists.head + // Check that the needed params are the same length and of matching types + params.size == paramTypes.size && + params.zip(paramTypes).forall { case (ps, pc) => + ps.typeSignature.typeSymbol == mirror.classSymbol(pc) + } } - }.map { applyMethodSymbol => - val expectedArgsCount = applyMethodSymbol.typeSignature.paramLists.head.size - val instanceMirror = mirror.reflect(moduleMirror.instance) - val method = instanceMirror.reflectMethod(applyMethodSymbol.asMethod) - (_args: Seq[AnyRef]) => { - // Drop the "outer" argument if it is provided - val args = if (_args.size == expectedArgsCount) _args else _args.tail - method.apply(args: _*).asInstanceOf[T] + .map { applyMethodSymbol => + val expectedArgsCount = applyMethodSymbol.typeSignature.paramLists.head.size + val instanceMirror = mirror.reflect(moduleMirror.instance) + val method = instanceMirror.reflectMethod(applyMethodSymbol.asMethod) + (_args: Seq[AnyRef]) => { + // Drop the "outer" argument if it is provided + val args = if (_args.size == expectedArgsCount) _args else _args.tail + method.apply(args: _*).asInstanceOf[T] + } } - } } } @@ -201,8 +209,10 @@ object ScalaReflection extends ScalaReflection { def definedByConstructorParams(tpe: Type): Boolean = cleanUpReflectionObjects { tpe.dealias match { // `Option` is a `Product`, but we don't wanna treat `Option[Int]` as a struct type. - case t if isSubtype(t, localTypeOf[Option[_]]) => definedByConstructorParams(t.typeArgs.head) - case _ => isSubtype(tpe.dealias, localTypeOf[Product]) || + case t if isSubtype(t, localTypeOf[Option[_]]) => + definedByConstructorParams(t.typeArgs.head) + case _ => + isSubtype(tpe.dealias, localTypeOf[Product]) || isSubtype(tpe.dealias, localTypeOf[DefinedByConstructorParams]) } } @@ -214,16 +224,15 @@ object ScalaReflection extends ScalaReflection { /** * Create an [[AgnosticEncoder]] from a [[TypeTag]]. * - * If the given type is not supported, i.e. there is no encoder can be built for this type, - * an [[SparkUnsupportedOperationException]] will be thrown with detailed error message to - * explain the type path walked so far and which class we are not supporting. - * There are 4 kinds of type path: - * * the root type: `root class: "abc.xyz.MyClass"` - * * the value type of [[Option]]: `option value class: "abc.xyz.MyClass"` - * * the element type of [[Array]] or [[Seq]]: `array element class: "abc.xyz.MyClass"` - * * the field of [[Product]]: `field (class: "abc.xyz.MyClass", name: "myField")` + * If the given type is not supported, i.e. there is no encoder can be built for this type, an + * [[SparkUnsupportedOperationException]] will be thrown with detailed error message to explain + * the type path walked so far and which class we are not supporting. There are 4 kinds of type + * path: * the root type: `root class: "abc.xyz.MyClass"` * the value type of [[Option]]: + * `option value class: "abc.xyz.MyClass"` * the element type of [[Array]] or [[Seq]]: `array + * element class: "abc.xyz.MyClass"` * the field of [[Product]]: `field (class: + * "abc.xyz.MyClass", name: "myField")` */ - def encoderFor[E : TypeTag]: AgnosticEncoder[E] = { + def encoderFor[E: TypeTag]: AgnosticEncoder[E] = { encoderFor(typeTag[E].in(mirror).tpe).asInstanceOf[AgnosticEncoder[E]] } @@ -239,13 +248,12 @@ object ScalaReflection extends ScalaReflection { /** * Create an [[AgnosticEncoder]] for a [[Type]]. */ - def encoderFor( - tpe: `Type`, - isRowEncoderSupported: Boolean = false): AgnosticEncoder[_] = cleanUpReflectionObjects { - val clsName = getClassNameFromType(tpe) - val walkedTypePath = WalkedTypePath().recordRoot(clsName) - encoderFor(tpe, Set.empty, walkedTypePath, isRowEncoderSupported) - } + def encoderFor(tpe: `Type`, isRowEncoderSupported: Boolean = false): AgnosticEncoder[_] = + cleanUpReflectionObjects { + val clsName = getClassNameFromType(tpe) + val walkedTypePath = WalkedTypePath().recordRoot(clsName) + encoderFor(tpe, Set.empty, walkedTypePath, isRowEncoderSupported) + } private def encoderFor( tpe: `Type`, @@ -327,14 +335,22 @@ object ScalaReflection extends ScalaReflection { // UDT encoders case t if t.typeSymbol.annotations.exists(_.tree.tpe =:= typeOf[SQLUserDefinedType]) => - val udt = getClassFromType(t).getAnnotation(classOf[SQLUserDefinedType]).udt(). - getConstructor().newInstance().asInstanceOf[UserDefinedType[Any]] + val udt = getClassFromType(t) + .getAnnotation(classOf[SQLUserDefinedType]) + .udt() + .getConstructor() + .newInstance() + .asInstanceOf[UserDefinedType[Any]] val udtClass = udt.userClass.getAnnotation(classOf[SQLUserDefinedType]).udt() UDTEncoder(udt, udtClass) case t if UDTRegistration.exists(getClassNameFromType(t)) => - val udt = UDTRegistration.getUDTFor(getClassNameFromType(t)).get.getConstructor(). - newInstance().asInstanceOf[UserDefinedType[Any]] + val udt = UDTRegistration + .getUDTFor(getClassNameFromType(t)) + .get + .getConstructor() + .newInstance() + .asInstanceOf[UserDefinedType[Any]] UDTEncoder(udt, udt.getClass) // Complex encoders @@ -380,20 +396,17 @@ object ScalaReflection extends ScalaReflection { if (seenTypeSet.contains(t)) { throw ExecutionErrors.cannotHaveCircularReferencesInClassError(t.toString) } - val params = getConstructorParameters(t).map { - case (fieldName, fieldType) => - if (SourceVersion.isKeyword(fieldName) || - !SourceVersion.isIdentifier(encodeFieldNameToIdentifier(fieldName))) { - throw ExecutionErrors.cannotUseInvalidJavaIdentifierAsFieldNameError( - fieldName, - path) - } - val encoder = encoderFor( - fieldType, - seenTypeSet + t, - path.recordField(getClassNameFromType(fieldType), fieldName), - isRowEncoderSupported) - EncoderField(fieldName, encoder, encoder.nullable, Metadata.empty) + val params = getConstructorParameters(t).map { case (fieldName, fieldType) => + if (SourceVersion.isKeyword(fieldName) || + !SourceVersion.isIdentifier(encodeFieldNameToIdentifier(fieldName))) { + throw ExecutionErrors.cannotUseInvalidJavaIdentifierAsFieldNameError(fieldName, path) + } + val encoder = encoderFor( + fieldType, + seenTypeSet + t, + path.recordField(getClassNameFromType(fieldType), fieldName), + isRowEncoderSupported) + EncoderField(fieldName, encoder, encoder.nullable, Metadata.empty) } val cls = getClassFromType(t) ProductEncoder(ClassTag(cls), params, Option(OuterScopes.getOuterScope(cls))) @@ -404,10 +417,11 @@ object ScalaReflection extends ScalaReflection { } /** - * Support for generating catalyst schemas for scala objects. Note that unlike its companion + * Support for generating catalyst schemas for scala objects. Note that unlike its companion * object, this trait able to work in both the runtime and the compile time (macro) universe. */ trait ScalaReflection extends Logging { + /** The universe we work in (runtime or macro) */ val universe: scala.reflect.api.Universe @@ -421,7 +435,8 @@ trait ScalaReflection extends Logging { * clean up the Scala reflection garbage automatically. Otherwise, it will leak some objects to * `scala.reflect.runtime.JavaUniverse.undoLog`. * - * @see https://github.com/scala/bug/issues/8302 + * @see + * https://github.com/scala/bug/issues/8302 */ def cleanUpReflectionObjects[T](func: => T): T = { universe.asInstanceOf[scala.reflect.runtime.JavaUniverse].undoLog.undo(func) @@ -430,12 +445,13 @@ trait ScalaReflection extends Logging { /** * Return the Scala Type for `T` in the current classloader mirror. * - * Use this method instead of the convenience method `universe.typeOf`, which - * assumes that all types can be found in the classloader that loaded scala-reflect classes. - * That's not necessarily the case when running using Eclipse launchers or even - * Sbt console or test (without `fork := true`). + * Use this method instead of the convenience method `universe.typeOf`, which assumes that all + * types can be found in the classloader that loaded scala-reflect classes. That's not + * necessarily the case when running using Eclipse launchers or even Sbt console or test + * (without `fork := true`). * - * @see SPARK-5281 + * @see + * SPARK-5281 */ def localTypeOf[T: TypeTag]: `Type` = { val tag = implicitly[TypeTag[T]] @@ -474,8 +490,8 @@ trait ScalaReflection extends Logging { } /** - * If our type is a Scala trait it may have a companion object that - * only defines a constructor via `apply` method. + * If our type is a Scala trait it may have a companion object that only defines a constructor + * via `apply` method. */ private def getCompanionConstructor(tpe: Type): Symbol = { def throwUnsupportedOperation = { @@ -483,10 +499,11 @@ trait ScalaReflection extends Logging { } tpe.typeSymbol.asClass.companion match { case NoSymbol => throwUnsupportedOperation - case sym => sym.asTerm.typeSignature.member(universe.TermName("apply")) match { - case NoSymbol => throwUnsupportedOperation - case constructorSym => constructorSym - } + case sym => + sym.asTerm.typeSignature.member(universe.TermName("apply")) match { + case NoSymbol => throwUnsupportedOperation + case constructorSym => constructorSym + } } } @@ -499,8 +516,9 @@ trait ScalaReflection extends Logging { constructorSymbol.asMethod.paramLists } else { // Find the primary constructor, and use its parameter ordering. - val primaryConstructorSymbol: Option[Symbol] = constructorSymbol.asTerm.alternatives.find( - s => s.isMethod && s.asMethod.isPrimaryConstructor) + val primaryConstructorSymbol: Option[Symbol] = + constructorSymbol.asTerm.alternatives.find(s => + s.isMethod && s.asMethod.isPrimaryConstructor) if (primaryConstructorSymbol.isEmpty) { throw ExecutionErrors.primaryConstructorNotFoundError(tpe.getClass) } else { diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/WalkedTypePath.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/WalkedTypePath.scala index cbf1f01344c92..a81c071295e2c 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/WalkedTypePath.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/WalkedTypePath.scala @@ -18,9 +18,9 @@ package org.apache.spark.sql.catalyst /** - * This class records the paths the serializer and deserializer walk through to reach current path. - * Note that this class adds new path in prior to recorded paths so it maintains - * the paths as reverse order. + * This class records the paths the serializer and deserializer walk through to reach current + * path. Note that this class adds new path in prior to recorded paths so it maintains the paths + * as reverse order. */ case class WalkedTypePath(private val walkedPaths: Seq[String] = Nil) { def recordRoot(className: String): WalkedTypePath = @@ -33,7 +33,8 @@ case class WalkedTypePath(private val walkedPaths: Seq[String] = Nil) { newInstance(s"""- array element class: "$elementClassName"""") def recordMap(keyClassName: String, valueClassName: String): WalkedTypePath = { - newInstance(s"""- map key class: "$keyClassName"""" + + newInstance( + s"""- map key class: "$keyClassName"""" + s""", value class: "$valueClassName"""") } diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/analysis/NonEmptyException.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/analysis/NonEmptyException.scala index 9955f1b7bd301..913881f326c90 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/analysis/NonEmptyException.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/analysis/NonEmptyException.scala @@ -20,20 +20,17 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.util.QuotingUtils.quoted - /** - * Thrown by a catalog when an item already exists. The analyzer will rethrow the exception - * as an [[org.apache.spark.sql.AnalysisException]] with the correct position information. + * Thrown by a catalog when an item already exists. The analyzer will rethrow the exception as an + * [[org.apache.spark.sql.AnalysisException]] with the correct position information. */ case class NonEmptyNamespaceException( namespace: Array[String], details: String, override val cause: Option[Throwable] = None) - extends AnalysisException( - errorClass = "_LEGACY_ERROR_TEMP_3103", - messageParameters = Map( - "namespace" -> quoted(namespace), - "details" -> details)) { + extends AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_3103", + messageParameters = Map("namespace" -> quoted(namespace), "details" -> details)) { def this(namespace: Array[String]) = this(namespace, "", None) } diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/analysis/SqlApiAnalysis.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/analysis/SqlApiAnalysis.scala index 9f5a5b8875b33..f218a12209d61 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/analysis/SqlApiAnalysis.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/analysis/SqlApiAnalysis.scala @@ -17,9 +17,10 @@ package org.apache.spark.sql.catalyst.analysis object SqlApiAnalysis { + /** - * Resolver should return true if the first string refers to the same entity as the second string. - * For example, by using case insensitive equality. + * Resolver should return true if the first string refers to the same entity as the second + * string. For example, by using case insensitive equality. */ type Resolver = (String, String) => Boolean } diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/analysis/alreadyExistException.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/analysis/alreadyExistException.scala index fae3711baf706..0c667dd8916ee 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/analysis/alreadyExistException.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/analysis/alreadyExistException.scala @@ -25,21 +25,21 @@ import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.util.ArrayImplicits._ /** - * Thrown by a catalog when an item already exists. The analyzer will rethrow the exception - * as an [[org.apache.spark.sql.AnalysisException]] with the correct position information. + * Thrown by a catalog when an item already exists. The analyzer will rethrow the exception as an + * [[org.apache.spark.sql.AnalysisException]] with the correct position information. */ class DatabaseAlreadyExistsException(db: String) - extends NamespaceAlreadyExistsException(Array(db)) + extends NamespaceAlreadyExistsException(Array(db)) // any changes to this class should be backward compatible as it may be used by external connectors -class NamespaceAlreadyExistsException private( +class NamespaceAlreadyExistsException private ( message: String, errorClass: Option[String], messageParameters: Map[String, String]) - extends AnalysisException( - message, - errorClass = errorClass, - messageParameters = messageParameters) { + extends AnalysisException( + message, + errorClass = errorClass, + messageParameters = messageParameters) { def this(errorClass: String, messageParameters: Map[String, String]) = { this( @@ -49,24 +49,28 @@ class NamespaceAlreadyExistsException private( } def this(namespace: Array[String]) = { - this(errorClass = "SCHEMA_ALREADY_EXISTS", + this( + errorClass = "SCHEMA_ALREADY_EXISTS", Map("schemaName" -> quoteNameParts(namespace.toImmutableArraySeq))) } } // any changes to this class should be backward compatible as it may be used by external connectors -class TableAlreadyExistsException private( +class TableAlreadyExistsException private ( message: String, cause: Option[Throwable], errorClass: Option[String], messageParameters: Map[String, String]) - extends AnalysisException( - message, - cause = cause, - errorClass = errorClass, - messageParameters = messageParameters) { + extends AnalysisException( + message, + cause = cause, + errorClass = errorClass, + messageParameters = messageParameters) { - def this(errorClass: String, messageParameters: Map[String, String], cause: Option[Throwable]) = { + def this( + errorClass: String, + messageParameters: Map[String, String], + cause: Option[Throwable]) = { this( SparkThrowableHelper.getMessage(errorClass, messageParameters), cause, @@ -75,47 +79,53 @@ class TableAlreadyExistsException private( } def this(db: String, table: String) = { - this(errorClass = "TABLE_OR_VIEW_ALREADY_EXISTS", - messageParameters = Map("relationName" -> - (quoteIdentifier(db) + "." + quoteIdentifier(table))), + this( + errorClass = "TABLE_OR_VIEW_ALREADY_EXISTS", + messageParameters = Map( + "relationName" -> + (quoteIdentifier(db) + "." + quoteIdentifier(table))), cause = None) } def this(table: String) = { - this(errorClass = "TABLE_OR_VIEW_ALREADY_EXISTS", - messageParameters = Map("relationName" -> - quoteNameParts(AttributeNameParser.parseAttributeName(table))), + this( + errorClass = "TABLE_OR_VIEW_ALREADY_EXISTS", + messageParameters = Map( + "relationName" -> + quoteNameParts(AttributeNameParser.parseAttributeName(table))), cause = None) } def this(table: Seq[String]) = { - this(errorClass = "TABLE_OR_VIEW_ALREADY_EXISTS", + this( + errorClass = "TABLE_OR_VIEW_ALREADY_EXISTS", messageParameters = Map("relationName" -> quoteNameParts(table)), cause = None) } def this(tableIdent: Identifier) = { - this(errorClass = "TABLE_OR_VIEW_ALREADY_EXISTS", + this( + errorClass = "TABLE_OR_VIEW_ALREADY_EXISTS", messageParameters = Map("relationName" -> quoted(tableIdent)), cause = None) } } -class TempTableAlreadyExistsException private( +class TempTableAlreadyExistsException private ( message: String, cause: Option[Throwable], errorClass: Option[String], messageParameters: Map[String, String]) - extends AnalysisException( - message, - cause = cause, - errorClass = errorClass, - messageParameters = messageParameters) { + extends AnalysisException( + message, + cause = cause, + errorClass = errorClass, + messageParameters = messageParameters) { def this( - errorClass: String, - messageParameters: Map[String, String], - cause: Option[Throwable] = None) = { + errorClass: String, + messageParameters: Map[String, String], + cause: Option[Throwable] = None) = { this( SparkThrowableHelper.getMessage(errorClass, messageParameters), cause, @@ -126,27 +136,31 @@ class TempTableAlreadyExistsException private( def this(table: String) = { this( errorClass = "TEMP_TABLE_OR_VIEW_ALREADY_EXISTS", - messageParameters = Map("relationName" - -> quoteNameParts(AttributeNameParser.parseAttributeName(table)))) + messageParameters = Map( + "relationName" + -> quoteNameParts(AttributeNameParser.parseAttributeName(table)))) } } // any changes to this class should be backward compatible as it may be used by external connectors class ViewAlreadyExistsException(errorClass: String, messageParameters: Map[String, String]) - extends AnalysisException(errorClass, messageParameters) { + extends AnalysisException(errorClass, messageParameters) { def this(ident: Identifier) = - this(errorClass = "VIEW_ALREADY_EXISTS", + this( + errorClass = "VIEW_ALREADY_EXISTS", messageParameters = Map("relationName" -> quoted(ident))) } // any changes to this class should be backward compatible as it may be used by external connectors class FunctionAlreadyExistsException(errorClass: String, messageParameters: Map[String, String]) - extends AnalysisException(errorClass, messageParameters) { + extends AnalysisException(errorClass, messageParameters) { def this(function: Seq[String]) = { - this (errorClass = "ROUTINE_ALREADY_EXISTS", - Map("routineName" -> quoteNameParts(function), + this( + errorClass = "ROUTINE_ALREADY_EXISTS", + Map( + "routineName" -> quoteNameParts(function), "newRoutineType" -> "routine", "existingRoutineType" -> "routine")) } @@ -157,16 +171,16 @@ class FunctionAlreadyExistsException(errorClass: String, messageParameters: Map[ } // any changes to this class should be backward compatible as it may be used by external connectors -class IndexAlreadyExistsException private( +class IndexAlreadyExistsException private ( message: String, cause: Option[Throwable], errorClass: Option[String], messageParameters: Map[String, String]) - extends AnalysisException( - message, - cause = cause, - errorClass = errorClass, - messageParameters = messageParameters) { + extends AnalysisException( + message, + cause = cause, + errorClass = errorClass, + messageParameters = messageParameters) { def this( errorClass: String, diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/analysis/noSuchItemsExceptions.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/analysis/noSuchItemsExceptions.scala index 1b836da45e802..dbc7622c761e7 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/analysis/noSuchItemsExceptions.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/analysis/noSuchItemsExceptions.scala @@ -24,21 +24,24 @@ import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.util.ArrayImplicits._ /** - * Thrown by a catalog when an item cannot be found. The analyzer will rethrow the exception - * as an [[org.apache.spark.sql.AnalysisException]] with the correct position information. + * Thrown by a catalog when an item cannot be found. The analyzer will rethrow the exception as an + * [[org.apache.spark.sql.AnalysisException]] with the correct position information. */ -class NoSuchDatabaseException private[analysis]( +class NoSuchDatabaseException private[analysis] ( message: String, cause: Option[Throwable], errorClass: Option[String], messageParameters: Map[String, String]) - extends AnalysisException( - message, - cause = cause, - errorClass = errorClass, - messageParameters = messageParameters) { + extends AnalysisException( + message, + cause = cause, + errorClass = errorClass, + messageParameters = messageParameters) { - def this(errorClass: String, messageParameters: Map[String, String], cause: Option[Throwable]) = { + def this( + errorClass: String, + messageParameters: Map[String, String], + cause: Option[Throwable]) = { this( SparkThrowableHelper.getMessage(errorClass, messageParameters), cause = cause, @@ -55,16 +58,16 @@ class NoSuchDatabaseException private[analysis]( } // any changes to this class should be backward compatible as it may be used by external connectors -class NoSuchNamespaceException private( +class NoSuchNamespaceException private ( message: String, cause: Option[Throwable], errorClass: Option[String], messageParameters: Map[String, String]) - extends NoSuchDatabaseException( - message, - cause = cause, - errorClass = errorClass, - messageParameters = messageParameters) { + extends NoSuchDatabaseException( + message, + cause = cause, + errorClass = errorClass, + messageParameters = messageParameters) { def this(errorClass: String, messageParameters: Map[String, String]) = { this( @@ -75,29 +78,32 @@ class NoSuchNamespaceException private( } def this(namespace: Seq[String]) = { - this(errorClass = "SCHEMA_NOT_FOUND", - Map("schemaName" -> quoteNameParts(namespace))) + this(errorClass = "SCHEMA_NOT_FOUND", Map("schemaName" -> quoteNameParts(namespace))) } def this(namespace: Array[String]) = { - this(errorClass = "SCHEMA_NOT_FOUND", + this( + errorClass = "SCHEMA_NOT_FOUND", Map("schemaName" -> quoteNameParts(namespace.toImmutableArraySeq))) } } // any changes to this class should be backward compatible as it may be used by external connectors -class NoSuchTableException private( +class NoSuchTableException private ( message: String, cause: Option[Throwable], errorClass: Option[String], messageParameters: Map[String, String]) - extends AnalysisException( - message, - cause = cause, - errorClass = errorClass, - messageParameters = messageParameters) { + extends AnalysisException( + message, + cause = cause, + errorClass = errorClass, + messageParameters = messageParameters) { - def this(errorClass: String, messageParameters: Map[String, String], cause: Option[Throwable]) = { + def this( + errorClass: String, + messageParameters: Map[String, String], + cause: Option[Throwable]) = { this( SparkThrowableHelper.getMessage(errorClass, messageParameters), cause = cause, @@ -108,12 +114,13 @@ class NoSuchTableException private( def this(db: String, table: String) = { this( errorClass = "TABLE_OR_VIEW_NOT_FOUND", - messageParameters = Map("relationName" -> - (quoteIdentifier(db) + "." + quoteIdentifier(table))), + messageParameters = Map( + "relationName" -> + (quoteIdentifier(db) + "." + quoteIdentifier(table))), cause = None) } - def this(name : Seq[String]) = { + def this(name: Seq[String]) = { this( errorClass = "TABLE_OR_VIEW_NOT_FOUND", messageParameters = Map("relationName" -> quoteNameParts(name)), @@ -130,28 +137,28 @@ class NoSuchTableException private( // any changes to this class should be backward compatible as it may be used by external connectors class NoSuchViewException(errorClass: String, messageParameters: Map[String, String]) - extends AnalysisException(errorClass, messageParameters) { + extends AnalysisException(errorClass, messageParameters) { def this(ident: Identifier) = - this(errorClass = "VIEW_NOT_FOUND", - messageParameters = Map("relationName" -> quoted(ident))) + this(errorClass = "VIEW_NOT_FOUND", messageParameters = Map("relationName" -> quoted(ident))) } class NoSuchPermanentFunctionException(db: String, func: String) - extends AnalysisException(errorClass = "ROUTINE_NOT_FOUND", - Map("routineName" -> (quoteIdentifier(db) + "." + quoteIdentifier(func)))) + extends AnalysisException( + errorClass = "ROUTINE_NOT_FOUND", + Map("routineName" -> (quoteIdentifier(db) + "." + quoteIdentifier(func)))) // any changes to this class should be backward compatible as it may be used by external connectors -class NoSuchFunctionException private( +class NoSuchFunctionException private ( message: String, cause: Option[Throwable], errorClass: Option[String], messageParameters: Map[String, String]) - extends AnalysisException( - message, - cause = cause, - errorClass = errorClass, - messageParameters = messageParameters) { + extends AnalysisException( + message, + cause = cause, + errorClass = errorClass, + messageParameters = messageParameters) { def this(errorClass: String, messageParameters: Map[String, String]) = { this( @@ -162,7 +169,8 @@ class NoSuchFunctionException private( } def this(db: String, func: String) = { - this(errorClass = "ROUTINE_NOT_FOUND", + this( + errorClass = "ROUTINE_NOT_FOUND", Map("routineName" -> (quoteIdentifier(db) + "." + quoteIdentifier(func)))) } @@ -172,19 +180,19 @@ class NoSuchFunctionException private( } class NoSuchTempFunctionException(func: String) - extends AnalysisException(errorClass = "ROUTINE_NOT_FOUND", Map("routineName" -> s"`$func`")) + extends AnalysisException(errorClass = "ROUTINE_NOT_FOUND", Map("routineName" -> s"`$func`")) // any changes to this class should be backward compatible as it may be used by external connectors -class NoSuchIndexException private( +class NoSuchIndexException private ( message: String, cause: Option[Throwable], errorClass: Option[String], messageParameters: Map[String, String]) - extends AnalysisException( - message, - cause = cause, - errorClass = errorClass, - messageParameters = messageParameters) { + extends AnalysisException( + message, + cause = cause, + errorClass = errorClass, + messageParameters = messageParameters) { def this( errorClass: String, @@ -205,8 +213,10 @@ class NoSuchIndexException private( class CannotReplaceMissingTableException( tableIdentifier: Identifier, cause: Option[Throwable] = None) - extends AnalysisException( - errorClass = "TABLE_OR_VIEW_NOT_FOUND", - messageParameters = Map("relationName" - -> quoteNameParts((tableIdentifier.namespace :+ tableIdentifier.name).toImmutableArraySeq)), - cause = cause) + extends AnalysisException( + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + messageParameters = Map( + "relationName" + -> quoteNameParts( + (tableIdentifier.namespace :+ tableIdentifier.name).toImmutableArraySeq)), + cause = cause) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/AgnosticEncoder.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/AgnosticEncoder.scala index 639b23f714149..f1f2ea34323b4 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/AgnosticEncoder.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/AgnosticEncoder.scala @@ -29,14 +29,13 @@ import org.apache.spark.unsafe.types.{CalendarInterval, VariantVal} import org.apache.spark.util.SparkClassUtils /** - * A non implementation specific encoder. This encoder containers all the information needed - * to generate an implementation specific encoder (e.g. InternalRow <=> Custom Object). + * A non implementation specific encoder. This encoder containers all the information needed to + * generate an implementation specific encoder (e.g. InternalRow <=> Custom Object). * * The input of the serialization does not need to match the external type of the encoder. This is - * called lenient serialization. An example of this is lenient date serialization, in this case both - * [[java.sql.Date]] and [[java.time.LocalDate]] are allowed. Deserialization is never lenient; it - * will always produce instance of the external type. - * + * called lenient serialization. An example of this is lenient date serialization, in this case + * both [[java.sql.Date]] and [[java.time.LocalDate]] are allowed. Deserialization is never + * lenient; it will always produce instance of the external type. */ trait AgnosticEncoder[T] extends Encoder[T] { def isPrimitive: Boolean @@ -49,14 +48,14 @@ trait AgnosticEncoder[T] extends Encoder[T] { object AgnosticEncoders { case class OptionEncoder[E](elementEncoder: AgnosticEncoder[E]) - extends AgnosticEncoder[Option[E]] { + extends AgnosticEncoder[Option[E]] { override def isPrimitive: Boolean = false override def dataType: DataType = elementEncoder.dataType override val clsTag: ClassTag[Option[E]] = ClassTag(classOf[Option[E]]) } case class ArrayEncoder[E](element: AgnosticEncoder[E], containsNull: Boolean) - extends AgnosticEncoder[Array[E]] { + extends AgnosticEncoder[Array[E]] { override def isPrimitive: Boolean = false override def dataType: DataType = ArrayType(element.dataType, containsNull) override val clsTag: ClassTag[Array[E]] = element.clsTag.wrap @@ -73,7 +72,7 @@ object AgnosticEncoders { element: AgnosticEncoder[E], containsNull: Boolean, override val lenientSerialization: Boolean) - extends AgnosticEncoder[C] { + extends AgnosticEncoder[C] { override def isPrimitive: Boolean = false override val dataType: DataType = ArrayType(element.dataType, containsNull) } @@ -83,12 +82,10 @@ object AgnosticEncoders { keyEncoder: AgnosticEncoder[K], valueEncoder: AgnosticEncoder[V], valueContainsNull: Boolean) - extends AgnosticEncoder[C] { + extends AgnosticEncoder[C] { override def isPrimitive: Boolean = false - override val dataType: DataType = MapType( - keyEncoder.dataType, - valueEncoder.dataType, - valueContainsNull) + override val dataType: DataType = + MapType(keyEncoder.dataType, valueEncoder.dataType, valueContainsNull) } case class EncoderField( @@ -114,16 +111,19 @@ object AgnosticEncoders { case class ProductEncoder[K]( override val clsTag: ClassTag[K], override val fields: Seq[EncoderField], - outerPointerGetter: Option[() => AnyRef]) extends StructEncoder[K] + outerPointerGetter: Option[() => AnyRef]) + extends StructEncoder[K] object ProductEncoder { val cachedCls = new ConcurrentHashMap[Int, Class[_]] private[sql] def tuple(encoders: Seq[AgnosticEncoder[_]]): AgnosticEncoder[_] = { - val fields = encoders.zipWithIndex.map { - case (e, id) => EncoderField(s"_${id + 1}", e, e.nullable, Metadata.empty) + val fields = encoders.zipWithIndex.map { case (e, id) => + EncoderField(s"_${id + 1}", e, e.nullable, Metadata.empty) } - val cls = cachedCls.computeIfAbsent(encoders.size, - _ => SparkClassUtils.getContextOrSparkClassLoader.loadClass(s"scala.Tuple${encoders.size}")) + val cls = cachedCls.computeIfAbsent( + encoders.size, + _ => + SparkClassUtils.getContextOrSparkClassLoader.loadClass(s"scala.Tuple${encoders.size}")) ProductEncoder[Any](ClassTag(cls), fields, None) } @@ -141,19 +141,19 @@ object AgnosticEncoders { object UnboundRowEncoder extends BaseRowEncoder { override val schema: StructType = new StructType() override val fields: Seq[EncoderField] = Seq.empty -} + } case class JavaBeanEncoder[K]( override val clsTag: ClassTag[K], override val fields: Seq[EncoderField]) - extends StructEncoder[K] + extends StructEncoder[K] // This will only work for encoding from/to Sparks' InternalRow format. // It is here for compatibility. case class UDTEncoder[E >: Null]( udt: UserDefinedType[E], udtClass: Class[_ <: UserDefinedType[_]]) - extends AgnosticEncoder[E] { + extends AgnosticEncoder[E] { override def isPrimitive: Boolean = false override def dataType: DataType = udt override def clsTag: ClassTag[E] = ClassTag(udt.userClass) @@ -164,21 +164,19 @@ object AgnosticEncoders { override def isPrimitive: Boolean = false override def dataType: DataType = StringType } - case class ScalaEnumEncoder[T, E]( - parent: Class[T], - override val clsTag: ClassTag[E]) - extends EnumEncoder[E] + case class ScalaEnumEncoder[T, E](parent: Class[T], override val clsTag: ClassTag[E]) + extends EnumEncoder[E] case class JavaEnumEncoder[E](override val clsTag: ClassTag[E]) extends EnumEncoder[E] - protected abstract class LeafEncoder[E : ClassTag](override val dataType: DataType) - extends AgnosticEncoder[E] { + protected abstract class LeafEncoder[E: ClassTag](override val dataType: DataType) + extends AgnosticEncoder[E] { override val clsTag: ClassTag[E] = classTag[E] override val isPrimitive: Boolean = clsTag.runtimeClass.isPrimitive } // Primitive encoders - abstract class PrimitiveLeafEncoder[E : ClassTag](dataType: DataType) - extends LeafEncoder[E](dataType) + abstract class PrimitiveLeafEncoder[E: ClassTag](dataType: DataType) + extends LeafEncoder[E](dataType) case object PrimitiveBooleanEncoder extends PrimitiveLeafEncoder[Boolean](BooleanType) case object PrimitiveByteEncoder extends PrimitiveLeafEncoder[Byte](ByteType) case object PrimitiveShortEncoder extends PrimitiveLeafEncoder[Short](ShortType) @@ -188,24 +186,24 @@ object AgnosticEncoders { case object PrimitiveDoubleEncoder extends PrimitiveLeafEncoder[Double](DoubleType) // Primitive wrapper encoders. - abstract class BoxedLeafEncoder[E : ClassTag, P]( + abstract class BoxedLeafEncoder[E: ClassTag, P]( dataType: DataType, val primitive: PrimitiveLeafEncoder[P]) - extends LeafEncoder[E](dataType) + extends LeafEncoder[E](dataType) case object BoxedBooleanEncoder - extends BoxedLeafEncoder[java.lang.Boolean, Boolean](BooleanType, PrimitiveBooleanEncoder) + extends BoxedLeafEncoder[java.lang.Boolean, Boolean](BooleanType, PrimitiveBooleanEncoder) case object BoxedByteEncoder - extends BoxedLeafEncoder[java.lang.Byte, Byte](ByteType, PrimitiveByteEncoder) + extends BoxedLeafEncoder[java.lang.Byte, Byte](ByteType, PrimitiveByteEncoder) case object BoxedShortEncoder - extends BoxedLeafEncoder[java.lang.Short, Short](ShortType, PrimitiveShortEncoder) + extends BoxedLeafEncoder[java.lang.Short, Short](ShortType, PrimitiveShortEncoder) case object BoxedIntEncoder - extends BoxedLeafEncoder[java.lang.Integer, Int](IntegerType, PrimitiveIntEncoder) + extends BoxedLeafEncoder[java.lang.Integer, Int](IntegerType, PrimitiveIntEncoder) case object BoxedLongEncoder - extends BoxedLeafEncoder[java.lang.Long, Long](LongType, PrimitiveLongEncoder) + extends BoxedLeafEncoder[java.lang.Long, Long](LongType, PrimitiveLongEncoder) case object BoxedFloatEncoder - extends BoxedLeafEncoder[java.lang.Float, Float](FloatType, PrimitiveFloatEncoder) + extends BoxedLeafEncoder[java.lang.Float, Float](FloatType, PrimitiveFloatEncoder) case object BoxedDoubleEncoder - extends BoxedLeafEncoder[java.lang.Double, Double](DoubleType, PrimitiveDoubleEncoder) + extends BoxedLeafEncoder[java.lang.Double, Double](DoubleType, PrimitiveDoubleEncoder) // Nullable leaf encoders case object NullEncoder extends LeafEncoder[java.lang.Void](NullType) @@ -218,19 +216,19 @@ object AgnosticEncoders { case object YearMonthIntervalEncoder extends LeafEncoder[Period](YearMonthIntervalType()) case object VariantEncoder extends LeafEncoder[VariantVal](VariantType) case class DateEncoder(override val lenientSerialization: Boolean) - extends LeafEncoder[jsql.Date](DateType) + extends LeafEncoder[jsql.Date](DateType) case class LocalDateEncoder(override val lenientSerialization: Boolean) - extends LeafEncoder[LocalDate](DateType) + extends LeafEncoder[LocalDate](DateType) case class TimestampEncoder(override val lenientSerialization: Boolean) - extends LeafEncoder[jsql.Timestamp](TimestampType) + extends LeafEncoder[jsql.Timestamp](TimestampType) case class InstantEncoder(override val lenientSerialization: Boolean) - extends LeafEncoder[Instant](TimestampType) + extends LeafEncoder[Instant](TimestampType) case object LocalDateTimeEncoder extends LeafEncoder[LocalDateTime](TimestampNTZType) case class SparkDecimalEncoder(dt: DecimalType) extends LeafEncoder[Decimal](dt) case class ScalaDecimalEncoder(dt: DecimalType) extends LeafEncoder[BigDecimal](dt) case class JavaDecimalEncoder(dt: DecimalType, override val lenientSerialization: Boolean) - extends LeafEncoder[JBigDecimal](dt) + extends LeafEncoder[JBigDecimal](dt) val STRICT_DATE_ENCODER: DateEncoder = DateEncoder(lenientSerialization = false) val STRICT_LOCAL_DATE_ENCODER: LocalDateEncoder = LocalDateEncoder(lenientSerialization = false) @@ -257,7 +255,8 @@ object AgnosticEncoders { case class TransformingEncoder[I, O]( clsTag: ClassTag[I], transformed: AgnosticEncoder[O], - codecProvider: () => Codec[_ >: I, O]) extends AgnosticEncoder[I] { + codecProvider: () => Codec[_ >: I, O]) + extends AgnosticEncoder[I] { override def isPrimitive: Boolean = transformed.isPrimitive override def dataType: DataType = transformed.dataType override def schema: StructType = transformed.schema diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/OuterScopes.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/OuterScopes.scala index 8587688956950..909556492847f 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/OuterScopes.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/OuterScopes.scala @@ -48,10 +48,9 @@ object OuterScopes { /** * Adds a new outer scope to this context that can be used when instantiating an `inner class` - * during deserialization. Inner classes are created when a case class is defined in the - * Spark REPL and registering the outer scope that this class was defined in allows us to create - * new instances on the spark executors. In normal use, users should not need to call this - * function. + * during deserialization. Inner classes are created when a case class is defined in the Spark + * REPL and registering the outer scope that this class was defined in allows us to create new + * instances on the spark executors. In normal use, users should not need to call this function. * * Warning: this function operates on the assumption that there is only ever one instance of any * given wrapper class. @@ -65,7 +64,7 @@ object OuterScopes { } /** - * Returns a function which can get the outer scope for the given inner class. By using function + * Returns a function which can get the outer scope for the given inner class. By using function * as return type, we can delay the process of getting outer pointer to execution time, which is * useful for inner class defined in REPL. */ @@ -134,8 +133,8 @@ object OuterScopes { } case _ => null } - } else { - () => outer + } else { () => + outer } } @@ -162,7 +161,7 @@ object OuterScopes { * dead entries after GC (using a [[ReferenceQueue]]). */ private[catalyst] class HashableWeakReference(v: AnyRef, queue: ReferenceQueue[AnyRef]) - extends WeakReference[AnyRef](v, queue) { + extends WeakReference[AnyRef](v, queue) { def this(v: AnyRef) = this(v, null) private[this] val hash = v.hashCode() override def hashCode(): Int = hash diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala index c507e952630f6..8b6da805a6e87 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala @@ -28,8 +28,8 @@ import org.apache.spark.sql.types._ import org.apache.spark.util.ArrayImplicits._ /** - * A factory for constructing encoders that convert external row to/from the Spark SQL - * internal binary representation. + * A factory for constructing encoders that convert external row to/from the Spark SQL internal + * binary representation. * * The following is a mapping between Spark SQL types and its allowed external types: * {{{ @@ -68,67 +68,65 @@ object RowEncoder extends DataTypeErrorsBase { encoderForDataType(schema, lenient).asInstanceOf[AgnosticEncoder[Row]] } - private[sql] def encoderForDataType( - dataType: DataType, - lenient: Boolean): AgnosticEncoder[_] = dataType match { - case NullType => NullEncoder - case BooleanType => BoxedBooleanEncoder - case ByteType => BoxedByteEncoder - case ShortType => BoxedShortEncoder - case IntegerType => BoxedIntEncoder - case LongType => BoxedLongEncoder - case FloatType => BoxedFloatEncoder - case DoubleType => BoxedDoubleEncoder - case dt: DecimalType => JavaDecimalEncoder(dt, lenientSerialization = true) - case BinaryType => BinaryEncoder - case _: StringType => StringEncoder - case TimestampType if SqlApiConf.get.datetimeJava8ApiEnabled => InstantEncoder(lenient) - case TimestampType => TimestampEncoder(lenient) - case TimestampNTZType => LocalDateTimeEncoder - case DateType if SqlApiConf.get.datetimeJava8ApiEnabled => LocalDateEncoder(lenient) - case DateType => DateEncoder(lenient) - case CalendarIntervalType => CalendarIntervalEncoder - case _: DayTimeIntervalType => DayTimeIntervalEncoder - case _: YearMonthIntervalType => YearMonthIntervalEncoder - case _: VariantType => VariantEncoder - case p: PythonUserDefinedType => - // TODO check if this works. - encoderForDataType(p.sqlType, lenient) - case udt: UserDefinedType[_] => - val annotation = udt.userClass.getAnnotation(classOf[SQLUserDefinedType]) - val udtClass: Class[_] = if (annotation != null) { - annotation.udt() - } else { - UDTRegistration.getUDTFor(udt.userClass.getName).getOrElse { - throw ExecutionErrors.userDefinedTypeNotAnnotatedAndRegisteredError(udt) + private[sql] def encoderForDataType(dataType: DataType, lenient: Boolean): AgnosticEncoder[_] = + dataType match { + case NullType => NullEncoder + case BooleanType => BoxedBooleanEncoder + case ByteType => BoxedByteEncoder + case ShortType => BoxedShortEncoder + case IntegerType => BoxedIntEncoder + case LongType => BoxedLongEncoder + case FloatType => BoxedFloatEncoder + case DoubleType => BoxedDoubleEncoder + case dt: DecimalType => JavaDecimalEncoder(dt, lenientSerialization = true) + case BinaryType => BinaryEncoder + case _: StringType => StringEncoder + case TimestampType if SqlApiConf.get.datetimeJava8ApiEnabled => InstantEncoder(lenient) + case TimestampType => TimestampEncoder(lenient) + case TimestampNTZType => LocalDateTimeEncoder + case DateType if SqlApiConf.get.datetimeJava8ApiEnabled => LocalDateEncoder(lenient) + case DateType => DateEncoder(lenient) + case CalendarIntervalType => CalendarIntervalEncoder + case _: DayTimeIntervalType => DayTimeIntervalEncoder + case _: YearMonthIntervalType => YearMonthIntervalEncoder + case _: VariantType => VariantEncoder + case p: PythonUserDefinedType => + // TODO check if this works. + encoderForDataType(p.sqlType, lenient) + case udt: UserDefinedType[_] => + val annotation = udt.userClass.getAnnotation(classOf[SQLUserDefinedType]) + val udtClass: Class[_] = if (annotation != null) { + annotation.udt() + } else { + UDTRegistration.getUDTFor(udt.userClass.getName).getOrElse { + throw ExecutionErrors.userDefinedTypeNotAnnotatedAndRegisteredError(udt) + } } - } - UDTEncoder(udt, udtClass.asInstanceOf[Class[_ <: UserDefinedType[_]]]) - case ArrayType(elementType, containsNull) => - IterableEncoder( - classTag[mutable.ArraySeq[_]], - encoderForDataType(elementType, lenient), - containsNull, - lenientSerialization = true) - case MapType(keyType, valueType, valueContainsNull) => - MapEncoder( - classTag[scala.collection.Map[_, _]], - encoderForDataType(keyType, lenient), - encoderForDataType(valueType, lenient), - valueContainsNull) - case StructType(fields) => - AgnosticRowEncoder(fields.map { field => - EncoderField( - field.name, - encoderForDataType(field.dataType, lenient), - field.nullable, - field.metadata) - }.toImmutableArraySeq) + UDTEncoder(udt, udtClass.asInstanceOf[Class[_ <: UserDefinedType[_]]]) + case ArrayType(elementType, containsNull) => + IterableEncoder( + classTag[mutable.ArraySeq[_]], + encoderForDataType(elementType, lenient), + containsNull, + lenientSerialization = true) + case MapType(keyType, valueType, valueContainsNull) => + MapEncoder( + classTag[scala.collection.Map[_, _]], + encoderForDataType(keyType, lenient), + encoderForDataType(valueType, lenient), + valueContainsNull) + case StructType(fields) => + AgnosticRowEncoder(fields.map { field => + EncoderField( + field.name, + encoderForDataType(field.dataType, lenient), + field.nullable, + field.metadata) + }.toImmutableArraySeq) - case _ => - throw new AnalysisException( - errorClass = "UNSUPPORTED_DATA_TYPE_FOR_ENCODER", - messageParameters = Map("dataType" -> toSQLType(dataType)) - ) - } + case _ => + throw new AnalysisException( + errorClass = "UNSUPPORTED_DATA_TYPE_FOR_ENCODER", + messageParameters = Map("dataType" -> toSQLType(dataType))) + } } diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/codecs.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/codecs.scala index 2f4965c35abe8..ceb615b13f99a 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/codecs.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/codecs.scala @@ -24,8 +24,10 @@ import org.apache.spark.util.{SparkClassUtils, SparkSerDeUtils} /** * Codec for doing conversions between two representations. * - * @tparam I input type (typically the external representation of the data. - * @tparam O output type (typically the internal representation of the data. + * @tparam I + * input type (typically the external representation of the data. + * @tparam O + * output type (typically the internal representation of the data. */ trait Codec[I, O] { def encode(in: I): O diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/expressions/OrderUtils.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/expressions/OrderUtils.scala index 385e0f00695a3..76442accbd35b 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/expressions/OrderUtils.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/expressions/OrderUtils.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.types.{ArrayType, AtomicType, DataType, NullType, StructType, UserDefinedType, VariantType} object OrderUtils { + /** * Returns true iff the data type can be ordered (i.e. can be sorted). */ diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala index 7f21ab25ad4e5..6977d9f3185ac 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala @@ -21,11 +21,12 @@ import org.apache.spark.sql.types.StructType import org.apache.spark.util.ArrayImplicits._ /** - * A row implementation that uses an array of objects as the underlying storage. Note that, while + * A row implementation that uses an array of objects as the underlying storage. Note that, while * the array is not copied, and thus could technically be mutated after creation, this is not * allowed. */ class GenericRow(protected[sql] val values: Array[Any]) extends Row { + /** No-arg constructor for serialization. */ protected def this() = this(null) @@ -41,7 +42,7 @@ class GenericRow(protected[sql] val values: Array[Any]) extends Row { } class GenericRowWithSchema(values: Array[Any], override val schema: StructType) - extends GenericRow(values) { + extends GenericRow(values) { /** No-arg constructor for serialization. */ protected def this() = this(null, null) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/DataTypeAstBuilder.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/DataTypeAstBuilder.scala index 38ecd29266db7..46fb4a3397c59 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/DataTypeAstBuilder.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/DataTypeAstBuilder.scala @@ -120,7 +120,7 @@ class DataTypeAstBuilder extends SqlBaseParserBaseVisitor[AnyRef] { override def visitDayTimeIntervalDataType(ctx: DayTimeIntervalDataTypeContext): DataType = { val startStr = ctx.from.getText.toLowerCase(Locale.ROOT) val start = DayTimeIntervalType.stringToField(startStr) - if (ctx.to != null ) { + if (ctx.to != null) { val endStr = ctx.to.getText.toLowerCase(Locale.ROOT) val end = DayTimeIntervalType.stringToField(endStr) if (end <= start) { diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/DataTypeParserInterface.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/DataTypeParserInterface.scala index ab665f360b0a6..1a1a9b01de3b1 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/DataTypeParserInterface.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/DataTypeParserInterface.scala @@ -22,9 +22,10 @@ import org.apache.spark.sql.types.{DataType, StructType} * Interface for [[DataType]] parsing functionality. */ trait DataTypeParserInterface { + /** - * Parse a string to a [[StructType]]. The passed SQL string should be a comma separated list - * of field definitions which will preserve the correct Hive metadata. + * Parse a string to a [[StructType]]. The passed SQL string should be a comma separated list of + * field definitions which will preserve the correct Hive metadata. */ @throws[ParseException]("Text cannot be parsed to a schema") def parseTableSchema(sqlText: String): StructType diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/LegacyTypeStringParser.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/LegacyTypeStringParser.scala index 8ac5939bca944..7d1986e727f79 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/LegacyTypeStringParser.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/LegacyTypeStringParser.scala @@ -23,35 +23,36 @@ import org.apache.spark.sql.errors.DataTypeErrors import org.apache.spark.sql.types._ /** - * Parser that turns case class strings into datatypes. This is only here to maintain compatibility - * with Parquet files written by Spark 1.1 and below. + * Parser that turns case class strings into datatypes. This is only here to maintain + * compatibility with Parquet files written by Spark 1.1 and below. */ object LegacyTypeStringParser extends RegexParsers { protected lazy val primitiveType: Parser[DataType] = - ( "StringType" ^^^ StringType - | "FloatType" ^^^ FloatType - | "IntegerType" ^^^ IntegerType - | "ByteType" ^^^ ByteType - | "ShortType" ^^^ ShortType - | "DoubleType" ^^^ DoubleType - | "LongType" ^^^ LongType - | "BinaryType" ^^^ BinaryType - | "BooleanType" ^^^ BooleanType - | "DateType" ^^^ DateType - | "DecimalType()" ^^^ DecimalType.USER_DEFAULT - | fixedDecimalType - | "TimestampType" ^^^ TimestampType - ) + ( + "StringType" ^^^ StringType + | "FloatType" ^^^ FloatType + | "IntegerType" ^^^ IntegerType + | "ByteType" ^^^ ByteType + | "ShortType" ^^^ ShortType + | "DoubleType" ^^^ DoubleType + | "LongType" ^^^ LongType + | "BinaryType" ^^^ BinaryType + | "BooleanType" ^^^ BooleanType + | "DateType" ^^^ DateType + | "DecimalType()" ^^^ DecimalType.USER_DEFAULT + | fixedDecimalType + | "TimestampType" ^^^ TimestampType + ) protected lazy val fixedDecimalType: Parser[DataType] = - ("DecimalType(" ~> "[0-9]+".r) ~ ("," ~> "[0-9]+".r <~ ")") ^^ { - case precision ~ scale => DecimalType(precision.toInt, scale.toInt) + ("DecimalType(" ~> "[0-9]+".r) ~ ("," ~> "[0-9]+".r <~ ")") ^^ { case precision ~ scale => + DecimalType(precision.toInt, scale.toInt) } protected lazy val arrayType: Parser[DataType] = - "ArrayType" ~> "(" ~> dataType ~ "," ~ boolVal <~ ")" ^^ { - case tpe ~ _ ~ containsNull => ArrayType(tpe, containsNull) + "ArrayType" ~> "(" ~> dataType ~ "," ~ boolVal <~ ")" ^^ { case tpe ~ _ ~ containsNull => + ArrayType(tpe, containsNull) } protected lazy val mapType: Parser[DataType] = @@ -66,21 +67,23 @@ object LegacyTypeStringParser extends RegexParsers { } protected lazy val boolVal: Parser[Boolean] = - ( "true" ^^^ true - | "false" ^^^ false - ) + ( + "true" ^^^ true + | "false" ^^^ false + ) protected lazy val structType: Parser[DataType] = - "StructType\\([A-zA-z]*\\(".r ~> repsep(structField, ",") <~ "))" ^^ { - case fields => StructType(fields) + "StructType\\([A-zA-z]*\\(".r ~> repsep(structField, ",") <~ "))" ^^ { case fields => + StructType(fields) } protected lazy val dataType: Parser[DataType] = - ( arrayType - | mapType - | structType - | primitiveType - ) + ( + arrayType + | mapType + | structType + | primitiveType + ) /** * Parses a string representation of a DataType. diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/SparkParserErrorStrategy.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/SparkParserErrorStrategy.scala index 99e63d783838f..461d79ec22cf0 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/SparkParserErrorStrategy.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/SparkParserErrorStrategy.scala @@ -32,7 +32,7 @@ class SparkRecognitionException( ctx: ParserRuleContext, val errorClass: Option[String] = None, val messageParameters: Map[String, String] = Map.empty) - extends RecognitionException(message, recognizer, input, ctx) { + extends RecognitionException(message, recognizer, input, ctx) { /** Construct from a given [[RecognitionException]], with additional error information. */ def this( @@ -50,7 +50,7 @@ class SparkRecognitionException( Some(errorClass), messageParameters) - /** Construct with pure errorClass and messageParameter information. */ + /** Construct with pure errorClass and messageParameter information. */ def this(errorClass: String, messageParameters: Map[String, String]) = this("", null, null, null, Some(errorClass), messageParameters) } @@ -59,12 +59,12 @@ class SparkRecognitionException( * A [[SparkParserErrorStrategy]] extends the [[DefaultErrorStrategy]], that does special handling * on errors. * - * The intention of this class is to provide more information of these errors encountered in - * ANTLR parser to the downstream consumers, to be able to apply the [[SparkThrowable]] error - * message framework to these exceptions. + * The intention of this class is to provide more information of these errors encountered in ANTLR + * parser to the downstream consumers, to be able to apply the [[SparkThrowable]] error message + * framework to these exceptions. */ class SparkParserErrorStrategy() extends DefaultErrorStrategy { - private val userWordDict : Map[String, String] = Map("''" -> "end of input") + private val userWordDict: Map[String, String] = Map("''" -> "end of input") /** Get the user-facing display of the error token. */ override def getTokenErrorDisplay(t: Token): String = { @@ -76,9 +76,7 @@ class SparkParserErrorStrategy() extends DefaultErrorStrategy { val exceptionWithErrorClass = new SparkRecognitionException( e, "PARSE_SYNTAX_ERROR", - messageParameters = Map( - "error" -> getTokenErrorDisplay(e.getOffendingToken), - "hint" -> "")) + messageParameters = Map("error" -> getTokenErrorDisplay(e.getOffendingToken), "hint" -> "")) recognizer.notifyErrorListeners(e.getOffendingToken, "", exceptionWithErrorClass) } @@ -116,18 +114,17 @@ class SparkParserErrorStrategy() extends DefaultErrorStrategy { /** * Inspired by [[org.antlr.v4.runtime.BailErrorStrategy]], which is used in two-stage parsing: - * This error strategy allows the first stage of two-stage parsing to immediately terminate - * if an error is encountered, and immediately fall back to the second stage. In addition to - * avoiding wasted work by attempting to recover from errors here, the empty implementation - * of sync improves the performance of the first stage. + * This error strategy allows the first stage of two-stage parsing to immediately terminate if an + * error is encountered, and immediately fall back to the second stage. In addition to avoiding + * wasted work by attempting to recover from errors here, the empty implementation of sync + * improves the performance of the first stage. */ class SparkParserBailErrorStrategy() extends SparkParserErrorStrategy { /** - * Instead of recovering from exception e, re-throw it wrapped - * in a [[ParseCancellationException]] so it is not caught by the - * rule function catches. Use [[Exception#getCause]] to get the - * original [[RecognitionException]]. + * Instead of recovering from exception e, re-throw it wrapped in a + * [[ParseCancellationException]] so it is not caught by the rule function catches. Use + * [[Exception#getCause]] to get the original [[RecognitionException]]. */ override def recover(recognizer: Parser, e: RecognitionException): Unit = { var context = recognizer.getContext @@ -139,8 +136,8 @@ class SparkParserBailErrorStrategy() extends SparkParserErrorStrategy { } /** - * Make sure we don't attempt to recover inline; if the parser - * successfully recovers, it won't throw an exception. + * Make sure we don't attempt to recover inline; if the parser successfully recovers, it won't + * throw an exception. */ @throws[RecognitionException] override def recoverInline(recognizer: Parser): Token = { diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/parsers.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/parsers.scala index 0b9e6ea021be1..10da24567545b 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/parsers.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/parsers.scala @@ -36,6 +36,7 @@ import org.apache.spark.sql.types.{DataType, StructType} * Base SQL parsing infrastructure. */ abstract class AbstractParser extends DataTypeParserInterface with Logging { + /** Creates/Resolves DataType for a given SQL string. */ override def parseDataType(sqlText: String): DataType = parse(sqlText) { parser => astBuilder.visitSingleDataType(parser.singleDataType()) @@ -78,8 +79,7 @@ abstract class AbstractParser extends DataTypeParserInterface with Logging { parser.setErrorHandler(new SparkParserBailErrorStrategy()) parser.getInterpreter.setPredictionMode(PredictionMode.SLL) toResult(parser) - } - catch { + } catch { case e: ParseCancellationException => // if we fail, parse with LL mode w/ SparkParserErrorStrategy tokenStream.seek(0) // rewind input stream @@ -90,8 +90,7 @@ abstract class AbstractParser extends DataTypeParserInterface with Logging { parser.getInterpreter.setPredictionMode(PredictionMode.LL) toResult(parser) } - } - catch { + } catch { case e: ParseException if e.command.isDefined => throw e case e: ParseException => @@ -187,7 +186,7 @@ case object ParseErrorListener extends BaseErrorListener { * A [[ParseException]] is an [[SparkException]] that is thrown during the parse process. It * contains fields and an extended error message that make reporting and diagnosing errors easier. */ -class ParseException private( +class ParseException private ( val command: Option[String], message: String, val start: Origin, @@ -195,17 +194,18 @@ class ParseException private( errorClass: Option[String] = None, messageParameters: Map[String, String] = Map.empty, queryContext: Array[QueryContext] = ParseException.getQueryContext()) - extends AnalysisException( - message, - start.line, - start.startPosition, - None, - errorClass, - messageParameters, - queryContext) { + extends AnalysisException( + message, + start.line, + start.startPosition, + None, + errorClass, + messageParameters, + queryContext) { def this(errorClass: String, messageParameters: Map[String, String], ctx: ParserRuleContext) = - this(Option(SparkParserUtils.command(ctx)), + this( + Option(SparkParserUtils.command(ctx)), SparkThrowableHelper.getMessage(errorClass, messageParameters), SparkParserUtils.position(ctx.getStart), SparkParserUtils.position(ctx.getStop), @@ -310,14 +310,16 @@ case object PostProcessor extends SqlBaseParserBaseListener { throw QueryParsingErrors.invalidIdentifierError(ident, ctx) } - /** Throws error message when unquoted identifier contains characters outside a-z, A-Z, 0-9, _ */ + /** + * Throws error message when unquoted identifier contains characters outside a-z, A-Z, 0-9, _ + */ override def exitUnquotedIdentifier(ctx: SqlBaseParser.UnquotedIdentifierContext): Unit = { val ident = ctx.getText if (ident.exists(c => - !(c >= 'a' && c <= 'z') && - !(c >= 'A' && c <= 'Z') && - !(c >= '0' && c <= '9') && - c != '_')) { + !(c >= 'a' && c <= 'z') && + !(c >= 'A' && c <= 'Z') && + !(c >= '0' && c <= '9') && + c != '_')) { throw QueryParsingErrors.invalidIdentifierError(ident, ctx) } } @@ -353,9 +355,7 @@ case object PostProcessor extends SqlBaseParserBaseListener { replaceTokenByIdentifier(ctx, 0)(identity) } - private def replaceTokenByIdentifier( - ctx: ParserRuleContext, - stripMargins: Int)( + private def replaceTokenByIdentifier(ctx: ParserRuleContext, stripMargins: Int)( f: CommonToken => CommonToken = identity): Unit = { val parent = ctx.getParent parent.removeLastChild() @@ -373,8 +373,8 @@ case object PostProcessor extends SqlBaseParserBaseListener { /** * The post-processor checks the unclosed bracketed comment. */ -case class UnclosedCommentProcessor( - command: String, tokenStream: CommonTokenStream) extends SqlBaseParserBaseListener { +case class UnclosedCommentProcessor(command: String, tokenStream: CommonTokenStream) + extends SqlBaseParserBaseListener { override def exitSingleDataType(ctx: SqlBaseParser.SingleDataTypeContext): Unit = { checkUnclosedComment(tokenStream, command) @@ -384,7 +384,8 @@ case class UnclosedCommentProcessor( checkUnclosedComment(tokenStream, command) } - override def exitSingleTableIdentifier(ctx: SqlBaseParser.SingleTableIdentifierContext): Unit = { + override def exitSingleTableIdentifier( + ctx: SqlBaseParser.SingleTableIdentifierContext): Unit = { checkUnclosedComment(tokenStream, command) } @@ -422,7 +423,8 @@ case class UnclosedCommentProcessor( // The last token is 'EOF' and the penultimate is unclosed bracketed comment val failedToken = tokenStream.get(tokenStream.size() - 2) assert(failedToken.getType() == SqlBaseParser.BRACKETED_COMMENT) - val position = Origin(Option(failedToken.getLine), Option(failedToken.getCharPositionInLine)) + val position = + Origin(Option(failedToken.getLine), Option(failedToken.getCharPositionInLine)) throw QueryParsingErrors.unclosedBracketedCommentError( command = command, start = Origin(Option(failedToken.getStartIndex)), diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TimeMode.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TimeMode.scala index 4c7c87504ffc4..e870a83ec4ae6 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TimeMode.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TimeMode.scala @@ -37,9 +37,10 @@ object TimeModes { ProcessingTime case "eventtime" => EventTime - case _ => throw new SparkIllegalArgumentException( - errorClass = "STATEFUL_PROCESSOR_UNKNOWN_TIME_MODE", - messageParameters = Map("timeMode" -> timeMode)) + case _ => + throw new SparkIllegalArgumentException( + errorClass = "STATEFUL_PROCESSOR_UNKNOWN_TIME_MODE", + messageParameters = Map("timeMode" -> timeMode)) } } } diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/streaming/InternalOutputModes.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/streaming/InternalOutputModes.scala index 0ccfc3cbc7bf2..8c54758ebc0bc 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/streaming/InternalOutputModes.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/streaming/InternalOutputModes.scala @@ -28,27 +28,26 @@ import org.apache.spark.sql.streaming.OutputMode private[sql] object InternalOutputModes { /** - * OutputMode in which only the new rows in the streaming DataFrame/Dataset will be - * written to the sink. This output mode can be only be used in queries that do not - * contain any aggregation. + * OutputMode in which only the new rows in the streaming DataFrame/Dataset will be written to + * the sink. This output mode can be only be used in queries that do not contain any + * aggregation. */ case object Append extends OutputMode /** - * OutputMode in which all the rows in the streaming DataFrame/Dataset will be written - * to the sink every time these is some updates. This output mode can only be used in queries - * that contain aggregations. + * OutputMode in which all the rows in the streaming DataFrame/Dataset will be written to the + * sink every time these is some updates. This output mode can only be used in queries that + * contain aggregations. */ case object Complete extends OutputMode /** - * OutputMode in which only the rows in the streaming DataFrame/Dataset that were updated will be - * written to the sink every time these is some updates. If the query doesn't contain + * OutputMode in which only the rows in the streaming DataFrame/Dataset that were updated will + * be written to the sink every time these is some updates. If the query doesn't contain * aggregations, it will be equivalent to `Append` mode. */ case object Update extends OutputMode - def apply(outputMode: String): OutputMode = { outputMode.toLowerCase(Locale.ROOT) match { case "append" => @@ -57,7 +56,8 @@ private[sql] object InternalOutputModes { OutputMode.Complete case "update" => OutputMode.Update - case _ => throw new SparkIllegalArgumentException( + case _ => + throw new SparkIllegalArgumentException( errorClass = "STREAMING_OUTPUT_MODE.INVALID", messageParameters = Map("outputMode" -> outputMode)) } diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/trees/QueryContexts.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/trees/QueryContexts.scala index 2b3f4674539e3..d5be65a2f36cf 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/trees/QueryContexts.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/trees/QueryContexts.scala @@ -27,7 +27,8 @@ case class SQLQueryContext( originStopIndex: Option[Int], sqlText: Option[String], originObjectType: Option[String], - originObjectName: Option[String]) extends QueryContext { + originObjectName: Option[String]) + extends QueryContext { override val contextType = QueryContextType.SQL override val objectType = originObjectType.getOrElse("") @@ -37,9 +38,8 @@ case class SQLQueryContext( /** * The SQL query context of current node. For example: - * == SQL of VIEW v1 (line 1, position 25) == - * SELECT '' AS five, i.f1, i.f1 - int('2') AS x FROM INT4_TBL i - * ^^^^^^^^^^^^^^^ + * ==SQL of VIEW v1 (line 1, position 25)== + * SELECT '' AS five, i.f1, i.f1 - int('2') AS x FROM INT4_TBL i ^^^^^^^^^^^^^^^ */ override lazy val summary: String = { // If the query context is missing or incorrect, simply return an empty string. @@ -127,8 +127,8 @@ case class SQLQueryContext( def isValid: Boolean = { sqlText.isDefined && originStartIndex.isDefined && originStopIndex.isDefined && - originStartIndex.get >= 0 && originStopIndex.get < sqlText.get.length && - originStartIndex.get <= originStopIndex.get + originStartIndex.get >= 0 && originStopIndex.get < sqlText.get.length && + originStartIndex.get <= originStopIndex.get } override def callSite: String = throw SparkUnsupportedOperationException() @@ -136,7 +136,8 @@ case class SQLQueryContext( case class DataFrameQueryContext( stackTrace: Seq[StackTraceElement], - pysparkErrorContext: Option[(String, String)]) extends QueryContext { + pysparkErrorContext: Option[(String, String)]) + extends QueryContext { override val contextType = QueryContextType.DataFrame override def objectType: String = throw SparkUnsupportedOperationException() @@ -146,19 +147,21 @@ case class DataFrameQueryContext( override val fragment: String = { pysparkErrorContext.map(_._1).getOrElse { - stackTrace.headOption.map { firstElem => - val methodName = firstElem.getMethodName - if (methodName.length > 1 && methodName(0) == '$') { - methodName.substring(1) - } else { - methodName + stackTrace.headOption + .map { firstElem => + val methodName = firstElem.getMethodName + if (methodName.length > 1 && methodName(0) == '$') { + methodName.substring(1) + } else { + methodName + } } - }.getOrElse("") + .getOrElse("") } } - override val callSite: String = pysparkErrorContext.map( - _._2).getOrElse(stackTrace.tail.mkString("\n")) + override val callSite: String = + pysparkErrorContext.map(_._2).getOrElse(stackTrace.tail.mkString("\n")) override lazy val summary: String = { val builder = new StringBuilder diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/trees/origin.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/trees/origin.scala index 3e0ebdd627b63..33fa17433abbd 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/trees/origin.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/trees/origin.scala @@ -23,9 +23,9 @@ import org.apache.spark.sql.internal.SqlApiConf import org.apache.spark.util.ArrayImplicits._ /** - * Contexts of TreeNodes, including location, SQL text, object type and object name. - * The only supported object type is "VIEW" now. In the future, we may support SQL UDF or other - * objects which contain SQL text. + * Contexts of TreeNodes, including location, SQL text, object type and object name. The only + * supported object type is "VIEW" now. In the future, we may support SQL UDF or other objects + * which contain SQL text. */ case class Origin( line: Option[Int] = None, @@ -41,8 +41,7 @@ case class Origin( lazy val context: QueryContext = if (stackTrace.isDefined) { DataFrameQueryContext(stackTrace.get.toImmutableArraySeq, pysparkErrorContext) } else { - SQLQueryContext( - line, startPosition, startIndex, stopIndex, sqlText, objectType, objectName) + SQLQueryContext(line, startPosition, startIndex, stopIndex, sqlText, objectType, objectName) } def getQueryContext: Array[QueryContext] = { @@ -61,7 +60,7 @@ trait WithOrigin { } /** - * Provides a location for TreeNodes to ask about the context of their origin. For example, which + * Provides a location for TreeNodes to ask about the context of their origin. For example, which * line of code is currently being parsed. */ object CurrentOrigin { @@ -75,8 +74,7 @@ object CurrentOrigin { def reset(): Unit = value.set(Origin()) def setPosition(line: Int, start: Int): Unit = { - value.set( - value.get.copy(line = Some(line), startPosition = Some(start))) + value.set(value.get.copy(line = Some(line), startPosition = Some(start))) } def withOrigin[A](o: Origin)(f: => A): A = { @@ -84,25 +82,29 @@ object CurrentOrigin { // this way withOrigin can be recursive val previous = get set(o) - val ret = try f finally { set(previous) } + val ret = + try f + finally { set(previous) } ret } /** - * This helper function captures the Spark API and its call site in the user code from the current - * stacktrace. + * This helper function captures the Spark API and its call site in the user code from the + * current stacktrace. * * As adding `withOrigin` explicitly to all Spark API definition would be a huge change, * `withOrigin` is used only at certain places where all API implementation surely pass through - * and the current stacktrace is filtered to the point where first Spark API code is invoked from - * the user code. + * and the current stacktrace is filtered to the point where first Spark API code is invoked + * from the user code. * * As there might be multiple nested `withOrigin` calls (e.g. any Spark API implementations can * invoke other APIs) only the first `withOrigin` is captured because that is closer to the user * code. * - * @param f The function that can use the origin. - * @return The result of `f`. + * @param f + * The function that can use the origin. + * @return + * The result of `f`. */ private[sql] def withOrigin[T](f: => T): T = { if (CurrentOrigin.get.stackTrace.isDefined) { @@ -114,9 +116,9 @@ object CurrentOrigin { while (i < st.length && !sparkCode(st(i))) i += 1 // Stop at the end of the first Spark code traces while (i < st.length && sparkCode(st(i))) i += 1 - val origin = Origin(stackTrace = Some(st.slice( - from = i - 1, - until = i + SqlApiConf.get.stackTracesInDataFrameContext)), + val origin = Origin( + stackTrace = + Some(st.slice(from = i - 1, until = i + SqlApiConf.get.stackTracesInDataFrameContext)), pysparkErrorContext = PySparkCurrentOrigin.get()) CurrentOrigin.withOrigin(origin)(f) } diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/AttributeNameParser.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/AttributeNameParser.scala index e47ab1978d0ed..533b09e82df13 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/AttributeNameParser.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/AttributeNameParser.scala @@ -19,11 +19,11 @@ package org.apache.spark.sql.catalyst.util import org.apache.spark.sql.errors.DataTypeErrors trait AttributeNameParser { + /** - * Used to split attribute name by dot with backticks rule. - * Backticks must appear in pairs, and the quoted string must be a complete name part, - * which means `ab..c`e.f is not allowed. - * We can use backtick only inside quoted name parts. + * Used to split attribute name by dot with backticks rule. Backticks must appear in pairs, and + * the quoted string must be a complete name part, which means `ab..c`e.f is not allowed. We can + * use backtick only inside quoted name parts. */ def parseAttributeName(name: String): Seq[String] = { def e = DataTypeErrors.attributeNameSyntaxError(name) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala index 640304efce4b4..9e90feeb782d6 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala @@ -23,12 +23,12 @@ import java.util.Locale * Builds a map in which keys are case insensitive. Input map can be accessed for cases where * case-sensitive information is required. The primary constructor is marked private to avoid * nested case-insensitive map creation, otherwise the keys in the original map will become - * case-insensitive in this scenario. - * Note: CaseInsensitiveMap is serializable. However, after transformation, e.g. `filterKeys()`, - * it may become not serializable. + * case-insensitive in this scenario. Note: CaseInsensitiveMap is serializable. However, after + * transformation, e.g. `filterKeys()`, it may become not serializable. */ -class CaseInsensitiveMap[T] private (val originalMap: Map[String, T]) extends Map[String, T] - with Serializable { +class CaseInsensitiveMap[T] private (val originalMap: Map[String, T]) + extends Map[String, T] + with Serializable { val keyLowerCasedMap = originalMap.map(kv => kv.copy(_1 = kv._1.toLowerCase(Locale.ROOT))) @@ -62,4 +62,3 @@ object CaseInsensitiveMap { case _ => new CaseInsensitiveMap(params) } } - diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/DataTypeJsonUtils.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/DataTypeJsonUtils.scala index e75429c58cc7b..b8ab633b2047f 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/DataTypeJsonUtils.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/DataTypeJsonUtils.scala @@ -25,15 +25,16 @@ import org.json4s.jackson.{JValueDeserializer, JValueSerializer} import org.apache.spark.sql.types.DataType object DataTypeJsonUtils { + /** * Jackson serializer for [[DataType]]. Internally this delegates to json4s based serialization. */ class DataTypeJsonSerializer extends JsonSerializer[DataType] { private val delegate = new JValueSerializer override def serialize( - value: DataType, - gen: JsonGenerator, - provider: SerializerProvider): Unit = { + value: DataType, + gen: JsonGenerator, + provider: SerializerProvider): Unit = { delegate.serialize(value.jsonValue, gen, provider) } } @@ -46,8 +47,8 @@ object DataTypeJsonUtils { private val delegate = new JValueDeserializer(classOf[Any]) override def deserialize( - jsonParser: JsonParser, - deserializationContext: DeserializationContext): DataType = { + jsonParser: JsonParser, + deserializationContext: DeserializationContext): DataType = { val json = delegate.deserialize(jsonParser, deserializationContext) DataType.parseDataType(json.asInstanceOf[JValue]) } diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala index 34d19bb67b71a..5eada9a7be670 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala @@ -43,7 +43,8 @@ class Iso8601DateFormatter( locale: Locale, legacyFormat: LegacyDateFormats.LegacyDateFormat, isParsing: Boolean) - extends DateFormatter with DateTimeFormatterHelper { + extends DateFormatter + with DateTimeFormatterHelper { @transient private lazy val formatter = getOrCreateFormatter(pattern, locale, isParsing) @@ -62,8 +63,7 @@ class Iso8601DateFormatter( override def format(localDate: LocalDate): String = { try { localDate.format(formatter) - } catch checkFormattedDiff(toJavaDate(localDateToDays(localDate)), - (d: Date) => format(d)) + } catch checkFormattedDiff(toJavaDate(localDateToDays(localDate)), (d: Date) => format(d)) } override def format(days: Int): String = { @@ -83,19 +83,22 @@ class Iso8601DateFormatter( } /** - * The formatter for dates which doesn't require users to specify a pattern. While formatting, - * it uses the default pattern [[DateFormatter.defaultPattern]]. In parsing, it follows the CAST + * The formatter for dates which doesn't require users to specify a pattern. While formatting, it + * uses the default pattern [[DateFormatter.defaultPattern]]. In parsing, it follows the CAST * logic in conversion of strings to Catalyst's DateType. * - * @param locale The locale overrides the system locale and is used in formatting. - * @param legacyFormat Defines the formatter used for legacy dates. - * @param isParsing Whether the formatter is used for parsing (`true`) or for formatting (`false`). + * @param locale + * The locale overrides the system locale and is used in formatting. + * @param legacyFormat + * Defines the formatter used for legacy dates. + * @param isParsing + * Whether the formatter is used for parsing (`true`) or for formatting (`false`). */ class DefaultDateFormatter( locale: Locale, legacyFormat: LegacyDateFormats.LegacyDateFormat, isParsing: Boolean) - extends Iso8601DateFormatter(DateFormatter.defaultPattern, locale, legacyFormat, isParsing) { + extends Iso8601DateFormatter(DateFormatter.defaultPattern, locale, legacyFormat, isParsing) { override def parse(s: String): Int = { try { @@ -125,11 +128,13 @@ trait LegacyDateFormatter extends DateFormatter { * JVM time zone intentionally for compatibility with Spark 2.4 and earlier versions. * * Note: Using of the default JVM time zone makes the formatter compatible with the legacy - * `SparkDateTimeUtils` methods `toJavaDate` and `fromJavaDate` that are based on the default - * JVM time zone too. + * `SparkDateTimeUtils` methods `toJavaDate` and `fromJavaDate` that are based on the default JVM + * time zone too. * - * @param pattern `java.text.SimpleDateFormat` compatible pattern. - * @param locale The locale overrides the system locale and is used in parsing/formatting. + * @param pattern + * `java.text.SimpleDateFormat` compatible pattern. + * @param locale + * The locale overrides the system locale and is used in parsing/formatting. */ class LegacyFastDateFormatter(pattern: String, locale: Locale) extends LegacyDateFormatter { @transient @@ -145,14 +150,16 @@ class LegacyFastDateFormatter(pattern: String, locale: Locale) extends LegacyDat * JVM time zone intentionally for compatibility with Spark 2.4 and earlier versions. * * Note: Using of the default JVM time zone makes the formatter compatible with the legacy - * `SparkDateTimeUtils` methods `toJavaDate` and `fromJavaDate` that are based on the default - * JVM time zone too. + * `SparkDateTimeUtils` methods `toJavaDate` and `fromJavaDate` that are based on the default JVM + * time zone too. * - * @param pattern The pattern describing the date and time format. - * See - * Date and Time Patterns - * @param locale The locale whose date format symbols should be used. It overrides the system - * locale in parsing/formatting. + * @param pattern + * The pattern describing the date and time format. See Date and + * Time Patterns + * @param locale + * The locale whose date format symbols should be used. It overrides the system locale in + * parsing/formatting. */ // scalastyle:on line.size.limit class LegacySimpleDateFormatter(pattern: String, locale: Locale) extends LegacyDateFormatter { diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala index 067e58893126c..343468dc0270c 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala @@ -40,13 +40,18 @@ trait DateTimeFormatterHelper { } private def verifyLocalDate( - accessor: TemporalAccessor, field: ChronoField, candidate: LocalDate): Unit = { + accessor: TemporalAccessor, + field: ChronoField, + candidate: LocalDate): Unit = { if (accessor.isSupported(field)) { val actual = accessor.get(field) val expected = candidate.get(field) if (actual != expected) { throw ExecutionErrors.fieldDiffersFromDerivedLocalDateError( - field, actual, expected, candidate) + field, + actual, + expected, + candidate) } } } @@ -133,7 +138,8 @@ trait DateTimeFormatterHelper { // SparkUpgradeException. On the contrary, if the legacy policy set to CORRECTED, // DateTimeParseException will address by the caller side. protected def checkParsedDiff[T]( - s: String, legacyParseFunc: String => T): PartialFunction[Throwable, T] = { + s: String, + legacyParseFunc: String => T): PartialFunction[Throwable, T] = { case e if needConvertToSparkUpgradeException(e) => try { legacyParseFunc(s) @@ -151,11 +157,12 @@ trait DateTimeFormatterHelper { d: T, legacyFormatFunc: T => String): PartialFunction[Throwable, String] = { case e if needConvertToSparkUpgradeException(e) => - val resultCandidate = try { - legacyFormatFunc(d) - } catch { - case _: Throwable => throw e - } + val resultCandidate = + try { + legacyFormatFunc(d) + } catch { + case _: Throwable => throw e + } throw ExecutionErrors.failToParseDateTimeInNewParserError(resultCandidate, e) } @@ -166,9 +173,11 @@ trait DateTimeFormatterHelper { * policy or follow our guide to correct their pattern. Otherwise, the original * IllegalArgumentException will be thrown. * - * @param pattern the date time pattern - * @param tryLegacyFormatter a func to capture exception, identically which forces a legacy - * datetime formatter to be initialized + * @param pattern + * the date time pattern + * @param tryLegacyFormatter + * a func to capture exception, identically which forces a legacy datetime formatter to be + * initialized */ protected def checkLegacyFormatter( pattern: String, @@ -214,8 +223,7 @@ private object DateTimeFormatterHelper { /** * Building a formatter for parsing seconds fraction with variable length */ - def createBuilderWithVarLengthSecondFraction( - pattern: String): DateTimeFormatterBuilder = { + def createBuilderWithVarLengthSecondFraction(pattern: String): DateTimeFormatterBuilder = { val builder = createBuilder() pattern.split("'").zipWithIndex.foreach { // Split string starting with the regex itself which is `'` here will produce an extra empty @@ -229,12 +237,14 @@ private object DateTimeFormatterHelper { case extractor(prefix, secondFraction, suffix) => builder.appendPattern(prefix) if (secondFraction.nonEmpty) { - builder.appendFraction(ChronoField.NANO_OF_SECOND, 1, secondFraction.length, false) + builder + .appendFraction(ChronoField.NANO_OF_SECOND, 1, secondFraction.length, false) } rest = suffix - case _ => throw new SparkIllegalArgumentException( - errorClass = "INVALID_DATETIME_PATTERN", - messageParameters = Map("pattern" -> pattern)) + case _ => + throw new SparkIllegalArgumentException( + errorClass = "INVALID_DATETIME_PATTERN", + messageParameters = Map("pattern" -> pattern)) } } case (patternPart, _) => builder.appendLiteral(patternPart) @@ -258,8 +268,10 @@ private object DateTimeFormatterHelper { val builder = createBuilder() .append(DateTimeFormatter.ISO_LOCAL_DATE) .appendLiteral(' ') - .appendValue(ChronoField.HOUR_OF_DAY, 2).appendLiteral(':') - .appendValue(ChronoField.MINUTE_OF_HOUR, 2).appendLiteral(':') + .appendValue(ChronoField.HOUR_OF_DAY, 2) + .appendLiteral(':') + .appendValue(ChronoField.MINUTE_OF_HOUR, 2) + .appendLiteral(':') .appendValue(ChronoField.SECOND_OF_MINUTE, 2) .appendFraction(ChronoField.NANO_OF_SECOND, 0, 9, true) toFormatter(builder, TimestampFormatter.defaultLocale) @@ -299,17 +311,21 @@ private object DateTimeFormatterHelper { * parsing/formatting datetime values. The pattern string is incompatible with the one defined * by SimpleDateFormat in Spark 2.4 and earlier. This function converts all incompatible pattern * for the new parser in Spark 3.0. See more details in SPARK-31030. - * @param pattern The input pattern. - * @return The pattern for new parser + * @param pattern + * The input pattern. + * @return + * The pattern for new parser */ def convertIncompatiblePattern(pattern: String, isParsing: Boolean): String = { - val eraDesignatorContained = pattern.split("'").zipWithIndex.exists { - case (patternPart, index) => + val eraDesignatorContained = + pattern.split("'").zipWithIndex.exists { case (patternPart, index) => // Text can be quoted using single quotes, we only check the non-quote parts. index % 2 == 0 && patternPart.contains("G") - } - (pattern + " ").split("'").zipWithIndex.map { - case (patternPart, index) => + } + (pattern + " ") + .split("'") + .zipWithIndex + .map { case (patternPart, index) => if (index % 2 == 0) { for (c <- patternPart if weekBasedLetters.contains(c)) { throw new SparkIllegalArgumentException( @@ -317,12 +333,10 @@ private object DateTimeFormatterHelper { messageParameters = Map("c" -> c.toString)) } for (c <- patternPart if unsupportedLetters.contains(c) || - (isParsing && unsupportedLettersForParsing.contains(c))) { + (isParsing && unsupportedLettersForParsing.contains(c))) { throw new SparkIllegalArgumentException( errorClass = "INVALID_DATETIME_PATTERN.ILLEGAL_CHARACTER", - messageParameters = Map( - "c" -> c.toString, - "pattern" -> pattern)) + messageParameters = Map("c" -> c.toString, "pattern" -> pattern)) } for (style <- unsupportedPatternLengths if patternPart.contains(style)) { throw new SparkIllegalArgumentException( @@ -340,6 +354,8 @@ private object DateTimeFormatterHelper { } else { patternPart } - }.mkString("'").stripSuffix(" ") + } + .mkString("'") + .stripSuffix(" ") } } diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/MathUtils.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/MathUtils.scala index 96c3fb81aa66f..b113bccc74dfb 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/MathUtils.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/MathUtils.scala @@ -89,10 +89,7 @@ object MathUtils { def floorMod(a: Long, b: Long): Long = withOverflow(Math.floorMod(a, b)) - def withOverflow[A]( - f: => A, - hint: String = "", - context: QueryContext = null): A = { + def withOverflow[A](f: => A, hint: String = "", context: QueryContext = null): A = { try { f } catch { diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/RebaseDateTime.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/RebaseDateTime.scala index f9566b0e1fb13..9c043320dc812 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/RebaseDateTime.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/RebaseDateTime.scala @@ -34,8 +34,8 @@ import org.apache.spark.util.SparkClassUtils /** * The collection of functions for rebasing days and microseconds from/to the hybrid calendar - * (Julian + Gregorian since 1582-10-15) which is used by Spark 2.4 and earlier versions - * to/from Proleptic Gregorian calendar which is used by Spark since version 3.0. See SPARK-26651. + * (Julian + Gregorian since 1582-10-15) which is used by Spark 2.4 and earlier versions to/from + * Proleptic Gregorian calendar which is used by Spark since version 3.0. See SPARK-26651. */ object RebaseDateTime { @@ -46,20 +46,22 @@ object RebaseDateTime { } /** - * Rebases days since the epoch from an original to an target calendar, for instance, - * from a hybrid (Julian + Gregorian) to Proleptic Gregorian calendar. + * Rebases days since the epoch from an original to an target calendar, for instance, from a + * hybrid (Julian + Gregorian) to Proleptic Gregorian calendar. * * It finds the latest switch day which is less than the given `days`, and adds the difference - * in days associated with the switch days to the given `days`. - * The function is based on linear search which starts from the most recent switch days. - * This allows to perform less comparisons for modern dates. + * in days associated with the switch days to the given `days`. The function is based on linear + * search which starts from the most recent switch days. This allows to perform less comparisons + * for modern dates. * - * @param switches The days when difference in days between original and target calendar - * was changed. - * @param diffs The differences in days between calendars. - * @param days The number of days since the epoch 1970-01-01 to be rebased - * to the target calendar. - * @return The rebased days. + * @param switches + * The days when difference in days between original and target calendar was changed. + * @param diffs + * The differences in days between calendars. + * @param days + * The number of days since the epoch 1970-01-01 to be rebased to the target calendar. + * @return + * The rebased days. */ private def rebaseDays(switches: Array[Int], diffs: Array[Int], days: Int): Int = { var i = switches.length @@ -77,9 +79,8 @@ object RebaseDateTime { // Julian calendar). This array is not applicable for dates before the staring point. // Rebasing switch days and diffs `julianGregDiffSwitchDay` and `julianGregDiffs` // was generated by the `localRebaseJulianToGregorianDays` function. - private val julianGregDiffSwitchDay = Array( - -719164, -682945, -646420, -609895, -536845, -500320, -463795, - -390745, -354220, -317695, -244645, -208120, -171595, -141427) + private val julianGregDiffSwitchDay = Array(-719164, -682945, -646420, -609895, -536845, + -500320, -463795, -390745, -354220, -317695, -244645, -208120, -171595, -141427) final val lastSwitchJulianDay: Int = julianGregDiffSwitchDay.last @@ -88,20 +89,20 @@ object RebaseDateTime { /** * Converts the given number of days since the epoch day 1970-01-01 to a local date in Julian - * calendar, interprets the result as a local date in Proleptic Gregorian calendar, and takes the - * number of days since the epoch from the Gregorian local date. + * calendar, interprets the result as a local date in Proleptic Gregorian calendar, and takes + * the number of days since the epoch from the Gregorian local date. * * This is used to guarantee backward compatibility, as Spark 2.4 and earlier versions use - * Julian calendar for dates before 1582-10-15, while Spark 3.0 and later use Proleptic Gregorian - * calendar. See SPARK-26651. + * Julian calendar for dates before 1582-10-15, while Spark 3.0 and later use Proleptic + * Gregorian calendar. See SPARK-26651. * - * For example: - * Julian calendar: 1582-01-01 -> -141704 - * Proleptic Gregorian calendar: 1582-01-01 -> -141714 - * The code below converts -141704 to -141714. + * For example: Julian calendar: 1582-01-01 -> -141704 Proleptic Gregorian calendar: 1582-01-01 + * -> -141714 The code below converts -141704 to -141714. * - * @param days The number of days since the epoch in Julian calendar. It can be negative. - * @return The rebased number of days in Gregorian calendar. + * @param days + * The number of days since the epoch in Julian calendar. It can be negative. + * @return + * The rebased number of days in Gregorian calendar. */ private[sql] def localRebaseJulianToGregorianDays(days: Int): Int = { val utcCal = new Calendar.Builder() @@ -111,14 +112,15 @@ object RebaseDateTime { .setTimeZone(TimeZoneUTC) .setInstant(Math.multiplyExact(days, MILLIS_PER_DAY)) .build() - val localDate = LocalDate.of( - utcCal.get(YEAR), - utcCal.get(MONTH) + 1, - // The number of days will be added later to handle non-existing - // Julian dates in Proleptic Gregorian calendar. - // For example, 1000-02-29 exists in Julian calendar because 1000 - // is a leap year but it is not a leap year in Gregorian calendar. - 1) + val localDate = LocalDate + .of( + utcCal.get(YEAR), + utcCal.get(MONTH) + 1, + // The number of days will be added later to handle non-existing + // Julian dates in Proleptic Gregorian calendar. + // For example, 1000-02-29 exists in Julian calendar because 1000 + // is a leap year but it is not a leap year in Gregorian calendar. + 1) .`with`(ChronoField.ERA, utcCal.get(ERA)) .plusDays(utcCal.get(DAY_OF_MONTH) - 1) Math.toIntExact(localDate.toEpochDay) @@ -129,8 +131,10 @@ object RebaseDateTime { * pre-calculated rebasing array to save calculation. For dates of Before Common Era, the * function falls back to the regular unoptimized version. * - * @param days The number of days since the epoch in Julian calendar. It can be negative. - * @return The rebased number of days in Gregorian calendar. + * @param days + * The number of days since the epoch in Julian calendar. It can be negative. + * @return + * The rebased number of days in Gregorian calendar. */ def rebaseJulianToGregorianDays(days: Int): Int = { if (days < julianCommonEraStartDay) { @@ -143,18 +147,17 @@ object RebaseDateTime { // The differences in days between Proleptic Gregorian and Julian dates. // The diff at the index `i` is applicable for all days in the date interval: // [gregJulianDiffSwitchDay(i), gregJulianDiffSwitchDay(i+1)) - private val gregJulianDiffs = Array( - -2, -1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 9, 8, 7, 6, 5, 4, 3, 2, 1, 0) + private val gregJulianDiffs = + Array(-2, -1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 9, 8, 7, 6, 5, 4, 3, 2, 1, 0) // The sorted days in Proleptic Gregorian calendar when difference in days between // Proleptic Gregorian and Julian was changed. // The starting point is the `0001-01-01` (-719162 days since the epoch in // Proleptic Gregorian calendar). This array is not applicable for dates before the staring point. // Rebasing switch days and diffs `gregJulianDiffSwitchDay` and `gregJulianDiffs` // was generated by the `localRebaseGregorianToJulianDays` function. - private val gregJulianDiffSwitchDay = Array( - -719162, -682944, -646420, -609896, -536847, -500323, -463799, -390750, - -354226, -317702, -244653, -208129, -171605, -141436, -141435, -141434, - -141433, -141432, -141431, -141430, -141429, -141428, -141427) + private val gregJulianDiffSwitchDay = Array(-719162, -682944, -646420, -609896, -536847, + -500323, -463799, -390750, -354226, -317702, -244653, -208129, -171605, -141436, -141435, + -141434, -141433, -141432, -141431, -141430, -141429, -141428, -141427) final val lastSwitchGregorianDay: Int = gregJulianDiffSwitchDay.last @@ -171,17 +174,16 @@ object RebaseDateTime { * number of days since the epoch from the Julian local date. * * This is used to guarantee backward compatibility, as Spark 2.4 and earlier versions use - * Julian calendar for dates before 1582-10-15, while Spark 3.0 and later use Proleptic Gregorian - * calendar. See SPARK-26651. + * Julian calendar for dates before 1582-10-15, while Spark 3.0 and later use Proleptic + * Gregorian calendar. See SPARK-26651. * - * For example: - * Proleptic Gregorian calendar: 1582-01-01 -> -141714 - * Julian calendar: 1582-01-01 -> -141704 - * The code below converts -141714 to -141704. + * For example: Proleptic Gregorian calendar: 1582-01-01 -> -141714 Julian calendar: 1582-01-01 + * -> -141704 The code below converts -141714 to -141704. * - * @param days The number of days since the epoch in Proleptic Gregorian calendar. - * It can be negative. - * @return The rebased number of days in Julian calendar. + * @param days + * The number of days since the epoch in Proleptic Gregorian calendar. It can be negative. + * @return + * The rebased number of days in Julian calendar. */ private[sql] def localRebaseGregorianToJulianDays(days: Int): Int = { var localDate = LocalDate.ofEpochDay(days) @@ -204,8 +206,10 @@ object RebaseDateTime { * pre-calculated rebasing array to save calculation. For dates of Before Common Era, the * function falls back to the regular unoptimized version. * - * @param days The number of days since the epoch in Gregorian calendar. It can be negative. - * @return The rebased number of days since the epoch in Julian calendar. + * @param days + * The number of days since the epoch in Gregorian calendar. It can be negative. + * @return + * The rebased number of days since the epoch in Julian calendar. */ def rebaseGregorianToJulianDays(days: Int): Int = { if (days < gregorianCommonEraStartDay) { @@ -215,10 +219,9 @@ object RebaseDateTime { } } - /** - * The class describes JSON records with microseconds rebasing info. - * Here is an example of JSON file: + * The class describes JSON records with microseconds rebasing info. Here is an example of JSON + * file: * {{{ * [ * { @@ -229,37 +232,44 @@ object RebaseDateTime { * ] * }}} * - * @param tz One of time zone ID which is expected to be acceptable by `ZoneId.of`. - * @param switches An ordered array of seconds since the epoch when the diff between - * two calendars are changed. - * @param diffs Differences in seconds associated with elements of `switches`. + * @param tz + * One of time zone ID which is expected to be acceptable by `ZoneId.of`. + * @param switches + * An ordered array of seconds since the epoch when the diff between two calendars are + * changed. + * @param diffs + * Differences in seconds associated with elements of `switches`. */ private case class JsonRebaseRecord(tz: String, switches: Array[Long], diffs: Array[Long]) /** * Rebasing info used to convert microseconds from an original to a target calendar. * - * @param switches An ordered array of microseconds since the epoch when the diff between - * two calendars are changed. - * @param diffs Differences in microseconds associated with elements of `switches`. + * @param switches + * An ordered array of microseconds since the epoch when the diff between two calendars are + * changed. + * @param diffs + * Differences in microseconds associated with elements of `switches`. */ private[sql] case class RebaseInfo(switches: Array[Long], diffs: Array[Long]) /** - * Rebases micros since the epoch from an original to an target calendar, for instance, - * from a hybrid (Julian + Gregorian) to Proleptic Gregorian calendar. + * Rebases micros since the epoch from an original to an target calendar, for instance, from a + * hybrid (Julian + Gregorian) to Proleptic Gregorian calendar. * * It finds the latest switch micros which is less than the given `micros`, and adds the - * difference in micros associated with the switch micros to the given `micros`. - * The function is based on linear search which starts from the most recent switch micros. - * This allows to perform less comparisons for modern timestamps. + * difference in micros associated with the switch micros to the given `micros`. The function is + * based on linear search which starts from the most recent switch micros. This allows to + * perform less comparisons for modern timestamps. * - * @param rebaseInfo The rebasing info contains an ordered micros when difference in micros - * between original and target calendar was changed, - * and differences in micros between calendars - * @param micros The number of micros since the epoch 1970-01-01T00:00:00Z to be rebased - * to the target calendar. It can be negative. - * @return The rebased micros. + * @param rebaseInfo + * The rebasing info contains an ordered micros when difference in micros between original and + * target calendar was changed, and differences in micros between calendars + * @param micros + * The number of micros since the epoch 1970-01-01T00:00:00Z to be rebased to the target + * calendar. It can be negative. + * @return + * The rebased micros. */ private def rebaseMicros(rebaseInfo: RebaseInfo, micros: Long): Long = { val switches = rebaseInfo.switches @@ -296,18 +306,19 @@ object RebaseDateTime { /** * A map of time zone IDs to its ordered time points (instants in microseconds since the epoch) - * when the difference between 2 instances associated with the same local timestamp in - * Proleptic Gregorian and the hybrid calendar was changed, and to the diff at the index `i` is - * applicable for all microseconds in the time interval: - * [gregJulianDiffSwitchMicros(i), gregJulianDiffSwitchMicros(i+1)) + * when the difference between 2 instances associated with the same local timestamp in Proleptic + * Gregorian and the hybrid calendar was changed, and to the diff at the index `i` is applicable + * for all microseconds in the time interval: [gregJulianDiffSwitchMicros(i), + * gregJulianDiffSwitchMicros(i+1)) */ private val gregJulianRebaseMap = loadRebaseRecords("gregorian-julian-rebase-micros.json") private def getLastSwitchTs(rebaseMap: AnyRefMap[String, RebaseInfo]): Long = { val latestTs = rebaseMap.values.map(_.switches.last).max - require(rebaseMap.values.forall(_.diffs.last == 0), + require( + rebaseMap.values.forall(_.diffs.last == 0), s"Differences between Julian and Gregorian calendar after ${microsToInstant(latestTs)} " + - "are expected to be zero for all available time zones.") + "are expected to be zero for all available time zones.") latestTs } // The switch time point after which all diffs between Gregorian and Julian calendars @@ -315,29 +326,30 @@ object RebaseDateTime { final val lastSwitchGregorianTs: Long = getLastSwitchTs(gregJulianRebaseMap) private final val gregorianStartTs = LocalDateTime.of(gregorianStartDate, LocalTime.MIDNIGHT) - private final val julianEndTs = LocalDateTime.of( - julianEndDate, - LocalTime.of(23, 59, 59, 999999999)) + private final val julianEndTs = + LocalDateTime.of(julianEndDate, LocalTime.of(23, 59, 59, 999999999)) /** * Converts the given number of microseconds since the epoch '1970-01-01T00:00:00Z', to a local - * date-time in Proleptic Gregorian calendar with timezone identified by `zoneId`, interprets the - * result as a local date-time in Julian calendar with the same timezone, and takes microseconds - * since the epoch from the Julian local date-time. + * date-time in Proleptic Gregorian calendar with timezone identified by `zoneId`, interprets + * the result as a local date-time in Julian calendar with the same timezone, and takes + * microseconds since the epoch from the Julian local date-time. * * This is used to guarantee backward compatibility, as Spark 2.4 and earlier versions use - * Julian calendar for dates before 1582-10-15, while Spark 3.0 and later use Proleptic Gregorian - * calendar. See SPARK-26651. + * Julian calendar for dates before 1582-10-15, while Spark 3.0 and later use Proleptic + * Gregorian calendar. See SPARK-26651. * - * For example: - * Proleptic Gregorian calendar: 1582-01-01 00:00:00.123456 -> -12244061221876544 - * Julian calendar: 1582-01-01 00:00:00.123456 -> -12243196799876544 - * The code below converts -12244061221876544 to -12243196799876544. + * For example: Proleptic Gregorian calendar: 1582-01-01 00:00:00.123456 -> -12244061221876544 + * Julian calendar: 1582-01-01 00:00:00.123456 -> -12243196799876544 The code below converts + * -12244061221876544 to -12243196799876544. * - * @param tz The time zone at which the rebasing should be performed. - * @param micros The number of microseconds since the epoch '1970-01-01T00:00:00Z' - * in Proleptic Gregorian calendar. It can be negative. - * @return The rebased microseconds since the epoch in Julian calendar. + * @param tz + * The time zone at which the rebasing should be performed. + * @param micros + * The number of microseconds since the epoch '1970-01-01T00:00:00Z' in Proleptic Gregorian + * calendar. It can be negative. + * @return + * The rebased microseconds since the epoch in Julian calendar. */ private[sql] def rebaseGregorianToJulianMicros(tz: TimeZone, micros: Long): Long = { val instant = microsToInstant(micros) @@ -380,10 +392,13 @@ object RebaseDateTime { * contain information about the given time zone `timeZoneId` or `micros` is related to Before * Common Era, the function falls back to the regular unoptimized version. * - * @param timeZoneId A string identifier of a time zone. - * @param micros The number of microseconds since the epoch '1970-01-01T00:00:00Z' - * in Proleptic Gregorian calendar. It can be negative. - * @return The rebased microseconds since the epoch in Julian calendar. + * @param timeZoneId + * A string identifier of a time zone. + * @param micros + * The number of microseconds since the epoch '1970-01-01T00:00:00Z' in Proleptic Gregorian + * calendar. It can be negative. + * @return + * The rebased microseconds since the epoch in Julian calendar. */ def rebaseGregorianToJulianMicros(timeZoneId: String, micros: Long): Long = { if (micros >= lastSwitchGregorianTs) { @@ -404,12 +419,14 @@ object RebaseDateTime { * contain information about the current JVM system time zone or `micros` is related to Before * Common Era, the function falls back to the regular unoptimized version. * - * Note: The function assumes that the input micros was derived from a local timestamp - * at the default system JVM time zone in Proleptic Gregorian calendar. + * Note: The function assumes that the input micros was derived from a local timestamp at the + * default system JVM time zone in Proleptic Gregorian calendar. * - * @param micros The number of microseconds since the epoch '1970-01-01T00:00:00Z' - * in Proleptic Gregorian calendar. It can be negative. - * @return The rebased microseconds since the epoch in Julian calendar. + * @param micros + * The number of microseconds since the epoch '1970-01-01T00:00:00Z' in Proleptic Gregorian + * calendar. It can be negative. + * @return + * The rebased microseconds since the epoch in Julian calendar. */ def rebaseGregorianToJulianMicros(micros: Long): Long = { rebaseGregorianToJulianMicros(TimeZone.getDefault.getID, micros) @@ -418,22 +435,24 @@ object RebaseDateTime { /** * Converts the given number of microseconds since the epoch '1970-01-01T00:00:00Z', to a local * date-time in Julian calendar with timezone identified by `zoneId`, interprets the result as a - * local date-time in Proleptic Gregorian calendar with the same timezone, and takes microseconds - * since the epoch from the Gregorian local date-time. + * local date-time in Proleptic Gregorian calendar with the same timezone, and takes + * microseconds since the epoch from the Gregorian local date-time. * * This is used to guarantee backward compatibility, as Spark 2.4 and earlier versions use - * Julian calendar for dates before 1582-10-15, while Spark 3.0 and later use Proleptic Gregorian - * calendar. See SPARK-26651. + * Julian calendar for dates before 1582-10-15, while Spark 3.0 and later use Proleptic + * Gregorian calendar. See SPARK-26651. * - * For example: - * Julian calendar: 1582-01-01 00:00:00.123456 -> -12243196799876544 - * Proleptic Gregorian calendar: 1582-01-01 00:00:00.123456 -> -12244061221876544 - * The code below converts -12243196799876544 to -12244061221876544. + * For example: Julian calendar: 1582-01-01 00:00:00.123456 -> -12243196799876544 Proleptic + * Gregorian calendar: 1582-01-01 00:00:00.123456 -> -12244061221876544 The code below converts + * -12243196799876544 to -12244061221876544. * - * @param tz The time zone at which the rebasing should be performed. - * @param micros The number of microseconds since the epoch '1970-01-01T00:00:00Z' - * in the Julian calendar. It can be negative. - * @return The rebased microseconds since the epoch in Proleptic Gregorian calendar. + * @param tz + * The time zone at which the rebasing should be performed. + * @param micros + * The number of microseconds since the epoch '1970-01-01T00:00:00Z' in the Julian calendar. + * It can be negative. + * @return + * The rebased microseconds since the epoch in Proleptic Gregorian calendar. */ private[sql] def rebaseJulianToGregorianMicros(tz: TimeZone, micros: Long): Long = { val cal = new Calendar.Builder() @@ -442,18 +461,19 @@ object RebaseDateTime { .setInstant(microsToMillis(micros)) .setTimeZone(tz) .build() - val localDateTime = LocalDateTime.of( - cal.get(YEAR), - cal.get(MONTH) + 1, - // The number of days will be added later to handle non-existing - // Julian dates in Proleptic Gregorian calendar. - // For example, 1000-02-29 exists in Julian calendar because 1000 - // is a leap year but it is not a leap year in Gregorian calendar. - 1, - cal.get(HOUR_OF_DAY), - cal.get(MINUTE), - cal.get(SECOND), - (Math.floorMod(micros, MICROS_PER_SECOND) * NANOS_PER_MICROS).toInt) + val localDateTime = LocalDateTime + .of( + cal.get(YEAR), + cal.get(MONTH) + 1, + // The number of days will be added later to handle non-existing + // Julian dates in Proleptic Gregorian calendar. + // For example, 1000-02-29 exists in Julian calendar because 1000 + // is a leap year but it is not a leap year in Gregorian calendar. + 1, + cal.get(HOUR_OF_DAY), + cal.get(MINUTE), + cal.get(SECOND), + (Math.floorMod(micros, MICROS_PER_SECOND) * NANOS_PER_MICROS).toInt) .`with`(ChronoField.ERA, cal.get(ERA)) .plusDays(cal.get(DAY_OF_MONTH) - 1) val zoneId = tz.toZoneId @@ -494,10 +514,13 @@ object RebaseDateTime { * contain information about the given time zone `timeZoneId` or `micros` is related to Before * Common Era, the function falls back to the regular unoptimized version. * - * @param timeZoneId A string identifier of a time zone. - * @param micros The number of microseconds since the epoch '1970-01-01T00:00:00Z' - * in the Julian calendar. It can be negative. - * @return The rebased microseconds since the epoch in Proleptic Gregorian calendar. + * @param timeZoneId + * A string identifier of a time zone. + * @param micros + * The number of microseconds since the epoch '1970-01-01T00:00:00Z' in the Julian calendar. + * It can be negative. + * @return + * The rebased microseconds since the epoch in Proleptic Gregorian calendar. */ def rebaseJulianToGregorianMicros(timeZoneId: String, micros: Long): Long = { if (micros >= lastSwitchJulianTs) { @@ -518,12 +541,14 @@ object RebaseDateTime { * contain information about the current JVM system time zone or `micros` is related to Before * Common Era, the function falls back to the regular unoptimized version. * - * Note: The function assumes that the input micros was derived from a local timestamp - * at the default system JVM time zone in the Julian calendar. + * Note: The function assumes that the input micros was derived from a local timestamp at the + * default system JVM time zone in the Julian calendar. * - * @param micros The number of microseconds since the epoch '1970-01-01T00:00:00Z' - * in the Julian calendar. It can be negative. - * @return The rebased microseconds since the epoch in Proleptic Gregorian calendar. + * @param micros + * The number of microseconds since the epoch '1970-01-01T00:00:00Z' in the Julian calendar. + * It can be negative. + * @return + * The rebased microseconds since the epoch in Proleptic Gregorian calendar. */ def rebaseJulianToGregorianMicros(micros: Long): Long = { rebaseJulianToGregorianMicros(TimeZone.getDefault.getID, micros) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkCharVarcharUtils.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkCharVarcharUtils.scala index 498eb83566eb3..2a26c079e8d4d 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkCharVarcharUtils.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkCharVarcharUtils.scala @@ -21,15 +21,18 @@ import org.apache.spark.sql.internal.SqlApiConf import org.apache.spark.sql.types.{ArrayType, CharType, DataType, MapType, StringType, StructType, VarcharType} trait SparkCharVarcharUtils { + /** - * Returns true if the given data type is CharType/VarcharType or has nested CharType/VarcharType. + * Returns true if the given data type is CharType/VarcharType or has nested + * CharType/VarcharType. */ def hasCharVarchar(dt: DataType): Boolean = { dt.existsRecursively(f => f.isInstanceOf[CharType] || f.isInstanceOf[VarcharType]) } /** - * Validate the given [[DataType]] to fail if it is char or varchar types or contains nested ones + * Validate the given [[DataType]] to fail if it is char or varchar types or contains nested + * ones */ def failIfHasCharVarchar(dt: DataType): DataType = { if (!SqlApiConf.get.charVarcharAsString && hasCharVarchar(dt)) { diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkDateTimeUtils.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkDateTimeUtils.scala index a6592ad51c65c..4e94bc6617357 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkDateTimeUtils.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkDateTimeUtils.scala @@ -54,8 +54,10 @@ trait SparkDateTimeUtils { /** * Converts an Java object to days. * - * @param obj Either an object of `java.sql.Date` or `java.time.LocalDate`. - * @return The number of days since 1970-01-01. + * @param obj + * Either an object of `java.sql.Date` or `java.time.LocalDate`. + * @return + * The number of days since 1970-01-01. */ def anyToDays(obj: Any): Int = obj match { case d: Date => fromJavaDate(d) @@ -65,8 +67,10 @@ trait SparkDateTimeUtils { /** * Converts an Java object to microseconds. * - * @param obj Either an object of `java.sql.Timestamp` or `java.time.{Instant,LocalDateTime}`. - * @return The number of micros since the epoch. + * @param obj + * Either an object of `java.sql.Timestamp` or `java.time.{Instant,LocalDateTime}`. + * @return + * The number of micros since the epoch. */ def anyToMicros(obj: Any): Long = obj match { case t: Timestamp => fromJavaTimestamp(t) @@ -75,8 +79,8 @@ trait SparkDateTimeUtils { } /** - * Converts the timestamp to milliseconds since epoch. In Spark timestamp values have microseconds - * precision, so this conversion is lossy. + * Converts the timestamp to milliseconds since epoch. In Spark timestamp values have + * microseconds precision, so this conversion is lossy. */ def microsToMillis(micros: Long): Long = { // When the timestamp is negative i.e before 1970, we need to adjust the milliseconds portion. @@ -97,8 +101,8 @@ trait SparkDateTimeUtils { private val MIN_SECONDS = Math.floorDiv(Long.MinValue, MICROS_PER_SECOND) /** - * Obtains an instance of `java.time.Instant` using microseconds from - * the epoch of 1970-01-01 00:00:00Z. + * Obtains an instance of `java.time.Instant` using microseconds from the epoch of 1970-01-01 + * 00:00:00Z. */ def microsToInstant(micros: Long): Instant = { val secs = Math.floorDiv(micros, MICROS_PER_SECOND) @@ -110,8 +114,8 @@ trait SparkDateTimeUtils { /** * Gets the number of microseconds since the epoch of 1970-01-01 00:00:00Z from the given - * instance of `java.time.Instant`. The epoch microsecond count is a simple incrementing count of - * microseconds where microsecond 0 is 1970-01-01 00:00:00Z. + * instance of `java.time.Instant`. The epoch microsecond count is a simple incrementing count + * of microseconds where microsecond 0 is 1970-01-01 00:00:00Z. */ def instantToMicros(instant: Instant): Long = { val secs = instant.getEpochSecond @@ -127,8 +131,8 @@ trait SparkDateTimeUtils { /** * Converts the timestamp `micros` from one timezone to another. * - * Time-zone rules, such as daylight savings, mean that not every local date-time - * is valid for the `toZone` time zone, thus the local date-time may be adjusted. + * Time-zone rules, such as daylight savings, mean that not every local date-time is valid for + * the `toZone` time zone, thus the local date-time may be adjusted. */ def convertTz(micros: Long, fromZone: ZoneId, toZone: ZoneId): Long = { val rebasedDateTime = getLocalDateTime(micros, toZone).atZone(fromZone) @@ -160,14 +164,16 @@ trait SparkDateTimeUtils { def daysToLocalDate(days: Int): LocalDate = LocalDate.ofEpochDay(days) /** - * Converts microseconds since 1970-01-01 00:00:00Z to days since 1970-01-01 at the given zone ID. + * Converts microseconds since 1970-01-01 00:00:00Z to days since 1970-01-01 at the given zone + * ID. */ def microsToDays(micros: Long, zoneId: ZoneId): Int = { localDateToDays(getLocalDateTime(micros, zoneId).toLocalDate) } /** - * Converts days since 1970-01-01 at the given zone ID to microseconds since 1970-01-01 00:00:00Z. + * Converts days since 1970-01-01 at the given zone ID to microseconds since 1970-01-01 + * 00:00:00Z. */ def daysToMicros(days: Int, zoneId: ZoneId): Long = { val instant = daysToLocalDate(days).atStartOfDay(zoneId).toInstant @@ -175,20 +181,22 @@ trait SparkDateTimeUtils { } /** - * Converts a local date at the default JVM time zone to the number of days since 1970-01-01 - * in the hybrid calendar (Julian + Gregorian) by discarding the time part. The resulted days are + * Converts a local date at the default JVM time zone to the number of days since 1970-01-01 in + * the hybrid calendar (Julian + Gregorian) by discarding the time part. The resulted days are * rebased from the hybrid to Proleptic Gregorian calendar. The days rebasing is performed via - * UTC time zone for simplicity because the difference between two calendars is the same in - * any given time zone and UTC time zone. + * UTC time zone for simplicity because the difference between two calendars is the same in any + * given time zone and UTC time zone. * - * Note: The date is shifted by the offset of the default JVM time zone for backward compatibility - * with Spark 2.4 and earlier versions. The goal of the shift is to get a local date derived - * from the number of days that has the same date fields (year, month, day) as the original - * `date` at the default JVM time zone. + * Note: The date is shifted by the offset of the default JVM time zone for backward + * compatibility with Spark 2.4 and earlier versions. The goal of the shift is to get a local + * date derived from the number of days that has the same date fields (year, month, day) as the + * original `date` at the default JVM time zone. * - * @param date It represents a specific instant in time based on the hybrid calendar which - * combines Julian and Gregorian calendars. - * @return The number of days since the epoch in Proleptic Gregorian calendar. + * @param date + * It represents a specific instant in time based on the hybrid calendar which combines Julian + * and Gregorian calendars. + * @return + * The number of days since the epoch in Proleptic Gregorian calendar. */ def fromJavaDate(date: Date): Int = { val millisUtc = date.getTime @@ -207,18 +215,20 @@ trait SparkDateTimeUtils { } /** - * Converts days since the epoch 1970-01-01 in Proleptic Gregorian calendar to a local date - * at the default JVM time zone in the hybrid calendar (Julian + Gregorian). It rebases the given + * Converts days since the epoch 1970-01-01 in Proleptic Gregorian calendar to a local date at + * the default JVM time zone in the hybrid calendar (Julian + Gregorian). It rebases the given * days from Proleptic Gregorian to the hybrid calendar at UTC time zone for simplicity because * the difference between two calendars doesn't depend on any time zone. The result is shifted - * by the time zone offset in wall clock to have the same date fields (year, month, day) - * at the default JVM time zone as the input `daysSinceEpoch` in Proleptic Gregorian calendar. + * by the time zone offset in wall clock to have the same date fields (year, month, day) at the + * default JVM time zone as the input `daysSinceEpoch` in Proleptic Gregorian calendar. * - * Note: The date is shifted by the offset of the default JVM time zone for backward compatibility - * with Spark 2.4 and earlier versions. + * Note: The date is shifted by the offset of the default JVM time zone for backward + * compatibility with Spark 2.4 and earlier versions. * - * @param days The number of days since 1970-01-01 in Proleptic Gregorian calendar. - * @return A local date in the hybrid calendar as `java.sql.Date` from number of days since epoch. + * @param days + * The number of days since 1970-01-01 in Proleptic Gregorian calendar. + * @return + * A local date in the hybrid calendar as `java.sql.Date` from number of days since epoch. */ def toJavaDate(days: Int): Date = { val rebasedDays = rebaseGregorianToJulianDays(days) @@ -233,20 +243,22 @@ trait SparkDateTimeUtils { } /** - * Converts microseconds since the epoch to an instance of `java.sql.Timestamp` - * via creating a local timestamp at the system time zone in Proleptic Gregorian - * calendar, extracting date and time fields like `year` and `hours`, and forming - * new timestamp in the hybrid calendar from the extracted fields. + * Converts microseconds since the epoch to an instance of `java.sql.Timestamp` via creating a + * local timestamp at the system time zone in Proleptic Gregorian calendar, extracting date and + * time fields like `year` and `hours`, and forming new timestamp in the hybrid calendar from + * the extracted fields. * - * The conversion is based on the JVM system time zone because the `java.sql.Timestamp` - * uses the time zone internally. + * The conversion is based on the JVM system time zone because the `java.sql.Timestamp` uses the + * time zone internally. * * The method performs the conversion via local timestamp fields to have the same date-time - * representation as `year`, `month`, `day`, ..., `seconds` in the original calendar - * and in the target calendar. + * representation as `year`, `month`, `day`, ..., `seconds` in the original calendar and in the + * target calendar. * - * @param micros The number of microseconds since 1970-01-01T00:00:00.000000Z. - * @return A `java.sql.Timestamp` from number of micros since epoch. + * @param micros + * The number of microseconds since 1970-01-01T00:00:00.000000Z. + * @return + * A `java.sql.Timestamp` from number of micros since epoch. */ def toJavaTimestamp(micros: Long): Timestamp = toJavaTimestampNoRebase(rebaseGregorianToJulianMicros(micros)) @@ -257,8 +269,10 @@ trait SparkDateTimeUtils { /** * Converts microseconds since the epoch to an instance of `java.sql.Timestamp`. * - * @param micros The number of microseconds since 1970-01-01T00:00:00.000000Z. - * @return A `java.sql.Timestamp` from number of micros since epoch. + * @param micros + * The number of microseconds since 1970-01-01T00:00:00.000000Z. + * @return + * A `java.sql.Timestamp` from number of micros since epoch. */ def toJavaTimestampNoRebase(micros: Long): Timestamp = { val seconds = Math.floorDiv(micros, MICROS_PER_SECOND) @@ -270,22 +284,22 @@ trait SparkDateTimeUtils { /** * Converts an instance of `java.sql.Timestamp` to the number of microseconds since - * 1970-01-01T00:00:00.000000Z. It extracts date-time fields from the input, builds - * a local timestamp in Proleptic Gregorian calendar from the fields, and binds - * the timestamp to the system time zone. The resulted instant is converted to - * microseconds since the epoch. + * 1970-01-01T00:00:00.000000Z. It extracts date-time fields from the input, builds a local + * timestamp in Proleptic Gregorian calendar from the fields, and binds the timestamp to the + * system time zone. The resulted instant is converted to microseconds since the epoch. * - * The conversion is performed via the system time zone because it is used internally - * in `java.sql.Timestamp` while extracting date-time fields. + * The conversion is performed via the system time zone because it is used internally in + * `java.sql.Timestamp` while extracting date-time fields. * * The goal of the function is to have the same local date-time in the original calendar - * - the hybrid calendar (Julian + Gregorian) and in the target calendar which is - * Proleptic Gregorian calendar, see SPARK-26651. + * - the hybrid calendar (Julian + Gregorian) and in the target calendar which is Proleptic + * Gregorian calendar, see SPARK-26651. * - * @param t It represents a specific instant in time based on - * the hybrid calendar which combines Julian and - * Gregorian calendars. - * @return The number of micros since epoch from `java.sql.Timestamp`. + * @param t + * It represents a specific instant in time based on the hybrid calendar which combines Julian + * and Gregorian calendars. + * @return + * The number of micros since epoch from `java.sql.Timestamp`. */ def fromJavaTimestamp(t: Timestamp): Long = rebaseJulianToGregorianMicros(fromJavaTimestampNoRebase(t)) @@ -297,30 +311,27 @@ trait SparkDateTimeUtils { * Converts an instance of `java.sql.Timestamp` to the number of microseconds since * 1970-01-01T00:00:00.000000Z. * - * @param t an instance of `java.sql.Timestamp`. - * @return The number of micros since epoch from `java.sql.Timestamp`. + * @param t + * an instance of `java.sql.Timestamp`. + * @return + * The number of micros since epoch from `java.sql.Timestamp`. */ def fromJavaTimestampNoRebase(t: Timestamp): Long = millisToMicros(t.getTime) + (t.getNanos / NANOS_PER_MICROS) % MICROS_PER_MILLIS /** - * Trims and parses a given UTF8 date string to a corresponding [[Int]] value. - * The return type is [[Option]] in order to distinguish between 0 and null. The following - * formats are allowed: + * Trims and parses a given UTF8 date string to a corresponding [[Int]] value. The return type + * is [[Option]] in order to distinguish between 0 and null. The following formats are allowed: * - * `[+-]yyyy*` - * `[+-]yyyy*-[m]m` - * `[+-]yyyy*-[m]m-[d]d` - * `[+-]yyyy*-[m]m-[d]d ` - * `[+-]yyyy*-[m]m-[d]d *` - * `[+-]yyyy*-[m]m-[d]dT*` + * `[+-]yyyy*` `[+-]yyyy*-[m]m` `[+-]yyyy*-[m]m-[d]d` `[+-]yyyy*-[m]m-[d]d ` + * `[+-]yyyy*-[m]m-[d]d *` `[+-]yyyy*-[m]m-[d]dT*` */ def stringToDate(s: UTF8String): Option[Int] = { def isValidDigits(segment: Int, digits: Int): Boolean = { // An integer is able to represent a date within [+-]5 million years. val maxDigitsYear = 7 (segment == 0 && digits >= 4 && digits <= maxDigitsYear) || - (segment != 0 && digits > 0 && digits <= 2) + (segment != 0 && digits > 0 && digits <= 2) } if (s == null) { return None @@ -380,24 +391,18 @@ trait SparkDateTimeUtils { } } - def stringToDateAnsi( - s: UTF8String, - context: QueryContext = null): Int = { + def stringToDateAnsi(s: UTF8String, context: QueryContext = null): Int = { stringToDate(s).getOrElse { throw ExecutionErrors.invalidInputInCastToDatetimeError(s, DateType, context) } } /** - * Trims and parses a given UTF8 timestamp string to the corresponding timestamp segments, - * time zone id and whether it is just time without a date. - * value. The return type is [[Option]] in order to distinguish between 0L and null. The following - * formats are allowed: + * Trims and parses a given UTF8 timestamp string to the corresponding timestamp segments, time + * zone id and whether it is just time without a date. value. The return type is [[Option]] in + * order to distinguish between 0L and null. The following formats are allowed: * - * `[+-]yyyy*` - * `[+-]yyyy*-[m]m` - * `[+-]yyyy*-[m]m-[d]d` - * `[+-]yyyy*-[m]m-[d]d ` + * `[+-]yyyy*` `[+-]yyyy*-[m]m` `[+-]yyyy*-[m]m-[d]d` `[+-]yyyy*-[m]m-[d]d ` * `[+-]yyyy*-[m]m-[d]d [h]h:[m]m:[s]s.[ms][ms][ms][us][us][us][zone_id]` * `[+-]yyyy*-[m]m-[d]dT[h]h:[m]m:[s]s.[ms][ms][ms][us][us][us][zone_id]` * `[h]h:[m]m:[s]s.[ms][ms][ms][us][us][us][zone_id]` @@ -407,16 +412,17 @@ trait SparkDateTimeUtils { * - Z - Zulu time zone UTC+0 * - +|-[h]h:[m]m * - A short id, see https://docs.oracle.com/javase/8/docs/api/java/time/ZoneId.html#SHORT_IDS - * - An id with one of the prefixes UTC+, UTC-, GMT+, GMT-, UT+ or UT-, - * and a suffix in the formats: + * - An id with one of the prefixes UTC+, UTC-, GMT+, GMT-, UT+ or UT-, and a suffix in the + * formats: * - +|-h[h] * - +|-hh[:]mm * - +|-hh:mm:ss * - +|-hhmmss - * - Region-based zone IDs in the form `area/city`, such as `Europe/Paris` + * - Region-based zone IDs in the form `area/city`, such as `Europe/Paris` * - * @return timestamp segments, time zone id and whether the input is just time without a date. If - * the input string can't be parsed as timestamp, the result timestamp segments are empty. + * @return + * timestamp segments, time zone id and whether the input is just time without a date. If the + * input string can't be parsed as timestamp, the result timestamp segments are empty. */ def parseTimestampString(s: UTF8String): (Array[Int], Option[ZoneId], Boolean) = { def isValidDigits(segment: Int, digits: Int): Boolean = { @@ -424,9 +430,9 @@ trait SparkDateTimeUtils { val maxDigitsYear = 6 // For the nanosecond part, more than 6 digits is allowed, but will be truncated. segment == 6 || (segment == 0 && digits >= 4 && digits <= maxDigitsYear) || - // For the zoneId segment(7), it's could be zero digits when it's a region-based zone ID - (segment == 7 && digits <= 2) || - (segment != 0 && segment != 6 && segment != 7 && digits > 0 && digits <= 2) + // For the zoneId segment(7), it's could be zero digits when it's a region-based zone ID + (segment == 7 && digits <= 2) || + (segment != 0 && segment != 6 && segment != 7 && digits > 0 && digits <= 2) } if (s == null) { return (Array.empty, None, false) @@ -523,7 +529,7 @@ trait SparkDateTimeUtils { tz = Some(new String(bytes, j, strEndTrimmed - j)) j = strEndTrimmed - 1 } - if (i == 6 && b != '.') { + if (i == 6 && b != '.') { i += 1 } } else { @@ -612,11 +618,11 @@ trait SparkDateTimeUtils { * * If the input string contains a component associated with time zone, the method will return * `None` if `allowTimeZone` is set to `false`. If `allowTimeZone` is set to `true`, the method - * will simply discard the time zone component. Enable the check to detect situations like parsing - * a timestamp with time zone as TimestampNTZType. + * will simply discard the time zone component. Enable the check to detect situations like + * parsing a timestamp with time zone as TimestampNTZType. * - * The return type is [[Option]] in order to distinguish between 0L and null. Please - * refer to `parseTimestampString` for the allowed formats. + * The return type is [[Option]] in order to distinguish between 0L and null. Please refer to + * `parseTimestampString` for the allowed formats. */ def stringToTimestampWithoutTimeZone(s: UTF8String, allowTimeZone: Boolean): Option[Long] = { try { @@ -637,10 +643,13 @@ trait SparkDateTimeUtils { } /** - * Returns the index of the first non-whitespace and non-ISO control character in the byte array. + * Returns the index of the first non-whitespace and non-ISO control character in the byte + * array. * - * @param bytes The byte array to be processed. - * @return The start index after trimming. + * @param bytes + * The byte array to be processed. + * @return + * The start index after trimming. */ @inline private def getTrimmedStart(bytes: Array[Byte]) = { var start = 0 @@ -655,9 +664,12 @@ trait SparkDateTimeUtils { /** * Returns the index of the last non-whitespace and non-ISO control character in the byte array. * - * @param start The starting index for the search. - * @param bytes The byte array to be processed. - * @return The end index after trimming. + * @param start + * The starting index for the search. + * @param bytes + * The byte array to be processed. + * @return + * The end index after trimming. */ @inline private def getTrimmedEnd(start: Int, bytes: Array[Byte]) = { var end = bytes.length - 1 diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkIntervalUtils.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkIntervalUtils.scala index 5e236187a4a0f..b8387b78ae3e2 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkIntervalUtils.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkIntervalUtils.scala @@ -38,17 +38,16 @@ trait SparkIntervalUtils { private final val minDurationSeconds = Math.floorDiv(Long.MinValue, MICROS_PER_SECOND) /** - * Converts this duration to the total length in microseconds. - *

    - * If this duration is too large to fit in a [[Long]] microseconds, then an - * exception is thrown. - *

    - * If this duration has greater than microsecond precision, then the conversion - * will drop any excess precision information as though the amount in nanoseconds - * was subject to integer division by one thousand. + * Converts this duration to the total length in microseconds.

    If this duration is too large + * to fit in a [[Long]] microseconds, then an exception is thrown.

    If this duration has + * greater than microsecond precision, then the conversion will drop any excess precision + * information as though the amount in nanoseconds was subject to integer division by one + * thousand. * - * @return The total length of the duration in microseconds - * @throws ArithmeticException If numeric overflow occurs + * @return + * The total length of the duration in microseconds + * @throws ArithmeticException + * If numeric overflow occurs */ def durationToMicros(duration: Duration): Long = { durationToMicros(duration, DT.SECOND) @@ -59,7 +58,8 @@ trait SparkIntervalUtils { val micros = if (seconds == minDurationSeconds) { val microsInSeconds = (minDurationSeconds + 1) * MICROS_PER_SECOND val nanoAdjustment = duration.getNano - assert(0 <= nanoAdjustment && nanoAdjustment < NANOS_PER_SECOND, + assert( + 0 <= nanoAdjustment && nanoAdjustment < NANOS_PER_SECOND, "Duration.getNano() must return the adjustment to the seconds field " + "in the range from 0 to 999999999 nanoseconds, inclusive.") Math.addExact(microsInSeconds, (nanoAdjustment - NANOS_PER_SECOND) / NANOS_PER_MICROS) @@ -77,14 +77,13 @@ trait SparkIntervalUtils { } /** - * Gets the total number of months in this period. - *

    - * This returns the total number of months in the period by multiplying the - * number of years by 12 and adding the number of months. - *

    + * Gets the total number of months in this period.

    This returns the total number of months + * in the period by multiplying the number of years by 12 and adding the number of months.

    * - * @return The total number of months in the period, may be negative - * @throws ArithmeticException If numeric overflow occurs + * @return + * The total number of months in the period, may be negative + * @throws ArithmeticException + * If numeric overflow occurs */ def periodToMonths(period: Period): Int = { periodToMonths(period, YM.MONTH) @@ -103,39 +102,41 @@ trait SparkIntervalUtils { /** * Obtains a [[Duration]] representing a number of microseconds. * - * @param micros The number of microseconds, positive or negative - * @return A [[Duration]], not null + * @param micros + * The number of microseconds, positive or negative + * @return + * A [[Duration]], not null */ def microsToDuration(micros: Long): Duration = Duration.of(micros, ChronoUnit.MICROS) /** - * Obtains a [[Period]] representing a number of months. The days unit will be zero, and the years - * and months units will be normalized. + * Obtains a [[Period]] representing a number of months. The days unit will be zero, and the + * years and months units will be normalized. * - *

    - * The months unit is adjusted to have an absolute value < 12, with the years unit being adjusted - * to compensate. For example, the method returns "2 years and 3 months" for the 27 input months. - *

    - * The sign of the years and months units will be the same after normalization. - * For example, -13 months will be converted to "-1 year and -1 month". + *

    The months unit is adjusted to have an absolute value < 12, with the years unit being + * adjusted to compensate. For example, the method returns "2 years and 3 months" for the 27 + * input months.

    The sign of the years and months units will be the same after + * normalization. For example, -13 months will be converted to "-1 year and -1 month". * - * @param months The number of months, positive or negative - * @return The period of months, not null + * @param months + * The number of months, positive or negative + * @return + * The period of months, not null */ def monthsToPeriod(months: Int): Period = Period.ofMonths(months).normalized() /** * Converts a string to [[CalendarInterval]] case-insensitively. * - * @throws IllegalArgumentException if the input string is not in valid interval format. + * @throws IllegalArgumentException + * if the input string is not in valid interval format. */ def stringToInterval(input: UTF8String): CalendarInterval = { import ParseState._ if (input == null) { throw new SparkIllegalArgumentException( errorClass = "INVALID_INTERVAL_FORMAT.INPUT_IS_NULL", - messageParameters = Map( - "input" -> "null")) + messageParameters = Map("input" -> "null")) } // scalastyle:off caselocale .toLowerCase val s = input.trimAll().toLowerCase @@ -144,8 +145,7 @@ trait SparkIntervalUtils { if (bytes.isEmpty) { throw new SparkIllegalArgumentException( errorClass = "INVALID_INTERVAL_FORMAT.INPUT_IS_EMPTY", - messageParameters = Map( - "input" -> input.toString)) + messageParameters = Map("input" -> input.toString)) } var state = PREFIX var i = 0 @@ -182,14 +182,11 @@ trait SparkIntervalUtils { if (s.numBytes() == intervalStr.numBytes()) { throw new SparkIllegalArgumentException( errorClass = "INVALID_INTERVAL_FORMAT.INPUT_IS_EMPTY", - messageParameters = Map( - "input" -> input.toString)) + messageParameters = Map("input" -> input.toString)) } else if (!Character.isWhitespace(bytes(i + intervalStr.numBytes()))) { throw new SparkIllegalArgumentException( errorClass = "INVALID_INTERVAL_FORMAT.INVALID_PREFIX", - messageParameters = Map( - "input" -> input.toString, - "prefix" -> currentWord)) + messageParameters = Map("input" -> input.toString, "prefix" -> currentWord)) } else { i += intervalStr.numBytes() + 1 } @@ -224,11 +221,10 @@ trait SparkIntervalUtils { pointPrefixed = true i += 1 state = VALUE_FRACTIONAL_PART - case _ => throw new SparkIllegalArgumentException( - errorClass = "INVALID_INTERVAL_FORMAT.UNRECOGNIZED_NUMBER", - messageParameters = Map( - "input" -> input.toString, - "number" -> currentWord)) + case _ => + throw new SparkIllegalArgumentException( + errorClass = "INVALID_INTERVAL_FORMAT.UNRECOGNIZED_NUMBER", + messageParameters = Map("input" -> input.toString, "number" -> currentWord)) } case TRIM_BEFORE_VALUE => trimToNextState(b, VALUE) case VALUE => @@ -237,20 +233,19 @@ trait SparkIntervalUtils { try { currentValue = Math.addExact(Math.multiplyExact(10, currentValue), (b - '0')) } catch { - case e: ArithmeticException => throw new SparkIllegalArgumentException( - errorClass = "INVALID_INTERVAL_FORMAT.ARITHMETIC_EXCEPTION", - messageParameters = Map( - "input" -> input.toString)) + case e: ArithmeticException => + throw new SparkIllegalArgumentException( + errorClass = "INVALID_INTERVAL_FORMAT.ARITHMETIC_EXCEPTION", + messageParameters = Map("input" -> input.toString)) } case _ if Character.isWhitespace(b) => state = TRIM_BEFORE_UNIT case '.' => fractionScale = initialFractionScale state = VALUE_FRACTIONAL_PART - case _ => throw new SparkIllegalArgumentException( - errorClass = "INVALID_INTERVAL_FORMAT.INVALID_VALUE", - messageParameters = Map( - "input" -> input.toString, - "value" -> currentWord)) + case _ => + throw new SparkIllegalArgumentException( + errorClass = "INVALID_INTERVAL_FORMAT.INVALID_VALUE", + messageParameters = Map("input" -> input.toString, "value" -> currentWord)) } i += 1 case VALUE_FRACTIONAL_PART => @@ -264,15 +259,11 @@ trait SparkIntervalUtils { } else if ('0' <= b && b <= '9') { throw new SparkIllegalArgumentException( errorClass = "INVALID_INTERVAL_FORMAT.INVALID_PRECISION", - messageParameters = Map( - "input" -> input.toString, - "value" -> currentWord)) + messageParameters = Map("input" -> input.toString, "value" -> currentWord)) } else { throw new SparkIllegalArgumentException( errorClass = "INVALID_INTERVAL_FORMAT.INVALID_VALUE", - messageParameters = Map( - "input" -> input.toString, - "value" -> currentWord)) + messageParameters = Map("input" -> input.toString, "value" -> currentWord)) } i += 1 case TRIM_BEFORE_UNIT => trimToNextState(b, UNIT_BEGIN) @@ -281,9 +272,7 @@ trait SparkIntervalUtils { if (b != 's' && fractionScale >= 0) { throw new SparkIllegalArgumentException( errorClass = "INVALID_INTERVAL_FORMAT.INVALID_FRACTION", - messageParameters = Map( - "input" -> input.toString, - "unit" -> currentWord)) + messageParameters = Map("input" -> input.toString, "unit" -> currentWord)) } if (isNegative) { currentValue = -currentValue @@ -328,44 +317,38 @@ trait SparkIntervalUtils { } else { throw new SparkIllegalArgumentException( errorClass = "INVALID_INTERVAL_FORMAT.INVALID_UNIT", - messageParameters = Map( - "input" -> input.toString, - "unit" -> currentWord)) + messageParameters = Map("input" -> input.toString, "unit" -> currentWord)) } - case _ => throw new SparkIllegalArgumentException( - errorClass = "INVALID_INTERVAL_FORMAT.INVALID_UNIT", - messageParameters = Map( - "input" -> input.toString, - "unit" -> currentWord)) + case _ => + throw new SparkIllegalArgumentException( + errorClass = "INVALID_INTERVAL_FORMAT.INVALID_UNIT", + messageParameters = Map("input" -> input.toString, "unit" -> currentWord)) } } catch { - case e: ArithmeticException => throw new SparkIllegalArgumentException( - errorClass = "INVALID_INTERVAL_FORMAT.ARITHMETIC_EXCEPTION", - messageParameters = Map( - "input" -> input.toString)) + case e: ArithmeticException => + throw new SparkIllegalArgumentException( + errorClass = "INVALID_INTERVAL_FORMAT.ARITHMETIC_EXCEPTION", + messageParameters = Map("input" -> input.toString)) } state = UNIT_SUFFIX case UNIT_SUFFIX => b match { case 's' => state = UNIT_END case _ if Character.isWhitespace(b) => state = TRIM_BEFORE_SIGN - case _ => throw new SparkIllegalArgumentException( - errorClass = "INVALID_INTERVAL_FORMAT.INVALID_UNIT", - messageParameters = Map( - "input" -> input.toString, - "unit" -> currentWord)) + case _ => + throw new SparkIllegalArgumentException( + errorClass = "INVALID_INTERVAL_FORMAT.INVALID_UNIT", + messageParameters = Map("input" -> input.toString, "unit" -> currentWord)) } i += 1 case UNIT_END => - if (Character.isWhitespace(b) ) { + if (Character.isWhitespace(b)) { i += 1 state = TRIM_BEFORE_SIGN } else { throw new SparkIllegalArgumentException( errorClass = "INVALID_INTERVAL_FORMAT.INVALID_UNIT", - messageParameters = Map( - "input" -> input.toString, - "unit" -> currentWord)) + messageParameters = Map("input" -> input.toString, "unit" -> currentWord)) } } } @@ -373,36 +356,37 @@ trait SparkIntervalUtils { val result = state match { case UNIT_SUFFIX | UNIT_END | TRIM_BEFORE_SIGN => new CalendarInterval(months, days, microseconds) - case TRIM_BEFORE_VALUE => throw new SparkIllegalArgumentException( - errorClass = "INVALID_INTERVAL_FORMAT.MISSING_NUMBER", - messageParameters = Map( - "input" -> input.toString, - "word" -> currentWord)) + case TRIM_BEFORE_VALUE => + throw new SparkIllegalArgumentException( + errorClass = "INVALID_INTERVAL_FORMAT.MISSING_NUMBER", + messageParameters = Map("input" -> input.toString, "word" -> currentWord)) case VALUE | VALUE_FRACTIONAL_PART => throw new SparkIllegalArgumentException( errorClass = "INVALID_INTERVAL_FORMAT.MISSING_UNIT", - messageParameters = Map( - "input" -> input.toString, - "word" -> currentWord)) - case _ => throw new SparkIllegalArgumentException( - errorClass = "INVALID_INTERVAL_FORMAT.UNKNOWN_PARSING_ERROR", - messageParameters = Map( - "input" -> input.toString, - "word" -> currentWord)) + messageParameters = Map("input" -> input.toString, "word" -> currentWord)) + case _ => + throw new SparkIllegalArgumentException( + errorClass = "INVALID_INTERVAL_FORMAT.UNKNOWN_PARSING_ERROR", + messageParameters = Map("input" -> input.toString, "word" -> currentWord)) } result } /** - * Converts an year-month interval as a number of months to its textual representation - * which conforms to the ANSI SQL standard. + * Converts an year-month interval as a number of months to its textual representation which + * conforms to the ANSI SQL standard. * - * @param months The number of months, positive or negative - * @param style The style of textual representation of the interval - * @param startField The start field (YEAR or MONTH) which the interval comprises of. - * @param endField The end field (YEAR or MONTH) which the interval comprises of. - * @return Year-month interval string + * @param months + * The number of months, positive or negative + * @param style + * The style of textual representation of the interval + * @param startField + * The start field (YEAR or MONTH) which the interval comprises of. + * @param endField + * The end field (YEAR or MONTH) which the interval comprises of. + * @return + * Year-month interval string */ def toYearMonthIntervalString( months: Int, @@ -434,14 +418,19 @@ trait SparkIntervalUtils { } /** - * Converts a day-time interval as a number of microseconds to its textual representation - * which conforms to the ANSI SQL standard. + * Converts a day-time interval as a number of microseconds to its textual representation which + * conforms to the ANSI SQL standard. * - * @param micros The number of microseconds, positive or negative - * @param style The style of textual representation of the interval - * @param startField The start field (DAY, HOUR, MINUTE, SECOND) which the interval comprises of. - * @param endField The end field (DAY, HOUR, MINUTE, SECOND) which the interval comprises of. - * @return Day-time interval string + * @param micros + * The number of microseconds, positive or negative + * @param style + * The style of textual representation of the interval + * @param startField + * The start field (DAY, HOUR, MINUTE, SECOND) which the interval comprises of. + * @param endField + * The end field (DAY, HOUR, MINUTE, SECOND) which the interval comprises of. + * @return + * Day-time interval string */ def toDayTimeIntervalString( micros: Long, @@ -514,8 +503,9 @@ trait SparkIntervalUtils { rest %= MICROS_PER_MINUTE case DT.SECOND => val leadZero = if (rest < 10 * MICROS_PER_SECOND) "0" else "" - formatBuilder.append(s"$leadZero" + - s"${java.math.BigDecimal.valueOf(rest, 6).stripTrailingZeros.toPlainString}") + formatBuilder.append( + s"$leadZero" + + s"${java.math.BigDecimal.valueOf(rest, 6).stripTrailingZeros.toPlainString}") } if (startField < DT.HOUR && DT.HOUR <= endField) { @@ -565,20 +555,11 @@ trait SparkIntervalUtils { protected val microsStr: UTF8String = unitToUtf8("microsecond") protected val nanosStr: UTF8String = unitToUtf8("nanosecond") - private object ParseState extends Enumeration { type ParseState = Value - val PREFIX, - TRIM_BEFORE_SIGN, - SIGN, - TRIM_BEFORE_VALUE, - VALUE, - VALUE_FRACTIONAL_PART, - TRIM_BEFORE_UNIT, - UNIT_BEGIN, - UNIT_SUFFIX, - UNIT_END = Value + val PREFIX, TRIM_BEFORE_SIGN, SIGN, TRIM_BEFORE_VALUE, VALUE, VALUE_FRACTIONAL_PART, + TRIM_BEFORE_UNIT, UNIT_BEGIN, UNIT_SUFFIX, UNIT_END = Value } } diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkParserUtils.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkParserUtils.scala index 7597cb1d9087d..01ee899085701 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkParserUtils.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkParserUtils.scala @@ -62,8 +62,8 @@ trait SparkParserUtils { val secondChar = s.charAt(start + 1) val thirdChar = s.charAt(start + 2) (firstChar == '0' || firstChar == '1') && - (secondChar >= '0' && secondChar <= '7') && - (thirdChar >= '0' && thirdChar <= '7') + (secondChar >= '0' && secondChar <= '7') && + (thirdChar >= '0' && thirdChar <= '7') } val isRawString = { @@ -97,15 +97,18 @@ trait SparkParserUtils { // \u0000 style 16-bit unicode character literals. sb.append(Integer.parseInt(b, i + 1, i + 1 + 4, 16).toChar) i += 1 + 4 - } else if (cAfterBackslash == 'U' && i + 1 + 8 <= length && allCharsAreHex(b, i + 1, 8)) { + } else if (cAfterBackslash == 'U' && i + 1 + 8 <= length && allCharsAreHex( + b, + i + 1, + 8)) { // \U00000000 style 32-bit unicode character literals. // Use Long to treat codePoint as unsigned in the range of 32-bit. val codePoint = JLong.parseLong(b, i + 1, i + 1 + 8, 16) if (codePoint < 0x10000) { - sb.append((codePoint & 0xFFFF).toChar) + sb.append((codePoint & 0xffff).toChar) } else { - val highSurrogate = (codePoint - 0x10000) / 0x400 + 0xD800 - val lowSurrogate = (codePoint - 0x10000) % 0x400 + 0xDC00 + val highSurrogate = (codePoint - 0x10000) / 0x400 + 0xd800 + val lowSurrogate = (codePoint - 0x10000) % 0x400 + 0xdc00 sb.append(highSurrogate.toChar) sb.append(lowSurrogate.toChar) } @@ -147,8 +150,13 @@ trait SparkParserUtils { if (text.isEmpty) { CurrentOrigin.set(position(ctx.getStart)) } else { - CurrentOrigin.set(positionAndText(ctx.getStart, ctx.getStop, text.get, - current.objectType, current.objectName)) + CurrentOrigin.set( + positionAndText( + ctx.getStart, + ctx.getStop, + text.get, + current.objectType, + current.objectName)) } try { f diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala index edb1ee371b156..0608322be13b3 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala @@ -24,9 +24,8 @@ import org.apache.spark.unsafe.array.ByteArrayUtils import org.apache.spark.util.ArrayImplicits._ /** - * Concatenation of sequence of strings to final string with cheap append method - * and one memory allocation for the final string. Can also bound the final size of - * the string. + * Concatenation of sequence of strings to final string with cheap append method and one memory + * allocation for the final string. Can also bound the final size of the string. */ class StringConcat(val maxLength: Int = ByteArrayUtils.MAX_ROUNDED_ARRAY_LENGTH) { protected val strings = new java.util.ArrayList[String] @@ -35,9 +34,9 @@ class StringConcat(val maxLength: Int = ByteArrayUtils.MAX_ROUNDED_ARRAY_LENGTH) def atLimit: Boolean = length >= maxLength /** - * Appends a string and accumulates its length to allocate a string buffer for all - * appended strings once in the toString method. Returns true if the string still - * has room for further appends before it hits its max limit. + * Appends a string and accumulates its length to allocate a string buffer for all appended + * strings once in the toString method. Returns true if the string still has room for further + * appends before it hits its max limit. */ def append(s: String): Unit = { if (s != null) { @@ -56,8 +55,8 @@ class StringConcat(val maxLength: Int = ByteArrayUtils.MAX_ROUNDED_ARRAY_LENGTH) } /** - * The method allocates memory for all appended strings, writes them to the memory and - * returns concatenated string. + * The method allocates memory for all appended strings, writes them to the memory and returns + * concatenated string. */ override def toString: String = { val finalLength = if (atLimit) maxLength else length @@ -68,6 +67,7 @@ class StringConcat(val maxLength: Int = ByteArrayUtils.MAX_ROUNDED_ARRAY_LENGTH) } object SparkStringUtils extends Logging { + /** Whether we have warned about plan string truncation yet. */ private val truncationWarningPrinted = new AtomicBoolean(false) @@ -75,7 +75,8 @@ object SparkStringUtils extends Logging { * Format a sequence with semantics similar to calling .mkString(). Any elements beyond * maxNumToStringFields will be dropped and replaced by a "... N more fields" placeholder. * - * @return the trimmed and formatted string. + * @return + * the trimmed and formatted string. */ def truncatedString[T]( seq: Seq[T], @@ -90,8 +91,9 @@ object SparkStringUtils extends Logging { s"behavior can be adjusted by setting 'spark.sql.debug.maxToStringFields'.") } val numFields = math.max(0, maxFields - 1) - seq.take(numFields).mkString( - start, sep, sep + "... " + (seq.length - numFields) + " more fields" + end) + seq + .take(numFields) + .mkString(start, sep, sep + "... " + (seq.length - numFields) + " more fields" + end) } else { seq.mkString(start, sep, end) } @@ -106,8 +108,8 @@ object SparkStringUtils extends Logging { HexFormat.of().withDelimiter(" ").withUpperCase() /** - * Returns a pretty string of the byte array which prints each byte as a hex digit and add spaces - * between them. For example, [1A C0]. + * Returns a pretty string of the byte array which prints each byte as a hex digit and add + * spaces between them. For example, [1A C0]. */ def getHexString(bytes: Array[Byte]): String = { s"[${SPACE_DELIMITED_UPPERCASE_HEX.formatHex(bytes)}]" @@ -122,8 +124,8 @@ object SparkStringUtils extends Logging { val leftPadded = left ++ Seq.fill(math.max(right.size - left.size, 0))("") val rightPadded = right ++ Seq.fill(math.max(left.size - right.size, 0))("") - leftPadded.zip(rightPadded).map { - case (l, r) => (if (l == r) " " else "!") + l + (" " * ((maxLeftSize - l.length) + 3)) + r + leftPadded.zip(rightPadded).map { case (l, r) => + (if (l == r) " " else "!") + l + (" " * ((maxLeftSize - l.length) + 3)) + r } } } diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala index 79d627b493fd8..4fcb84daf187d 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala @@ -41,14 +41,20 @@ import org.apache.spark.sql.types.{Decimal, TimestampNTZType} import org.apache.spark.unsafe.types.UTF8String sealed trait TimestampFormatter extends Serializable { + /** * Parses a timestamp in a string and converts it to microseconds. * - * @param s - string with timestamp to parse - * @return microseconds since epoch. - * @throws ParseException can be thrown by legacy parser - * @throws DateTimeParseException can be thrown by new parser - * @throws DateTimeException unable to obtain local date or time + * @param s + * \- string with timestamp to parse + * @return + * microseconds since epoch. + * @throws ParseException + * can be thrown by legacy parser + * @throws DateTimeParseException + * can be thrown by new parser + * @throws DateTimeException + * unable to obtain local date or time */ @throws(classOf[ParseException]) @throws(classOf[DateTimeParseException]) @@ -58,11 +64,16 @@ sealed trait TimestampFormatter extends Serializable { /** * Parses a timestamp in a string and converts it to an optional number of microseconds. * - * @param s - string with timestamp to parse - * @return An optional number of microseconds since epoch. The result is None on invalid input. - * @throws ParseException can be thrown by legacy parser - * @throws DateTimeParseException can be thrown by new parser - * @throws DateTimeException unable to obtain local date or time + * @param s + * \- string with timestamp to parse + * @return + * An optional number of microseconds since epoch. The result is None on invalid input. + * @throws ParseException + * can be thrown by legacy parser + * @throws DateTimeParseException + * can be thrown by new parser + * @throws DateTimeException + * unable to obtain local date or time */ @throws(classOf[ParseException]) @throws(classOf[DateTimeParseException]) @@ -75,16 +86,24 @@ sealed trait TimestampFormatter extends Serializable { } /** - * Parses a timestamp in a string and converts it to microseconds since Unix Epoch in local time. + * Parses a timestamp in a string and converts it to microseconds since Unix Epoch in local + * time. * - * @param s - string with timestamp to parse - * @param allowTimeZone - indicates strict parsing of timezone - * @return microseconds since epoch. - * @throws ParseException can be thrown by legacy parser - * @throws DateTimeParseException can be thrown by new parser - * @throws DateTimeException unable to obtain local date or time - * @throws IllegalStateException The formatter for timestamp without time zone should always - * implement this method. The exception should never be hit. + * @param s + * \- string with timestamp to parse + * @param allowTimeZone + * \- indicates strict parsing of timezone + * @return + * microseconds since epoch. + * @throws ParseException + * can be thrown by legacy parser + * @throws DateTimeParseException + * can be thrown by new parser + * @throws DateTimeException + * unable to obtain local date or time + * @throws IllegalStateException + * The formatter for timestamp without time zone should always implement this method. The + * exception should never be hit. */ @throws(classOf[ParseException]) @throws(classOf[DateTimeParseException]) @@ -99,14 +118,21 @@ sealed trait TimestampFormatter extends Serializable { * Parses a timestamp in a string and converts it to an optional number of microseconds since * Unix Epoch in local time. * - * @param s - string with timestamp to parse - * @param allowTimeZone - indicates strict parsing of timezone - * @return An optional number of microseconds since epoch. The result is None on invalid input. - * @throws ParseException can be thrown by legacy parser - * @throws DateTimeParseException can be thrown by new parser - * @throws DateTimeException unable to obtain local date or time - * @throws IllegalStateException The formatter for timestamp without time zone should always - * implement this method. The exception should never be hit. + * @param s + * \- string with timestamp to parse + * @param allowTimeZone + * \- indicates strict parsing of timezone + * @return + * An optional number of microseconds since epoch. The result is None on invalid input. + * @throws ParseException + * can be thrown by legacy parser + * @throws DateTimeParseException + * can be thrown by new parser + * @throws DateTimeException + * unable to obtain local date or time + * @throws IllegalStateException + * The formatter for timestamp without time zone should always implement this method. The + * exception should never be hit. */ @throws(classOf[ParseException]) @throws(classOf[DateTimeParseException]) @@ -120,8 +146,8 @@ sealed trait TimestampFormatter extends Serializable { } /** - * Parses a timestamp in a string and converts it to microseconds since Unix Epoch in local time. - * Zone-id and zone-offset components are ignored. + * Parses a timestamp in a string and converts it to microseconds since Unix Epoch in local + * time. Zone-id and zone-offset components are ignored. */ @throws(classOf[ParseException]) @throws(classOf[DateTimeParseException]) @@ -144,9 +170,10 @@ sealed trait TimestampFormatter extends Serializable { /** * Validates the pattern string. - * @param checkLegacy if true and the pattern is invalid, check whether the pattern is valid for - * legacy formatters and show hints for using legacy formatter. - * Otherwise, simply check the pattern string. + * @param checkLegacy + * if true and the pattern is invalid, check whether the pattern is valid for legacy + * formatters and show hints for using legacy formatter. Otherwise, simply check the pattern + * string. */ def validatePatternString(checkLegacy: Boolean): Unit } @@ -157,7 +184,8 @@ class Iso8601TimestampFormatter( locale: Locale, legacyFormat: LegacyDateFormat = LENIENT_SIMPLE_DATE_FORMAT, isParsing: Boolean) - extends TimestampFormatter with DateTimeFormatterHelper { + extends TimestampFormatter + with DateTimeFormatterHelper { @transient protected lazy val formatter: DateTimeFormatter = getOrCreateFormatter(pattern, locale, isParsing) @@ -166,8 +194,8 @@ class Iso8601TimestampFormatter( private lazy val zonedFormatter: DateTimeFormatter = formatter.withZone(zoneId) @transient - protected lazy val legacyFormatter = TimestampFormatter.getLegacyFormatter( - pattern, zoneId, locale, legacyFormat) + protected lazy val legacyFormatter = + TimestampFormatter.getLegacyFormatter(pattern, zoneId, locale, legacyFormat) override def parseOptional(s: String): Option[Long] = { try { @@ -235,8 +263,8 @@ class Iso8601TimestampFormatter( override def format(instant: Instant): String = { try { zonedFormatter.format(instant) - } catch checkFormattedDiff(toJavaTimestamp(instantToMicros(instant)), - (t: Timestamp) => format(t)) + } catch + checkFormattedDiff(toJavaTimestamp(instantToMicros(instant)), (t: Timestamp) => format(t)) } override def format(us: Long): String = { @@ -256,8 +284,8 @@ class Iso8601TimestampFormatter( if (checkLegacy) { try { formatter - } catch checkLegacyFormatter(pattern, - legacyFormatter.validatePatternString(checkLegacy = true)) + } catch + checkLegacyFormatter(pattern, legacyFormatter.validatePatternString(checkLegacy = true)) () } else { try { @@ -268,22 +296,30 @@ class Iso8601TimestampFormatter( } /** - * The formatter for timestamps which doesn't require users to specify a pattern. While formatting, - * it uses the default pattern [[TimestampFormatter.defaultPattern()]]. In parsing, it follows - * the CAST logic in conversion of strings to Catalyst's TimestampType. + * The formatter for timestamps which doesn't require users to specify a pattern. While + * formatting, it uses the default pattern [[TimestampFormatter.defaultPattern()]]. In parsing, it + * follows the CAST logic in conversion of strings to Catalyst's TimestampType. * - * @param zoneId The time zone ID in which timestamps should be formatted or parsed. - * @param locale The locale overrides the system locale and is used in formatting. - * @param legacyFormat Defines the formatter used for legacy timestamps. - * @param isParsing Whether the formatter is used for parsing (`true`) or for formatting (`false`). + * @param zoneId + * The time zone ID in which timestamps should be formatted or parsed. + * @param locale + * The locale overrides the system locale and is used in formatting. + * @param legacyFormat + * Defines the formatter used for legacy timestamps. + * @param isParsing + * Whether the formatter is used for parsing (`true`) or for formatting (`false`). */ class DefaultTimestampFormatter( zoneId: ZoneId, locale: Locale, legacyFormat: LegacyDateFormat = LENIENT_SIMPLE_DATE_FORMAT, isParsing: Boolean) - extends Iso8601TimestampFormatter( - TimestampFormatter.defaultPattern(), zoneId, locale, legacyFormat, isParsing) { + extends Iso8601TimestampFormatter( + TimestampFormatter.defaultPattern(), + zoneId, + locale, + legacyFormat, + isParsing) { override def parse(s: String): Long = { try { @@ -299,7 +335,9 @@ class DefaultTimestampFormatter( val utf8Value = UTF8String.fromString(s) SparkDateTimeUtils.stringToTimestampWithoutTimeZone(utf8Value, allowTimeZone).getOrElse { throw ExecutionErrors.cannotParseStringAsDataTypeError( - TimestampFormatter.defaultPattern(), s, TimestampNTZType) + TimestampFormatter.defaultPattern(), + s, + TimestampNTZType) } } catch checkParsedDiff(s, legacyFormatter.parse) } @@ -311,20 +349,21 @@ class DefaultTimestampFormatter( } /** - * The formatter parses/formats timestamps according to the pattern `yyyy-MM-dd HH:mm:ss.[..fff..]` - * where `[..fff..]` is a fraction of second up to microsecond resolution. The formatter does not - * output trailing zeros in the fraction. For example, the timestamp `2019-03-05 15:00:01.123400` is - * formatted as the string `2019-03-05 15:00:01.1234`. + * The formatter parses/formats timestamps according to the pattern `yyyy-MM-dd + * HH:mm:ss.[..fff..]` where `[..fff..]` is a fraction of second up to microsecond resolution. The + * formatter does not output trailing zeros in the fraction. For example, the timestamp + * `2019-03-05 15:00:01.123400` is formatted as the string `2019-03-05 15:00:01.1234`. * - * @param zoneId the time zone identifier in which the formatter parses or format timestamps + * @param zoneId + * the time zone identifier in which the formatter parses or format timestamps */ class FractionTimestampFormatter(zoneId: ZoneId) - extends Iso8601TimestampFormatter( - TimestampFormatter.defaultPattern(), - zoneId, - TimestampFormatter.defaultLocale, - LegacyDateFormats.FAST_DATE_FORMAT, - isParsing = false) { + extends Iso8601TimestampFormatter( + TimestampFormatter.defaultPattern(), + zoneId, + TimestampFormatter.defaultLocale, + LegacyDateFormats.FAST_DATE_FORMAT, + isParsing = false) { @transient override protected lazy val formatter = DateTimeFormatterHelper.fractionFormatter @@ -366,16 +405,14 @@ class FractionTimestampFormatter(zoneId: ZoneId) } /** - * The custom sub-class of `GregorianCalendar` is needed to get access to - * protected `fields` immediately after parsing. We cannot use - * the `get()` method because it performs normalization of the fraction - * part. Accordingly, the `MILLISECOND` field doesn't contain original value. + * The custom sub-class of `GregorianCalendar` is needed to get access to protected `fields` + * immediately after parsing. We cannot use the `get()` method because it performs normalization + * of the fraction part. Accordingly, the `MILLISECOND` field doesn't contain original value. * - * Also this class allows to set raw value to the `MILLISECOND` field - * directly before formatting. + * Also this class allows to set raw value to the `MILLISECOND` field directly before formatting. */ class MicrosCalendar(tz: TimeZone, digitsInFraction: Int) - extends GregorianCalendar(tz, Locale.US) { + extends GregorianCalendar(tz, Locale.US) { // Converts parsed `MILLISECOND` field to seconds fraction in microsecond precision. // For example if the fraction pattern is `SSSS` then `digitsInFraction` = 4, and // if the `MILLISECOND` field was parsed to `1234`. @@ -397,16 +434,13 @@ class MicrosCalendar(tz: TimeZone, digitsInFraction: Int) } } -class LegacyFastTimestampFormatter( - pattern: String, - zoneId: ZoneId, - locale: Locale) extends TimestampFormatter { +class LegacyFastTimestampFormatter(pattern: String, zoneId: ZoneId, locale: Locale) + extends TimestampFormatter { @transient private lazy val fastDateFormat = FastDateFormat.getInstance(pattern, TimeZone.getTimeZone(zoneId), locale) - @transient private lazy val cal = new MicrosCalendar( - fastDateFormat.getTimeZone, - fastDateFormat.getPattern.count(_ == 'S')) + @transient private lazy val cal = + new MicrosCalendar(fastDateFormat.getTimeZone, fastDateFormat.getPattern.count(_ == 'S')) override def parse(s: String): Long = { cal.clear() // Clear the calendar because it can be re-used many times @@ -464,7 +498,8 @@ class LegacySimpleTimestampFormatter( pattern: String, zoneId: ZoneId, locale: Locale, - lenient: Boolean = true) extends TimestampFormatter { + lenient: Boolean = true) + extends TimestampFormatter { @transient private lazy val sdf = { val formatter = new SimpleDateFormat(pattern, locale) formatter.setTimeZone(TimeZone.getTimeZone(zoneId)) @@ -586,14 +621,10 @@ object TimestampFormatter { legacyFormat: LegacyDateFormat, isParsing: Boolean, forTimestampNTZ: Boolean): TimestampFormatter = { - getFormatter(Some(format), zoneId, defaultLocale, legacyFormat, isParsing, - forTimestampNTZ) + getFormatter(Some(format), zoneId, defaultLocale, legacyFormat, isParsing, forTimestampNTZ) } - def apply( - format: String, - zoneId: ZoneId, - isParsing: Boolean): TimestampFormatter = { + def apply(format: String, zoneId: ZoneId, isParsing: Boolean): TimestampFormatter = { getFormatter(Some(format), zoneId, isParsing = isParsing) } diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/UDTUtils.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/UDTUtils.scala index a98aa26d02ef7..73ab43f04a5a0 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/UDTUtils.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/UDTUtils.scala @@ -28,11 +28,12 @@ import org.apache.spark.util.SparkClassUtils * with catalyst because they (amongst others) require access to Spark SQLs internal data * representation. * - * This interface and its companion object provide an escape hatch for working with UDTs from within - * the api project (e.g. Row.toJSON). The companion will try to bind to an implementation of the - * interface in catalyst, if none is found it will bind to [[DefaultUDTUtils]]. + * This interface and its companion object provide an escape hatch for working with UDTs from + * within the api project (e.g. Row.toJSON). The companion will try to bind to an implementation + * of the interface in catalyst, if none is found it will bind to [[DefaultUDTUtils]]. */ private[sql] trait UDTUtils { + /** * Convert the UDT instance to something that is compatible with [[org.apache.spark.sql.Row]]. * The returned value must conform to the schema of the UDT. @@ -41,13 +42,14 @@ private[sql] trait UDTUtils { } private[sql] object UDTUtils extends UDTUtils { - private val delegate = try { - val cls = SparkClassUtils.classForName("org.apache.spark.sql.catalyst.util.UDTUtilsImpl") - cls.getConstructor().newInstance().asInstanceOf[UDTUtils] - } catch { - case NonFatal(_) => - DefaultUDTUtils - } + private val delegate = + try { + val cls = SparkClassUtils.classForName("org.apache.spark.sql.catalyst.util.UDTUtilsImpl") + cls.getConstructor().newInstance().asInstanceOf[UDTUtils] + } catch { + case NonFatal(_) => + DefaultUDTUtils + } override def toRow(value: Any, udt: UserDefinedType[Any]): Any = delegate.toRow(value, udt) } diff --git a/sql/api/src/main/scala/org/apache/spark/sql/errors/CompilationErrors.scala b/sql/api/src/main/scala/org/apache/spark/sql/errors/CompilationErrors.scala index 6034c41906313..3e63b8281f739 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/errors/CompilationErrors.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/errors/CompilationErrors.scala @@ -23,9 +23,7 @@ private[sql] trait CompilationErrors extends DataTypeErrorsBase { def ambiguousColumnOrFieldError(name: Seq[String], numMatches: Int): AnalysisException = { new AnalysisException( errorClass = "AMBIGUOUS_COLUMN_OR_FIELD", - messageParameters = Map( - "name" -> toSQLId(name), - "n" -> numMatches.toString)) + messageParameters = Map("name" -> toSQLId(name), "n" -> numMatches.toString)) } def columnNotFoundError(colName: String): AnalysisException = { @@ -51,9 +49,7 @@ private[sql] trait CompilationErrors extends DataTypeErrorsBase { } def usingUntypedScalaUDFError(): Throwable = { - new AnalysisException( - errorClass = "UNTYPED_SCALA_UDF", - messageParameters = Map.empty) + new AnalysisException(errorClass = "UNTYPED_SCALA_UDF", messageParameters = Map.empty) } def invalidBoundaryStartError(start: Long): Throwable = { @@ -81,14 +77,11 @@ private[sql] trait CompilationErrors extends DataTypeErrorsBase { def invalidSaveModeError(saveMode: String): Throwable = { new AnalysisException( errorClass = "INVALID_SAVE_MODE", - messageParameters = Map("mode" -> toDSOption(saveMode)) - ) + messageParameters = Map("mode" -> toDSOption(saveMode))) } def sortByWithoutBucketingError(): Throwable = { - new AnalysisException( - errorClass = "SORT_BY_WITHOUT_BUCKETING", - messageParameters = Map.empty) + new AnalysisException(errorClass = "SORT_BY_WITHOUT_BUCKETING", messageParameters = Map.empty) } def bucketByUnsupportedByOperationError(operation: String): Throwable = { diff --git a/sql/api/src/main/scala/org/apache/spark/sql/errors/DataTypeErrors.scala b/sql/api/src/main/scala/org/apache/spark/sql/errors/DataTypeErrors.scala index 32b4198dc1a63..fcd8440d46a61 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/errors/DataTypeErrors.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/errors/DataTypeErrors.scala @@ -25,9 +25,9 @@ import org.apache.spark.sql.types.{DataType, Decimal, StringType} import org.apache.spark.unsafe.types.UTF8String /** - * Object for grouping error messages from (most) exceptions thrown during query execution. - * This does not include exceptions thrown during the eager execution of commands, which are - * grouped into [[CompilationErrors]]. + * Object for grouping error messages from (most) exceptions thrown during query execution. This + * does not include exceptions thrown during the eager execution of commands, which are grouped + * into [[CompilationErrors]]. */ private[sql] object DataTypeErrors extends DataTypeErrorsBase { def unsupportedOperationExceptionError(): SparkUnsupportedOperationException = { @@ -35,13 +35,12 @@ private[sql] object DataTypeErrors extends DataTypeErrorsBase { } def decimalPrecisionExceedsMaxPrecisionError( - precision: Int, maxPrecision: Int): SparkArithmeticException = { + precision: Int, + maxPrecision: Int): SparkArithmeticException = { new SparkArithmeticException( errorClass = "DECIMAL_PRECISION_EXCEEDS_MAX_PRECISION", - messageParameters = Map( - "precision" -> precision.toString, - "maxPrecision" -> maxPrecision.toString - ), + messageParameters = + Map("precision" -> precision.toString, "maxPrecision" -> maxPrecision.toString), context = Array.empty, summary = "") } @@ -53,8 +52,7 @@ private[sql] object DataTypeErrors extends DataTypeErrorsBase { def outOfDecimalTypeRangeError(str: UTF8String): SparkArithmeticException = { new SparkArithmeticException( errorClass = "NUMERIC_OUT_OF_SUPPORTED_RANGE", - messageParameters = Map( - "value" -> str.toString), + messageParameters = Map("value" -> str.toString), context = Array.empty, summary = "") } @@ -68,25 +66,20 @@ private[sql] object DataTypeErrors extends DataTypeErrorsBase { def nullLiteralsCannotBeCastedError(name: String): SparkUnsupportedOperationException = { new SparkUnsupportedOperationException( errorClass = "_LEGACY_ERROR_TEMP_2226", - messageParameters = Map( - "name" -> name)) + messageParameters = Map("name" -> name)) } def notUserDefinedTypeError(name: String, userClass: String): Throwable = { new SparkException( errorClass = "_LEGACY_ERROR_TEMP_2227", - messageParameters = Map( - "name" -> name, - "userClass" -> userClass), + messageParameters = Map("name" -> name, "userClass" -> userClass), cause = null) } def cannotLoadUserDefinedTypeError(name: String, userClass: String): Throwable = { new SparkException( errorClass = "_LEGACY_ERROR_TEMP_2228", - messageParameters = Map( - "name" -> name, - "userClass" -> userClass), + messageParameters = Map("name" -> name, "userClass" -> userClass), cause = null) } @@ -99,50 +92,42 @@ private[sql] object DataTypeErrors extends DataTypeErrorsBase { def schemaFailToParseError(schema: String, e: Throwable): Throwable = { new AnalysisException( errorClass = "INVALID_SCHEMA.PARSE_ERROR", - messageParameters = Map( - "inputSchema" -> toSQLSchema(schema), - "reason" -> e.getMessage - ), + messageParameters = Map("inputSchema" -> toSQLSchema(schema), "reason" -> e.getMessage), cause = Some(e)) } def invalidDayTimeIntervalType(startFieldName: String, endFieldName: String): Throwable = { new AnalysisException( errorClass = "_LEGACY_ERROR_TEMP_1224", - messageParameters = Map( - "startFieldName" -> startFieldName, - "endFieldName" -> endFieldName)) + messageParameters = Map("startFieldName" -> startFieldName, "endFieldName" -> endFieldName)) } def invalidDayTimeField(field: Byte, supportedIds: Seq[String]): Throwable = { new AnalysisException( errorClass = "_LEGACY_ERROR_TEMP_1223", - messageParameters = Map( - "field" -> field.toString, - "supportedIds" -> supportedIds.mkString(", "))) + messageParameters = + Map("field" -> field.toString, "supportedIds" -> supportedIds.mkString(", "))) } def invalidYearMonthField(field: Byte, supportedIds: Seq[String]): Throwable = { new AnalysisException( errorClass = "_LEGACY_ERROR_TEMP_1225", - messageParameters = Map( - "field" -> field.toString, - "supportedIds" -> supportedIds.mkString(", "))) + messageParameters = + Map("field" -> field.toString, "supportedIds" -> supportedIds.mkString(", "))) } def decimalCannotGreaterThanPrecisionError(scale: Int, precision: Int): Throwable = { new AnalysisException( errorClass = "_LEGACY_ERROR_TEMP_1228", - messageParameters = Map( - "scale" -> scale.toString, - "precision" -> precision.toString)) + messageParameters = Map("scale" -> scale.toString, "precision" -> precision.toString)) } def negativeScaleNotAllowedError(scale: Int): Throwable = { val sqlConf = QuotingUtils.toSQLConf("spark.sql.legacy.allowNegativeScaleOfDecimal") - SparkException.internalError(s"Negative scale is not allowed: ${scale.toString}." + - s" Set the config ${sqlConf}" + - " to \"true\" to allow it.") + SparkException.internalError( + s"Negative scale is not allowed: ${scale.toString}." + + s" Set the config ${sqlConf}" + + " to \"true\" to allow it.") } def attributeNameSyntaxError(name: String): Throwable = { @@ -154,19 +139,17 @@ private[sql] object DataTypeErrors extends DataTypeErrorsBase { def cannotMergeIncompatibleDataTypesError(left: DataType, right: DataType): Throwable = { new SparkException( errorClass = "CANNOT_MERGE_INCOMPATIBLE_DATA_TYPE", - messageParameters = Map( - "left" -> toSQLType(left), - "right" -> toSQLType(right)), + messageParameters = Map("left" -> toSQLType(left), "right" -> toSQLType(right)), cause = null) } def cannotMergeDecimalTypesWithIncompatibleScaleError( - leftScale: Int, rightScale: Int): Throwable = { + leftScale: Int, + rightScale: Int): Throwable = { new SparkException( errorClass = "_LEGACY_ERROR_TEMP_2124", - messageParameters = Map( - "leftScale" -> leftScale.toString(), - "rightScale" -> rightScale.toString()), + messageParameters = + Map("leftScale" -> leftScale.toString(), "rightScale" -> rightScale.toString()), cause = null) } @@ -179,9 +162,7 @@ private[sql] object DataTypeErrors extends DataTypeErrorsBase { def invalidFieldName(fieldName: Seq[String], path: Seq[String], context: Origin): Throwable = { new AnalysisException( errorClass = "INVALID_FIELD_NAME", - messageParameters = Map( - "fieldName" -> toSQLId(fieldName), - "path" -> toSQLId(path)), + messageParameters = Map("fieldName" -> toSQLId(fieldName), "path" -> toSQLId(path)), origin = context) } @@ -234,12 +215,12 @@ private[sql] object DataTypeErrors extends DataTypeErrorsBase { } def ambiguousColumnOrFieldError( - name: Seq[String], numMatches: Int, context: Origin): Throwable = { + name: Seq[String], + numMatches: Int, + context: Origin): Throwable = { new AnalysisException( errorClass = "AMBIGUOUS_COLUMN_OR_FIELD", - messageParameters = Map( - "name" -> toSQLId(name), - "n" -> numMatches.toString), + messageParameters = Map("name" -> toSQLId(name), "n" -> numMatches.toString), origin = context) } @@ -267,15 +248,13 @@ private[sql] object DataTypeErrors extends DataTypeErrorsBase { messageParameters = Map( "methodName" -> "fieldIndex", "className" -> "Row", - "fieldName" -> toSQLId(fieldName)) - ) + "fieldName" -> toSQLId(fieldName))) } def valueIsNullError(index: Int): Throwable = { new SparkRuntimeException( errorClass = "ROW_VALUE_IS_NULL", - messageParameters = Map( - "index" -> index.toString), + messageParameters = Map("index" -> index.toString), cause = null) } diff --git a/sql/api/src/main/scala/org/apache/spark/sql/errors/ExecutionErrors.scala b/sql/api/src/main/scala/org/apache/spark/sql/errors/ExecutionErrors.scala index d83b81d1bf525..99fffff678a79 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/errors/ExecutionErrors.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/errors/ExecutionErrors.scala @@ -60,7 +60,8 @@ private[sql] trait ExecutionErrors extends DataTypeErrorsBase { } def failToRecognizePatternAfterUpgradeError( - pattern: String, e: Throwable): SparkUpgradeException = { + pattern: String, + e: Throwable): SparkUpgradeException = { new SparkUpgradeException( errorClass = "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION", messageParameters = Map( @@ -73,9 +74,8 @@ private[sql] trait ExecutionErrors extends DataTypeErrorsBase { def failToRecognizePatternError(pattern: String, e: Throwable): SparkRuntimeException = { new SparkRuntimeException( errorClass = "_LEGACY_ERROR_TEMP_2130", - messageParameters = Map( - "pattern" -> toSQLValue(pattern), - "docroot" -> SparkBuildInfo.spark_doc_root), + messageParameters = + Map("pattern" -> toSQLValue(pattern), "docroot" -> SparkBuildInfo.spark_doc_root), cause = e) } @@ -93,9 +93,9 @@ private[sql] trait ExecutionErrors extends DataTypeErrorsBase { } def invalidInputInCastToDatetimeError( - value: Double, - to: DataType, - context: QueryContext): SparkDateTimeException = { + value: Double, + to: DataType, + context: QueryContext): SparkDateTimeException = { invalidInputInCastToDatetimeErrorInternal(toSQLValue(value), DoubleType, to, context) } @@ -132,8 +132,10 @@ private[sql] trait ExecutionErrors extends DataTypeErrorsBase { summary = getSummary(context)) } - def cannotParseStringAsDataTypeError(pattern: String, value: String, dataType: DataType) - : Throwable = { + def cannotParseStringAsDataTypeError( + pattern: String, + value: String, + dataType: DataType): Throwable = { SparkException.internalError( s"Cannot parse field value ${toSQLValue(value)} for pattern ${toSQLValue(pattern)} " + s"as the target spark data type ${toSQLType(dataType)}.") @@ -161,17 +163,14 @@ private[sql] trait ExecutionErrors extends DataTypeErrorsBase { def userDefinedTypeNotAnnotatedAndRegisteredError(udt: UserDefinedType[_]): Throwable = { new SparkException( errorClass = "_LEGACY_ERROR_TEMP_2155", - messageParameters = Map( - "userClass" -> udt.userClass.getName), + messageParameters = Map("userClass" -> udt.userClass.getName), cause = null) } def cannotFindEncoderForTypeError(typeName: String): SparkUnsupportedOperationException = { new SparkUnsupportedOperationException( errorClass = "ENCODER_NOT_FOUND", - messageParameters = Map( - "typeName" -> typeName, - "docroot" -> SparkBuildInfo.spark_doc_root)) + messageParameters = Map("typeName" -> typeName, "docroot" -> SparkBuildInfo.spark_doc_root)) } def cannotHaveCircularReferencesInBeanClassError( @@ -184,8 +183,7 @@ private[sql] trait ExecutionErrors extends DataTypeErrorsBase { def cannotFindConstructorForTypeError(tpe: String): SparkUnsupportedOperationException = { new SparkUnsupportedOperationException( errorClass = "_LEGACY_ERROR_TEMP_2144", - messageParameters = Map( - "tpe" -> tpe)) + messageParameters = Map("tpe" -> tpe)) } def cannotHaveCircularReferencesInClassError(t: String): SparkUnsupportedOperationException = { @@ -195,12 +193,12 @@ private[sql] trait ExecutionErrors extends DataTypeErrorsBase { } def cannotUseInvalidJavaIdentifierAsFieldNameError( - fieldName: String, walkedTypePath: WalkedTypePath): SparkUnsupportedOperationException = { + fieldName: String, + walkedTypePath: WalkedTypePath): SparkUnsupportedOperationException = { new SparkUnsupportedOperationException( errorClass = "_LEGACY_ERROR_TEMP_2140", - messageParameters = Map( - "fieldName" -> fieldName, - "walkedTypePath" -> walkedTypePath.toString)) + messageParameters = + Map("fieldName" -> fieldName, "walkedTypePath" -> walkedTypePath.toString)) } def primaryConstructorNotFoundError(cls: Class[_]): SparkRuntimeException = { @@ -213,14 +211,11 @@ private[sql] trait ExecutionErrors extends DataTypeErrorsBase { def cannotGetOuterPointerForInnerClassError(innerCls: Class[_]): SparkRuntimeException = { new SparkRuntimeException( errorClass = "_LEGACY_ERROR_TEMP_2154", - messageParameters = Map( - "innerCls" -> innerCls.getName)) + messageParameters = Map("innerCls" -> innerCls.getName)) } def cannotUseKryoSerialization(): SparkRuntimeException = { - new SparkRuntimeException( - errorClass = "CANNOT_USE_KRYO", - messageParameters = Map.empty) + new SparkRuntimeException(errorClass = "CANNOT_USE_KRYO", messageParameters = Map.empty) } } diff --git a/sql/api/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala b/sql/api/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala index e7ae9f2bfb7bb..5f7fcb92f7bd1 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala @@ -27,8 +27,8 @@ import org.apache.spark.sql.catalyst.parser.SqlBaseParser._ import org.apache.spark.sql.catalyst.trees.Origin /** - * Object for grouping all error messages of the query parsing. - * Currently it includes all ParseException. + * Object for grouping all error messages of the query parsing. Currently it includes all + * ParseException. */ private[sql] object QueryParsingErrors extends DataTypeErrorsBase { @@ -37,9 +37,7 @@ private[sql] object QueryParsingErrors extends DataTypeErrorsBase { } def parserStackOverflow(parserRuleContext: ParserRuleContext): Throwable = { - throw new ParseException( - errorClass = "FAILED_TO_PARSE_TOO_COMPLEX", - ctx = parserRuleContext) + throw new ParseException(errorClass = "FAILED_TO_PARSE_TOO_COMPLEX", ctx = parserRuleContext) } def insertOverwriteDirectoryUnsupportedError(): Throwable = { @@ -160,7 +158,9 @@ private[sql] object QueryParsingErrors extends DataTypeErrorsBase { } def incompatibleJoinTypesError( - joinType1: String, joinType2: String, ctx: ParserRuleContext): Throwable = { + joinType1: String, + joinType2: String, + ctx: ParserRuleContext): Throwable = { new ParseException( errorClass = "INCOMPATIBLE_JOIN_TYPES", messageParameters = Map( @@ -209,13 +209,12 @@ private[sql] object QueryParsingErrors extends DataTypeErrorsBase { } def cannotParseValueTypeError( - valueType: String, value: String, ctx: TypeConstructorContext): Throwable = { + valueType: String, + value: String, + ctx: TypeConstructorContext): Throwable = { new ParseException( errorClass = "INVALID_TYPED_LITERAL", - messageParameters = Map( - "valueType" -> toSQLType(valueType), - "value" -> toSQLValue(value) - ), + messageParameters = Map("valueType" -> toSQLType(valueType), "value" -> toSQLValue(value)), ctx) } @@ -231,8 +230,12 @@ private[sql] object QueryParsingErrors extends DataTypeErrorsBase { ctx) } - def invalidNumericLiteralRangeError(rawStrippedQualifier: String, minValue: BigDecimal, - maxValue: BigDecimal, typeName: String, ctx: NumberContext): Throwable = { + def invalidNumericLiteralRangeError( + rawStrippedQualifier: String, + minValue: BigDecimal, + maxValue: BigDecimal, + typeName: String, + ctx: NumberContext): Throwable = { new ParseException( errorClass = "INVALID_NUMERIC_LITERAL_RANGE", messageParameters = Map( @@ -259,7 +262,9 @@ private[sql] object QueryParsingErrors extends DataTypeErrorsBase { } def fromToIntervalUnsupportedError( - from: String, to: String, ctx: ParserRuleContext): Throwable = { + from: String, + to: String, + ctx: ParserRuleContext): Throwable = { new ParseException( errorClass = "_LEGACY_ERROR_TEMP_0028", messageParameters = Map("from" -> from, "to" -> to), @@ -288,7 +293,8 @@ private[sql] object QueryParsingErrors extends DataTypeErrorsBase { } def nestedTypeMissingElementTypeError( - dataType: String, ctx: PrimitiveDataTypeContext): Throwable = { + dataType: String, + ctx: PrimitiveDataTypeContext): Throwable = { dataType.toUpperCase(Locale.ROOT) match { case "ARRAY" => new ParseException( @@ -309,23 +315,25 @@ private[sql] object QueryParsingErrors extends DataTypeErrorsBase { } def partitionTransformNotExpectedError( - name: String, expr: String, ctx: ApplyTransformContext): Throwable = { + name: String, + expr: String, + ctx: ApplyTransformContext): Throwable = { new ParseException( errorClass = "INVALID_SQL_SYNTAX.INVALID_COLUMN_REFERENCE", - messageParameters = Map( - "transform" -> toSQLId(name), - "expr" -> expr), + messageParameters = Map("transform" -> toSQLId(name), "expr" -> expr), ctx) } def wrongNumberArgumentsForTransformError( - name: String, actualNum: Int, ctx: ApplyTransformContext): Throwable = { + name: String, + actualNum: Int, + ctx: ApplyTransformContext): Throwable = { new ParseException( errorClass = "INVALID_SQL_SYNTAX.TRANSFORM_WRONG_NUM_ARGS", messageParameters = Map( "transform" -> toSQLId(name), - "expectedNum" -> "1", - "actualNum" -> actualNum.toString), + "expectedNum" -> "1", + "actualNum" -> actualNum.toString), ctx) } @@ -337,7 +345,9 @@ private[sql] object QueryParsingErrors extends DataTypeErrorsBase { } def cannotCleanReservedNamespacePropertyError( - property: String, ctx: ParserRuleContext, msg: String): Throwable = { + property: String, + ctx: ParserRuleContext, + msg: String): Throwable = { new ParseException( errorClass = "UNSUPPORTED_FEATURE.SET_NAMESPACE_PROPERTY", messageParameters = Map("property" -> property, "msg" -> msg), @@ -348,12 +358,13 @@ private[sql] object QueryParsingErrors extends DataTypeErrorsBase { new ParseException( errorClass = "UNSUPPORTED_FEATURE.SET_PROPERTIES_AND_DBPROPERTIES", messageParameters = Map.empty, - ctx - ) + ctx) } def cannotCleanReservedTablePropertyError( - property: String, ctx: ParserRuleContext, msg: String): Throwable = { + property: String, + ctx: ParserRuleContext, + msg: String): Throwable = { new ParseException( errorClass = "UNSUPPORTED_FEATURE.SET_TABLE_PROPERTY", messageParameters = Map("property" -> property, "msg" -> msg), @@ -361,12 +372,12 @@ private[sql] object QueryParsingErrors extends DataTypeErrorsBase { } def duplicatedTablePathsFoundError( - pathOne: String, pathTwo: String, ctx: ParserRuleContext): Throwable = { + pathOne: String, + pathTwo: String, + ctx: ParserRuleContext): Throwable = { new ParseException( errorClass = "_LEGACY_ERROR_TEMP_0032", - messageParameters = Map( - "pathOne" -> pathOne, - "pathTwo" -> pathTwo), + messageParameters = Map("pathOne" -> pathOne, "pathTwo" -> pathTwo), ctx) } @@ -374,15 +385,17 @@ private[sql] object QueryParsingErrors extends DataTypeErrorsBase { new ParseException(errorClass = "_LEGACY_ERROR_TEMP_0033", ctx) } - def operationInHiveStyleCommandUnsupportedError(operation: String, - command: String, ctx: StatementContext, msgOpt: Option[String] = None): Throwable = { + def operationInHiveStyleCommandUnsupportedError( + operation: String, + command: String, + ctx: StatementContext, + msgOpt: Option[String] = None): Throwable = { new ParseException( errorClass = "_LEGACY_ERROR_TEMP_0034", messageParameters = Map( "operation" -> operation, "command" -> command, - "msg" -> msgOpt.map(m => s", $m").getOrElse("") - ), + "msg" -> msgOpt.map(m => s", $m").getOrElse("")), ctx) } @@ -415,7 +428,8 @@ private[sql] object QueryParsingErrors extends DataTypeErrorsBase { } def addCatalogInCacheTableAsSelectNotAllowedError( - quoted: String, ctx: CacheTableContext): Throwable = { + quoted: String, + ctx: CacheTableContext): Throwable = { new ParseException( errorClass = "_LEGACY_ERROR_TEMP_0037", messageParameters = Map("quoted" -> quoted), @@ -479,22 +493,22 @@ private[sql] object QueryParsingErrors extends DataTypeErrorsBase { } def invalidPropertyKeyForSetQuotedConfigurationError( - keyCandidate: String, valueStr: String, ctx: ParserRuleContext): Throwable = { + keyCandidate: String, + valueStr: String, + ctx: ParserRuleContext): Throwable = { new ParseException( errorClass = "INVALID_PROPERTY_KEY", - messageParameters = Map( - "key" -> toSQLConf(keyCandidate), - "value" -> toSQLConf(valueStr)), + messageParameters = Map("key" -> toSQLConf(keyCandidate), "value" -> toSQLConf(valueStr)), ctx) } def invalidPropertyValueForSetQuotedConfigurationError( - valueCandidate: String, keyStr: String, ctx: ParserRuleContext): Throwable = { + valueCandidate: String, + keyStr: String, + ctx: ParserRuleContext): Throwable = { new ParseException( errorClass = "INVALID_PROPERTY_VALUE", - messageParameters = Map( - "value" -> toSQLConf(valueCandidate), - "key" -> toSQLConf(keyStr)), + messageParameters = Map("value" -> toSQLConf(valueCandidate), "key" -> toSQLConf(keyStr)), ctx) } @@ -547,7 +561,8 @@ private[sql] object QueryParsingErrors extends DataTypeErrorsBase { } def temporaryViewWithSchemaBindingMode(ctx: StatementContext): Throwable = { - new ParseException(errorClass = "UNSUPPORTED_FEATURE.TEMPORARY_VIEW_WITH_SCHEMA_BINDING_MODE", + new ParseException( + errorClass = "UNSUPPORTED_FEATURE.TEMPORARY_VIEW_WITH_SCHEMA_BINDING_MODE", messageParameters = Map.empty, ctx) } @@ -581,9 +596,7 @@ private[sql] object QueryParsingErrors extends DataTypeErrorsBase { } def defineTempFuncWithIfNotExistsError(ctx: ParserRuleContext): Throwable = { - new ParseException( - errorClass = "INVALID_SQL_SYNTAX.CREATE_TEMP_FUNC_WITH_IF_NOT_EXISTS", - ctx) + new ParseException(errorClass = "INVALID_SQL_SYNTAX.CREATE_TEMP_FUNC_WITH_IF_NOT_EXISTS", ctx) } def unsupportedFunctionNameError(funcName: Seq[String], ctx: ParserRuleContext): Throwable = { @@ -632,9 +645,8 @@ private[sql] object QueryParsingErrors extends DataTypeErrorsBase { def invalidNameForDropTempFunc(name: Seq[String], ctx: ParserRuleContext): Throwable = { new ParseException( errorClass = "INVALID_SQL_SYNTAX.MULTI_PART_NAME", - messageParameters = Map( - "statement" -> toSQLStmt("DROP TEMPORARY FUNCTION"), - "funcName" -> toSQLId(name)), + messageParameters = + Map("statement" -> toSQLStmt("DROP TEMPORARY FUNCTION"), "funcName" -> toSQLId(name)), ctx) } @@ -650,9 +662,7 @@ private[sql] object QueryParsingErrors extends DataTypeErrorsBase { new ParseException(errorClass = "REF_DEFAULT_VALUE_IS_NOT_ALLOWED_IN_PARTITION", ctx) } - def duplicateArgumentNamesError( - arguments: Seq[String], - ctx: ParserRuleContext): Throwable = { + def duplicateArgumentNamesError(arguments: Seq[String], ctx: ParserRuleContext): Throwable = { new ParseException( errorClass = "EXEC_IMMEDIATE_DUPLICATE_ARGUMENT_ALIASES", messageParameters = Map("aliases" -> arguments.map(toSQLId).mkString(", ")), @@ -679,12 +689,9 @@ private[sql] object QueryParsingErrors extends DataTypeErrorsBase { } new ParseException( errorClass = errorClass, - messageParameters = alterTypeMap ++ Map( - "columnName" -> columnName, - "optionName" -> optionName - ), - ctx - ) + messageParameters = + alterTypeMap ++ Map("columnName" -> columnName, "optionName" -> optionName), + ctx) } def invalidDatetimeUnitError( @@ -697,19 +704,17 @@ private[sql] object QueryParsingErrors extends DataTypeErrorsBase { "functionName" -> toSQLId(functionName), "parameter" -> toSQLId("unit"), "invalidValue" -> invalidValue), - ctx - ) + ctx) } def invalidTableFunctionIdentifierArgumentMissingParentheses( - ctx: ParserRuleContext, argumentName: String): Throwable = { + ctx: ParserRuleContext, + argumentName: String): Throwable = { new ParseException( errorClass = "INVALID_SQL_SYNTAX.INVALID_TABLE_FUNCTION_IDENTIFIER_ARGUMENT_MISSING_PARENTHESES", - messageParameters = Map( - "argumentName" -> toSQLId(argumentName)), - ctx - ) + messageParameters = Map("argumentName" -> toSQLId(argumentName)), + ctx) } def clusterByWithPartitionedBy(ctx: ParserRuleContext): Throwable = { diff --git a/sql/api/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala b/sql/api/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala index 146012b4266dd..7d8b33aa5e228 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala @@ -88,8 +88,8 @@ object ProcessingTimeTrigger { } /** - * A [[Trigger]] that continuously processes streaming data, asynchronously checkpointing at - * the specified interval. + * A [[Trigger]] that continuously processes streaming data, asynchronously checkpointing at the + * specified interval. */ case class ContinuousTrigger(intervalMs: Long) extends Trigger { Triggers.validate(intervalMs) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/expressions/Aggregator.scala b/sql/api/src/main/scala/org/apache/spark/sql/expressions/Aggregator.scala index 1a2fbdc1fd116..5bdaebe3b073a 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/expressions/Aggregator.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/expressions/Aggregator.scala @@ -43,9 +43,12 @@ import org.apache.spark.sql.internal.{InvokeInlineUserDefinedFunction, UserDefin * * Based loosely on Aggregator from Algebird: https://github.com/twitter/algebird * - * @tparam IN The input type for the aggregation. - * @tparam BUF The type of the intermediate value of the reduction. - * @tparam OUT The type of the final output result. + * @tparam IN + * The input type for the aggregation. + * @tparam BUF + * The type of the intermediate value of the reduction. + * @tparam OUT + * The type of the final output result. * @since 1.6.0 */ @SerialVersionUID(2093413866369130093L) @@ -58,7 +61,7 @@ abstract class Aggregator[-IN, BUF, OUT] extends Serializable with UserDefinedFu def zero: BUF /** - * Combine two values to produce a new value. For performance, the function may modify `b` and + * Combine two values to produce a new value. For performance, the function may modify `b` and * return it instead of constructing new object for b. * @since 1.6.0 */ @@ -93,8 +96,6 @@ abstract class Aggregator[-IN, BUF, OUT] extends Serializable with UserDefinedFu * @since 1.6.0 */ def toColumn: TypedColumn[IN, OUT] = { - new TypedColumn[IN, OUT]( - InvokeInlineUserDefinedFunction(this, Nil), - outputEncoder) + new TypedColumn[IN, OUT](InvokeInlineUserDefinedFunction(this, Nil), outputEncoder) } } diff --git a/sql/api/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala b/sql/api/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala index c9e0e366a7447..6a22cbfaf351e 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala @@ -52,8 +52,8 @@ sealed abstract class UserDefinedFunction extends UserDefinedFunctionLike { def nullable: Boolean /** - * Returns true iff the UDF is deterministic, i.e. the UDF produces the same output given the same - * input. + * Returns true iff the UDF is deterministic, i.e. the UDF produces the same output given the + * same input. * * @since 2.3.0 */ @@ -98,7 +98,8 @@ private[spark] case class SparkUserDefinedFunction( outputEncoder: Option[Encoder[_]] = None, givenName: Option[String] = None, nullable: Boolean = true, - deterministic: Boolean = true) extends UserDefinedFunction { + deterministic: Boolean = true) + extends UserDefinedFunction { override def withName(name: String): SparkUserDefinedFunction = { copy(givenName = Option(name)) @@ -169,7 +170,8 @@ private[sql] case class UserDefinedAggregator[IN, BUF, OUT]( inputEncoder: Encoder[IN], givenName: Option[String] = None, nullable: Boolean = true, - deterministic: Boolean = true) extends UserDefinedFunction { + deterministic: Boolean = true) + extends UserDefinedFunction { override def withName(name: String): UserDefinedAggregator[IN, BUF, OUT] = { copy(givenName = Option(name)) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/expressions/Window.scala b/sql/api/src/main/scala/org/apache/spark/sql/expressions/Window.scala index 9c4499ee243f5..dbe2da8f97341 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/expressions/Window.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/expressions/Window.scala @@ -32,9 +32,10 @@ import org.apache.spark.sql.Column * Window.partitionBy("country").orderBy("date").rowsBetween(-3, 3) * }}} * - * @note When ordering is not defined, an unbounded window frame (rowFrame, unboundedPreceding, - * unboundedFollowing) is used by default. When ordering is defined, a growing window frame - * (rangeFrame, unboundedPreceding, currentRow) is used by default. + * @note + * When ordering is not defined, an unbounded window frame (rowFrame, unboundedPreceding, + * unboundedFollowing) is used by default. When ordering is defined, a growing window frame + * (rangeFrame, unboundedPreceding, currentRow) is used by default. * * @since 1.4.0 */ @@ -47,7 +48,7 @@ object Window { */ @scala.annotation.varargs def partitionBy(colName: String, colNames: String*): WindowSpec = { - spec.partitionBy(colName, colNames : _*) + spec.partitionBy(colName, colNames: _*) } /** @@ -56,7 +57,7 @@ object Window { */ @scala.annotation.varargs def partitionBy(cols: Column*): WindowSpec = { - spec.partitionBy(cols : _*) + spec.partitionBy(cols: _*) } /** @@ -65,7 +66,7 @@ object Window { */ @scala.annotation.varargs def orderBy(colName: String, colNames: String*): WindowSpec = { - spec.orderBy(colName, colNames : _*) + spec.orderBy(colName, colNames: _*) } /** @@ -74,12 +75,12 @@ object Window { */ @scala.annotation.varargs def orderBy(cols: Column*): WindowSpec = { - spec.orderBy(cols : _*) + spec.orderBy(cols: _*) } /** - * Value representing the first row in the partition, equivalent to "UNBOUNDED PRECEDING" in SQL. - * This can be used to specify the frame boundaries: + * Value representing the first row in the partition, equivalent to "UNBOUNDED PRECEDING" in + * SQL. This can be used to specify the frame boundaries: * * {{{ * Window.rowsBetween(Window.unboundedPreceding, Window.currentRow) @@ -113,22 +114,22 @@ object Window { def currentRow: Long = 0 /** - * Creates a [[WindowSpec]] with the frame boundaries defined, - * from `start` (inclusive) to `end` (inclusive). + * Creates a [[WindowSpec]] with the frame boundaries defined, from `start` (inclusive) to `end` + * (inclusive). * * Both `start` and `end` are positions relative to the current row. For example, "0" means * "current row", while "-1" means the row before the current row, and "5" means the fifth row * after the current row. * - * We recommend users use `Window.unboundedPreceding`, `Window.unboundedFollowing`, - * and `Window.currentRow` to specify special boundary values, rather than using integral - * values directly. + * We recommend users use `Window.unboundedPreceding`, `Window.unboundedFollowing`, and + * `Window.currentRow` to specify special boundary values, rather than using integral values + * directly. * - * A row based boundary is based on the position of the row within the partition. - * An offset indicates the number of rows above or below the current row, the frame for the - * current row starts or ends. For instance, given a row based sliding frame with a lower bound - * offset of -1 and a upper bound offset of +2. The frame for row with index 5 would range from - * index 4 to index 7. + * A row based boundary is based on the position of the row within the partition. An offset + * indicates the number of rows above or below the current row, the frame for the current row + * starts or ends. For instance, given a row based sliding frame with a lower bound offset of -1 + * and a upper bound offset of +2. The frame for row with index 5 would range from index 4 to + * index 7. * * {{{ * import org.apache.spark.sql.expressions.Window @@ -150,10 +151,12 @@ object Window { * +---+--------+---+ * }}} * - * @param start boundary start, inclusive. The frame is unbounded if this is - * the minimum long value (`Window.unboundedPreceding`). - * @param end boundary end, inclusive. The frame is unbounded if this is the - * maximum long value (`Window.unboundedFollowing`). + * @param start + * boundary start, inclusive. The frame is unbounded if this is the minimum long value + * (`Window.unboundedPreceding`). + * @param end + * boundary end, inclusive. The frame is unbounded if this is the maximum long value + * (`Window.unboundedFollowing`). * @since 2.1.0 */ // Note: when updating the doc for this method, also update WindowSpec.rowsBetween. @@ -162,25 +165,24 @@ object Window { } /** - * Creates a [[WindowSpec]] with the frame boundaries defined, - * from `start` (inclusive) to `end` (inclusive). + * Creates a [[WindowSpec]] with the frame boundaries defined, from `start` (inclusive) to `end` + * (inclusive). * * Both `start` and `end` are relative to the current row. For example, "0" means "current row", - * while "-1" means one off before the current row, and "5" means the five off after the - * current row. + * while "-1" means one off before the current row, and "5" means the five off after the current + * row. * - * We recommend users use `Window.unboundedPreceding`, `Window.unboundedFollowing`, - * and `Window.currentRow` to specify special boundary values, rather than using long values + * We recommend users use `Window.unboundedPreceding`, `Window.unboundedFollowing`, and + * `Window.currentRow` to specify special boundary values, rather than using long values * directly. * - * A range-based boundary is based on the actual value of the ORDER BY - * expression(s). An offset is used to alter the value of the ORDER BY expression, - * for instance if the current ORDER BY expression has a value of 10 and the lower bound offset - * is -3, the resulting lower bound for the current row will be 10 - 3 = 7. This however puts a - * number of constraints on the ORDER BY expressions: there can be only one expression and this - * expression must have a numerical data type. An exception can be made when the offset is - * unbounded, because no value modification is needed, in this case multiple and non-numeric - * ORDER BY expression are allowed. + * A range-based boundary is based on the actual value of the ORDER BY expression(s). An offset + * is used to alter the value of the ORDER BY expression, for instance if the current ORDER BY + * expression has a value of 10 and the lower bound offset is -3, the resulting lower bound for + * the current row will be 10 - 3 = 7. This however puts a number of constraints on the ORDER BY + * expressions: there can be only one expression and this expression must have a numerical data + * type. An exception can be made when the offset is unbounded, because no value modification is + * needed, in this case multiple and non-numeric ORDER BY expression are allowed. * * {{{ * import org.apache.spark.sql.expressions.Window @@ -202,10 +204,12 @@ object Window { * +---+--------+---+ * }}} * - * @param start boundary start, inclusive. The frame is unbounded if this is - * the minimum long value (`Window.unboundedPreceding`). - * @param end boundary end, inclusive. The frame is unbounded if this is the - * maximum long value (`Window.unboundedFollowing`). + * @param start + * boundary start, inclusive. The frame is unbounded if this is the minimum long value + * (`Window.unboundedPreceding`). + * @param end + * boundary end, inclusive. The frame is unbounded if this is the maximum long value + * (`Window.unboundedFollowing`). * @since 2.1.0 */ // Note: when updating the doc for this method, also update WindowSpec.rangeBetween. @@ -234,4 +238,4 @@ object Window { * @since 1.4.0 */ @Stable -class Window private() // So we can see Window in JavaDoc. +class Window private () // So we can see Window in JavaDoc. diff --git a/sql/api/src/main/scala/org/apache/spark/sql/expressions/WindowSpec.scala b/sql/api/src/main/scala/org/apache/spark/sql/expressions/WindowSpec.scala index a888563d66a71..9abdee9c79ebc 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/expressions/WindowSpec.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/expressions/WindowSpec.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.internal.{ColumnNode, SortOrder, Window => EvalWindo * @since 1.4.0 */ @Stable -class WindowSpec private[sql]( +class WindowSpec private[sql] ( partitionSpec: Seq[ColumnNode], orderSpec: Seq[SortOrder], frame: Option[WindowFrame]) { @@ -78,15 +78,15 @@ class WindowSpec private[sql]( * "current row", while "-1" means the row before the current row, and "5" means the fifth row * after the current row. * - * We recommend users use `Window.unboundedPreceding`, `Window.unboundedFollowing`, - * and `Window.currentRow` to specify special boundary values, rather than using integral - * values directly. + * We recommend users use `Window.unboundedPreceding`, `Window.unboundedFollowing`, and + * `Window.currentRow` to specify special boundary values, rather than using integral values + * directly. * - * A row based boundary is based on the position of the row within the partition. - * An offset indicates the number of rows above or below the current row, the frame for the - * current row starts or ends. For instance, given a row based sliding frame with a lower bound - * offset of -1 and a upper bound offset of +2. The frame for row with index 5 would range from - * index 4 to index 7. + * A row based boundary is based on the position of the row within the partition. An offset + * indicates the number of rows above or below the current row, the frame for the current row + * starts or ends. For instance, given a row based sliding frame with a lower bound offset of -1 + * and a upper bound offset of +2. The frame for row with index 5 would range from index 4 to + * index 7. * * {{{ * import org.apache.spark.sql.expressions.Window @@ -108,10 +108,12 @@ class WindowSpec private[sql]( * +---+--------+---+ * }}} * - * @param start boundary start, inclusive. The frame is unbounded if this is - * the minimum long value (`Window.unboundedPreceding`). - * @param end boundary end, inclusive. The frame is unbounded if this is the - * maximum long value (`Window.unboundedFollowing`). + * @param start + * boundary start, inclusive. The frame is unbounded if this is the minimum long value + * (`Window.unboundedPreceding`). + * @param end + * boundary end, inclusive. The frame is unbounded if this is the maximum long value + * (`Window.unboundedFollowing`). * @since 1.4.0 */ // Note: when updating the doc for this method, also update Window.rowsBetween. @@ -136,22 +138,21 @@ class WindowSpec private[sql]( /** * Defines the frame boundaries, from `start` (inclusive) to `end` (inclusive). * - * Both `start` and `end` are relative from the current row. For example, "0" means - * "current row", while "-1" means one off before the current row, and "5" means the five off - * after the current row. + * Both `start` and `end` are relative from the current row. For example, "0" means "current + * row", while "-1" means one off before the current row, and "5" means the five off after the + * current row. * - * We recommend users use `Window.unboundedPreceding`, `Window.unboundedFollowing`, - * and `Window.currentRow` to specify special boundary values, rather than using long values + * We recommend users use `Window.unboundedPreceding`, `Window.unboundedFollowing`, and + * `Window.currentRow` to specify special boundary values, rather than using long values * directly. * - * A range-based boundary is based on the actual value of the ORDER BY - * expression(s). An offset is used to alter the value of the ORDER BY expression, for - * instance if the current order by expression has a value of 10 and the lower bound offset - * is -3, the resulting lower bound for the current row will be 10 - 3 = 7. This however puts a - * number of constraints on the ORDER BY expressions: there can be only one expression and this - * expression must have a numerical data type. An exception can be made when the offset is - * unbounded, because no value modification is needed, in this case multiple and non-numeric - * ORDER BY expression are allowed. + * A range-based boundary is based on the actual value of the ORDER BY expression(s). An offset + * is used to alter the value of the ORDER BY expression, for instance if the current order by + * expression has a value of 10 and the lower bound offset is -3, the resulting lower bound for + * the current row will be 10 - 3 = 7. This however puts a number of constraints on the ORDER BY + * expressions: there can be only one expression and this expression must have a numerical data + * type. An exception can be made when the offset is unbounded, because no value modification is + * needed, in this case multiple and non-numeric ORDER BY expression are allowed. * * {{{ * import org.apache.spark.sql.expressions.Window @@ -173,10 +174,12 @@ class WindowSpec private[sql]( * +---+--------+---+ * }}} * - * @param start boundary start, inclusive. The frame is unbounded if this is - * the minimum long value (`Window.unboundedPreceding`). - * @param end boundary end, inclusive. The frame is unbounded if this is the - * maximum long value (`Window.unboundedFollowing`). + * @param start + * boundary start, inclusive. The frame is unbounded if this is the minimum long value + * (`Window.unboundedPreceding`). + * @param end + * boundary end, inclusive. The frame is unbounded if this is the maximum long value + * (`Window.unboundedFollowing`). * @since 1.4.0 */ // Note: when updating the doc for this method, also update Window.rangeBetween. diff --git a/sql/api/src/main/scala/org/apache/spark/sql/expressions/udaf.scala b/sql/api/src/main/scala/org/apache/spark/sql/expressions/udaf.scala index a4aa9c312aff2..5e7c993fae414 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/expressions/udaf.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/expressions/udaf.scala @@ -26,18 +26,21 @@ import org.apache.spark.sql.types._ * The base class for implementing user-defined aggregate functions (UDAF). * * @since 1.5.0 - * @deprecated UserDefinedAggregateFunction is deprecated. - * Aggregator[IN, BUF, OUT] should now be registered as a UDF via the functions.udaf(agg) method. + * @deprecated + * UserDefinedAggregateFunction is deprecated. Aggregator[IN, BUF, OUT] should now be registered + * as a UDF via the functions.udaf(agg) method. */ @Stable -@deprecated("Aggregator[IN, BUF, OUT] should now be registered as a UDF" + - " via the functions.udaf(agg) method.", "3.0.0") +@deprecated( + "Aggregator[IN, BUF, OUT] should now be registered as a UDF" + + " via the functions.udaf(agg) method.", + "3.0.0") abstract class UserDefinedAggregateFunction extends Serializable with UserDefinedFunctionLike { /** - * A `StructType` represents data types of input arguments of this aggregate function. - * For example, if a [[UserDefinedAggregateFunction]] expects two input arguments - * with type of `DoubleType` and `LongType`, the returned `StructType` will look like + * A `StructType` represents data types of input arguments of this aggregate function. For + * example, if a [[UserDefinedAggregateFunction]] expects two input arguments with type of + * `DoubleType` and `LongType`, the returned `StructType` will look like * * ``` * new StructType() @@ -45,18 +48,17 @@ abstract class UserDefinedAggregateFunction extends Serializable with UserDefine * .add("longInput", LongType) * ``` * - * The name of a field of this `StructType` is only used to identify the corresponding - * input argument. Users can choose names to identify the input arguments. + * The name of a field of this `StructType` is only used to identify the corresponding input + * argument. Users can choose names to identify the input arguments. * * @since 1.5.0 */ def inputSchema: StructType /** - * A `StructType` represents data types of values in the aggregation buffer. - * For example, if a [[UserDefinedAggregateFunction]]'s buffer has two values - * (i.e. two intermediate values) with type of `DoubleType` and `LongType`, - * the returned `StructType` will look like + * A `StructType` represents data types of values in the aggregation buffer. For example, if a + * [[UserDefinedAggregateFunction]]'s buffer has two values (i.e. two intermediate values) with + * type of `DoubleType` and `LongType`, the returned `StructType` will look like * * ``` * new StructType() @@ -64,8 +66,8 @@ abstract class UserDefinedAggregateFunction extends Serializable with UserDefine * .add("longInput", LongType) * ``` * - * The name of a field of this `StructType` is only used to identify the corresponding - * buffer value. Users can choose names to identify the input arguments. + * The name of a field of this `StructType` is only used to identify the corresponding buffer + * value. Users can choose names to identify the input arguments. * * @since 1.5.0 */ @@ -79,8 +81,8 @@ abstract class UserDefinedAggregateFunction extends Serializable with UserDefine def dataType: DataType /** - * Returns true iff this function is deterministic, i.e. given the same input, - * always return the same output. + * Returns true iff this function is deterministic, i.e. given the same input, always return the + * same output. * * @since 1.5.0 */ @@ -90,8 +92,8 @@ abstract class UserDefinedAggregateFunction extends Serializable with UserDefine * Initializes the given aggregation buffer, i.e. the zero value of the aggregation buffer. * * The contract should be that applying the merge function on two initial buffers should just - * return the initial buffer itself, i.e. - * `merge(initialBuffer, initialBuffer)` should equal `initialBuffer`. + * return the initial buffer itself, i.e. `merge(initialBuffer, initialBuffer)` should equal + * `initialBuffer`. * * @since 1.5.0 */ @@ -134,8 +136,8 @@ abstract class UserDefinedAggregateFunction extends Serializable with UserDefine } /** - * Creates a `Column` for this UDAF using the distinct values of the given - * `Column`s as input arguments. + * Creates a `Column` for this UDAF using the distinct values of the given `Column`s as input + * arguments. * * @since 1.5.0 */ diff --git a/sql/api/src/main/scala/org/apache/spark/sql/functions.scala b/sql/api/src/main/scala/org/apache/spark/sql/functions.scala index e46d6c95b31ae..771fb824a70c4 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/functions.scala @@ -104,9 +104,9 @@ object functions { /** * Creates a [[Column]] of literal value. * - * The passed in object is returned directly if it is already a [[Column]]. - * If the object is a Scala Symbol, it is converted into a [[Column]] also. - * Otherwise, a new [[Column]] is created to represent the literal value. + * The passed in object is returned directly if it is already a [[Column]]. If the object is a + * Scala Symbol, it is converted into a [[Column]] also. Otherwise, a new [[Column]] is created + * to represent the literal value. * * @group normal_funcs * @since 1.3.0 @@ -121,7 +121,7 @@ object functions { // method, `typedLit[Any](literal)` will always fail and fallback to `Literal.apply`. Hence, // we can just manually call `Literal.apply` to skip the expensive `ScalaReflection` code. // This is significantly better when there are many threads calling `lit` concurrently. - Column(internal.Literal(literal)) + Column(internal.Literal(literal)) } } @@ -133,26 +133,26 @@ object functions { * @group normal_funcs * @since 2.2.0 */ - def typedLit[T : TypeTag](literal: T): Column = { + def typedLit[T: TypeTag](literal: T): Column = { typedlit(literal) } /** * Creates a [[Column]] of literal value. * - * The passed in object is returned directly if it is already a [[Column]]. - * If the object is a Scala Symbol, it is converted into a [[Column]] also. - * Otherwise, a new [[Column]] is created to represent the literal value. - * The difference between this function and [[lit]] is that this function - * can handle parameterized scala types e.g.: List, Seq and Map. + * The passed in object is returned directly if it is already a [[Column]]. If the object is a + * Scala Symbol, it is converted into a [[Column]] also. Otherwise, a new [[Column]] is created + * to represent the literal value. The difference between this function and [[lit]] is that this + * function can handle parameterized scala types e.g.: List, Seq and Map. * - * @note `typedlit` will call expensive Scala reflection APIs. `lit` is preferred if parameterized - * Scala types are not used. + * @note + * `typedlit` will call expensive Scala reflection APIs. `lit` is preferred if parameterized + * Scala types are not used. * * @group normal_funcs * @since 3.2.0 */ - def typedlit[T : TypeTag](literal: T): Column = { + def typedlit[T: TypeTag](literal: T): Column = { literal match { case c: Column => c case s: Symbol => new ColumnName(s.name) @@ -178,8 +178,8 @@ object functions { def asc(columnName: String): Column = Column(columnName).asc /** - * Returns a sort expression based on ascending order of the column, - * and null values return before non-null values. + * Returns a sort expression based on ascending order of the column, and null values return + * before non-null values. * {{{ * df.sort(asc_nulls_first("dept"), desc("age")) * }}} @@ -190,8 +190,8 @@ object functions { def asc_nulls_first(columnName: String): Column = Column(columnName).asc_nulls_first /** - * Returns a sort expression based on ascending order of the column, - * and null values appear after non-null values. + * Returns a sort expression based on ascending order of the column, and null values appear + * after non-null values. * {{{ * df.sort(asc_nulls_last("dept"), desc("age")) * }}} @@ -213,8 +213,8 @@ object functions { def desc(columnName: String): Column = Column(columnName).desc /** - * Returns a sort expression based on the descending order of the column, - * and null values appear before non-null values. + * Returns a sort expression based on the descending order of the column, and null values appear + * before non-null values. * {{{ * df.sort(asc("dept"), desc_nulls_first("age")) * }}} @@ -225,8 +225,8 @@ object functions { def desc_nulls_first(columnName: String): Column = Column(columnName).desc_nulls_first /** - * Returns a sort expression based on the descending order of the column, - * and null values appear after non-null values. + * Returns a sort expression based on the descending order of the column, and null values appear + * after non-null values. * {{{ * df.sort(asc("dept"), desc_nulls_last("age")) * }}} @@ -236,7 +236,6 @@ object functions { */ def desc_nulls_last(columnName: String): Column = Column(columnName).desc_nulls_last - ////////////////////////////////////////////////////////////////////////////////////////////// // Aggregate functions ////////////////////////////////////////////////////////////////////////////////////////////// @@ -285,12 +284,14 @@ object functions { * @group agg_funcs * @since 2.1.0 */ - def approx_count_distinct(columnName: String): Column = approx_count_distinct(column(columnName)) + def approx_count_distinct(columnName: String): Column = approx_count_distinct( + column(columnName)) /** * Aggregate function: returns the approximate number of distinct items in a group. * - * @param rsd maximum relative standard deviation allowed (default = 0.05) + * @param rsd + * maximum relative standard deviation allowed (default = 0.05) * * @group agg_funcs * @since 2.1.0 @@ -302,7 +303,8 @@ object functions { /** * Aggregate function: returns the approximate number of distinct items in a group. * - * @param rsd maximum relative standard deviation allowed (default = 0.05) + * @param rsd + * maximum relative standard deviation allowed (default = 0.05) * * @group agg_funcs * @since 2.1.0 @@ -330,8 +332,9 @@ object functions { /** * Aggregate function: returns a list of objects with duplicates. * - * @note The function is non-deterministic because the order of collected results depends - * on the order of the rows which may be non-deterministic after a shuffle. + * @note + * The function is non-deterministic because the order of collected results depends on the + * order of the rows which may be non-deterministic after a shuffle. * * @group agg_funcs * @since 1.6.0 @@ -341,8 +344,9 @@ object functions { /** * Aggregate function: returns a list of objects with duplicates. * - * @note The function is non-deterministic because the order of collected results depends - * on the order of the rows which may be non-deterministic after a shuffle. + * @note + * The function is non-deterministic because the order of collected results depends on the + * order of the rows which may be non-deterministic after a shuffle. * * @group agg_funcs * @since 1.6.0 @@ -352,8 +356,9 @@ object functions { /** * Aggregate function: returns a set of objects with duplicate elements eliminated. * - * @note The function is non-deterministic because the order of collected results depends - * on the order of the rows which may be non-deterministic after a shuffle. + * @note + * The function is non-deterministic because the order of collected results depends on the + * order of the rows which may be non-deterministic after a shuffle. * * @group agg_funcs * @since 1.6.0 @@ -363,8 +368,9 @@ object functions { /** * Aggregate function: returns a set of objects with duplicate elements eliminated. * - * @note The function is non-deterministic because the order of collected results depends - * on the order of the rows which may be non-deterministic after a shuffle. + * @note + * The function is non-deterministic because the order of collected results depends on the + * order of the rows which may be non-deterministic after a shuffle. * * @group agg_funcs * @since 1.6.0 @@ -372,10 +378,10 @@ object functions { def collect_set(columnName: String): Column = collect_set(Column(columnName)) /** - * Returns a count-min sketch of a column with the given esp, confidence and seed. The result - * is an array of bytes, which can be deserialized to a `CountMinSketch` before usage. - * Count-min sketch is a probabilistic data structure used for cardinality estimation using - * sub-linear space. + * Returns a count-min sketch of a column with the given esp, confidence and seed. The result is + * an array of bytes, which can be deserialized to a `CountMinSketch` before usage. Count-min + * sketch is a probabilistic data structure used for cardinality estimation using sub-linear + * space. * * @group agg_funcs * @since 3.5.0 @@ -443,7 +449,7 @@ object functions { */ @scala.annotation.varargs def countDistinct(columnName: String, columnNames: String*): Column = - count_distinct(Column(columnName), columnNames.map(Column.apply) : _*) + count_distinct(Column(columnName), columnNames.map(Column.apply): _*) /** * Aggregate function: returns the number of distinct items in a group. @@ -499,8 +505,9 @@ object functions { * The function by default returns the first values it sees. It will return the first non-null * value it sees when ignoreNulls is set to true. If all values are null, then null is returned. * - * @note The function is non-deterministic because its results depends on the order of the rows - * which may be non-deterministic after a shuffle. + * @note + * The function is non-deterministic because its results depends on the order of the rows + * which may be non-deterministic after a shuffle. * * @group agg_funcs * @since 2.0.0 @@ -514,8 +521,9 @@ object functions { * The function by default returns the first values it sees. It will return the first non-null * value it sees when ignoreNulls is set to true. If all values are null, then null is returned. * - * @note The function is non-deterministic because its results depends on the order of the rows - * which may be non-deterministic after a shuffle. + * @note + * The function is non-deterministic because its results depends on the order of the rows + * which may be non-deterministic after a shuffle. * * @group agg_funcs * @since 2.0.0 @@ -530,8 +538,9 @@ object functions { * The function by default returns the first values it sees. It will return the first non-null * value it sees when ignoreNulls is set to true. If all values are null, then null is returned. * - * @note The function is non-deterministic because its results depends on the order of the rows - * which may be non-deterministic after a shuffle. + * @note + * The function is non-deterministic because its results depends on the order of the rows + * which may be non-deterministic after a shuffle. * * @group agg_funcs * @since 1.3.0 @@ -544,8 +553,9 @@ object functions { * The function by default returns the first values it sees. It will return the first non-null * value it sees when ignoreNulls is set to true. If all values are null, then null is returned. * - * @note The function is non-deterministic because its results depends on the order of the rows - * which may be non-deterministic after a shuffle. + * @note + * The function is non-deterministic because its results depends on the order of the rows + * which may be non-deterministic after a shuffle. * * @group agg_funcs * @since 1.3.0 @@ -555,8 +565,9 @@ object functions { /** * Aggregate function: returns the first value in a group. * - * @note The function is non-deterministic because its results depends on the order of the rows - * which may be non-deterministic after a shuffle. + * @note + * The function is non-deterministic because its results depends on the order of the rows + * which may be non-deterministic after a shuffle. * * @group agg_funcs * @since 3.5.0 @@ -569,8 +580,9 @@ object functions { * The function by default returns the first values it sees. It will return the first non-null * value it sees when ignoreNulls is set to true. If all values are null, then null is returned. * - * @note The function is non-deterministic because its results depends on the order of the rows - * which may be non-deterministic after a shuffle. + * @note + * The function is non-deterministic because its results depends on the order of the rows + * which may be non-deterministic after a shuffle. * * @group agg_funcs * @since 3.5.0 @@ -579,8 +591,8 @@ object functions { Column.fn("first_value", e, ignoreNulls) /** - * Aggregate function: indicates whether a specified column in a GROUP BY list is aggregated - * or not, returns 1 for aggregated or 0 for not aggregated in the result set. + * Aggregate function: indicates whether a specified column in a GROUP BY list is aggregated or + * not, returns 1 for aggregated or 0 for not aggregated in the result set. * * @group agg_funcs * @since 2.0.0 @@ -588,8 +600,8 @@ object functions { def grouping(e: Column): Column = Column.fn("grouping", e) /** - * Aggregate function: indicates whether a specified column in a GROUP BY list is aggregated - * or not, returns 1 for aggregated or 0 for not aggregated in the result set. + * Aggregate function: indicates whether a specified column in a GROUP BY list is aggregated or + * not, returns 1 for aggregated or 0 for not aggregated in the result set. * * @group agg_funcs * @since 2.0.0 @@ -603,8 +615,9 @@ object functions { * (grouping(c1) <<; (n-1)) + (grouping(c2) <<; (n-2)) + ... + grouping(cn) * }}} * - * @note The list of columns should match with grouping columns exactly, or empty (means all the - * grouping columns). + * @note + * The list of columns should match with grouping columns exactly, or empty (means all the + * grouping columns). * * @group agg_funcs * @since 2.0.0 @@ -618,18 +631,19 @@ object functions { * (grouping(c1) <<; (n-1)) + (grouping(c2) <<; (n-2)) + ... + grouping(cn) * }}} * - * @note The list of columns should match with grouping columns exactly. + * @note + * The list of columns should match with grouping columns exactly. * * @group agg_funcs * @since 2.0.0 */ def grouping_id(colName: String, colNames: String*): Column = { - grouping_id((Seq(colName) ++ colNames).map(n => Column(n)) : _*) + grouping_id((Seq(colName) ++ colNames).map(n => Column(n)): _*) } /** - * Aggregate function: returns the updatable binary representation of the Datasketches - * HllSketch configured with lgConfigK arg. + * Aggregate function: returns the updatable binary representation of the Datasketches HllSketch + * configured with lgConfigK arg. * * @group agg_funcs * @since 3.5.0 @@ -638,8 +652,8 @@ object functions { Column.fn("hll_sketch_agg", e, lgConfigK) /** - * Aggregate function: returns the updatable binary representation of the Datasketches - * HllSketch configured with lgConfigK arg. + * Aggregate function: returns the updatable binary representation of the Datasketches HllSketch + * configured with lgConfigK arg. * * @group agg_funcs * @since 3.5.0 @@ -648,8 +662,8 @@ object functions { Column.fn("hll_sketch_agg", e, lit(lgConfigK)) /** - * Aggregate function: returns the updatable binary representation of the Datasketches - * HllSketch configured with lgConfigK arg. + * Aggregate function: returns the updatable binary representation of the Datasketches HllSketch + * configured with lgConfigK arg. * * @group agg_funcs * @since 3.5.0 @@ -659,8 +673,8 @@ object functions { } /** - * Aggregate function: returns the updatable binary representation of the Datasketches - * HllSketch configured with default lgConfigK value. + * Aggregate function: returns the updatable binary representation of the Datasketches HllSketch + * configured with default lgConfigK value. * * @group agg_funcs * @since 3.5.0 @@ -669,8 +683,8 @@ object functions { Column.fn("hll_sketch_agg", e) /** - * Aggregate function: returns the updatable binary representation of the Datasketches - * HllSketch configured with default lgConfigK value. + * Aggregate function: returns the updatable binary representation of the Datasketches HllSketch + * configured with default lgConfigK value. * * @group agg_funcs * @since 3.5.0 @@ -681,9 +695,9 @@ object functions { /** * Aggregate function: returns the updatable binary representation of the Datasketches - * HllSketch, generated by merging previously created Datasketches HllSketch instances - * via a Datasketches Union instance. Throws an exception if sketches have different - * lgConfigK values and allowDifferentLgConfigK is set to false. + * HllSketch, generated by merging previously created Datasketches HllSketch instances via a + * Datasketches Union instance. Throws an exception if sketches have different lgConfigK values + * and allowDifferentLgConfigK is set to false. * * @group agg_funcs * @since 3.5.0 @@ -693,9 +707,9 @@ object functions { /** * Aggregate function: returns the updatable binary representation of the Datasketches - * HllSketch, generated by merging previously created Datasketches HllSketch instances - * via a Datasketches Union instance. Throws an exception if sketches have different - * lgConfigK values and allowDifferentLgConfigK is set to false. + * HllSketch, generated by merging previously created Datasketches HllSketch instances via a + * Datasketches Union instance. Throws an exception if sketches have different lgConfigK values + * and allowDifferentLgConfigK is set to false. * * @group agg_funcs * @since 3.5.0 @@ -705,9 +719,9 @@ object functions { /** * Aggregate function: returns the updatable binary representation of the Datasketches - * HllSketch, generated by merging previously created Datasketches HllSketch instances - * via a Datasketches Union instance. Throws an exception if sketches have different - * lgConfigK values and allowDifferentLgConfigK is set to false. + * HllSketch, generated by merging previously created Datasketches HllSketch instances via a + * Datasketches Union instance. Throws an exception if sketches have different lgConfigK values + * and allowDifferentLgConfigK is set to false. * * @group agg_funcs * @since 3.5.0 @@ -718,9 +732,8 @@ object functions { /** * Aggregate function: returns the updatable binary representation of the Datasketches - * HllSketch, generated by merging previously created Datasketches HllSketch instances - * via a Datasketches Union instance. Throws an exception if sketches have different - * lgConfigK values. + * HllSketch, generated by merging previously created Datasketches HllSketch instances via a + * Datasketches Union instance. Throws an exception if sketches have different lgConfigK values. * * @group agg_funcs * @since 3.5.0 @@ -730,9 +743,8 @@ object functions { /** * Aggregate function: returns the updatable binary representation of the Datasketches - * HllSketch, generated by merging previously created Datasketches HllSketch instances - * via a Datasketches Union instance. Throws an exception if sketches have different - * lgConfigK values. + * HllSketch, generated by merging previously created Datasketches HllSketch instances via a + * Datasketches Union instance. Throws an exception if sketches have different lgConfigK values. * * @group agg_funcs * @since 3.5.0 @@ -763,8 +775,9 @@ object functions { * The function by default returns the last values it sees. It will return the last non-null * value it sees when ignoreNulls is set to true. If all values are null, then null is returned. * - * @note The function is non-deterministic because its results depends on the order of the rows - * which may be non-deterministic after a shuffle. + * @note + * The function is non-deterministic because its results depends on the order of the rows + * which may be non-deterministic after a shuffle. * * @group agg_funcs * @since 2.0.0 @@ -778,8 +791,9 @@ object functions { * The function by default returns the last values it sees. It will return the last non-null * value it sees when ignoreNulls is set to true. If all values are null, then null is returned. * - * @note The function is non-deterministic because its results depends on the order of the rows - * which may be non-deterministic after a shuffle. + * @note + * The function is non-deterministic because its results depends on the order of the rows + * which may be non-deterministic after a shuffle. * * @group agg_funcs * @since 2.0.0 @@ -794,8 +808,9 @@ object functions { * The function by default returns the last values it sees. It will return the last non-null * value it sees when ignoreNulls is set to true. If all values are null, then null is returned. * - * @note The function is non-deterministic because its results depends on the order of the rows - * which may be non-deterministic after a shuffle. + * @note + * The function is non-deterministic because its results depends on the order of the rows + * which may be non-deterministic after a shuffle. * * @group agg_funcs * @since 1.3.0 @@ -808,8 +823,9 @@ object functions { * The function by default returns the last values it sees. It will return the last non-null * value it sees when ignoreNulls is set to true. If all values are null, then null is returned. * - * @note The function is non-deterministic because its results depends on the order of the rows - * which may be non-deterministic after a shuffle. + * @note + * The function is non-deterministic because its results depends on the order of the rows + * which may be non-deterministic after a shuffle. * * @group agg_funcs * @since 1.3.0 @@ -819,8 +835,9 @@ object functions { /** * Aggregate function: returns the last value in a group. * - * @note The function is non-deterministic because its results depends on the order of the rows - * which may be non-deterministic after a shuffle. + * @note + * The function is non-deterministic because its results depends on the order of the rows + * which may be non-deterministic after a shuffle. * * @group agg_funcs * @since 3.5.0 @@ -833,8 +850,9 @@ object functions { * The function by default returns the last values it sees. It will return the last non-null * value it sees when ignoreNulls is set to true. If all values are null, then null is returned. * - * @note The function is non-deterministic because its results depends on the order of the rows - * which may be non-deterministic after a shuffle. + * @note + * The function is non-deterministic because its results depends on the order of the rows + * which may be non-deterministic after a shuffle. * * @group agg_funcs * @since 3.5.0 @@ -881,8 +899,9 @@ object functions { /** * Aggregate function: returns the value associated with the maximum value of ord. * - * @note The function is non-deterministic so the output order can be different for - * those associated the same values of `e`. + * @note + * The function is non-deterministic so the output order can be different for those associated + * the same values of `e`. * * @group agg_funcs * @since 3.3.0 @@ -890,8 +909,7 @@ object functions { def max_by(e: Column, ord: Column): Column = Column.fn("max_by", e, ord) /** - * Aggregate function: returns the average of the values in a group. - * Alias for avg. + * Aggregate function: returns the average of the values in a group. Alias for avg. * * @group agg_funcs * @since 1.4.0 @@ -899,8 +917,7 @@ object functions { def mean(e: Column): Column = avg(e) /** - * Aggregate function: returns the average of the values in a group. - * Alias for avg. + * Aggregate function: returns the average of the values in a group. Alias for avg. * * @group agg_funcs * @since 1.4.0 @@ -934,8 +951,9 @@ object functions { /** * Aggregate function: returns the value associated with the minimum value of ord. * - * @note The function is non-deterministic so the output order can be different for - * those associated the same values of `e`. + * @note + * The function is non-deterministic so the output order can be different for those associated + * the same values of `e`. * * @group agg_funcs * @since 3.3.0 @@ -943,8 +961,8 @@ object functions { def min_by(e: Column, ord: Column): Column = Column.fn("min_by", e, ord) /** - * Aggregate function: returns the exact percentile(s) of numeric column `expr` at the - * given percentage(s) with value range in [0.0, 1.0]. + * Aggregate function: returns the exact percentile(s) of numeric column `expr` at the given + * percentage(s) with value range in [0.0, 1.0]. * * @group agg_funcs * @since 3.5.0 @@ -952,8 +970,8 @@ object functions { def percentile(e: Column, percentage: Column): Column = Column.fn("percentile", e, percentage) /** - * Aggregate function: returns the exact percentile(s) of numeric column `expr` at the - * given percentage(s) with value range in [0.0, 1.0]. + * Aggregate function: returns the exact percentile(s) of numeric column `expr` at the given + * percentage(s) with value range in [0.0, 1.0]. * * @group agg_funcs * @since 3.5.0 @@ -962,17 +980,16 @@ object functions { Column.fn("percentile", e, percentage, frequency) /** - * Aggregate function: returns the approximate `percentile` of the numeric column `col` which - * is the smallest value in the ordered `col` values (sorted from least to greatest) such that - * no more than `percentage` of `col` values is less than the value or equal to that value. + * Aggregate function: returns the approximate `percentile` of the numeric column `col` which is + * the smallest value in the ordered `col` values (sorted from least to greatest) such that no + * more than `percentage` of `col` values is less than the value or equal to that value. * - * If percentage is an array, each value must be between 0.0 and 1.0. - * If it is a single floating point value, it must be between 0.0 and 1.0. + * If percentage is an array, each value must be between 0.0 and 1.0. If it is a single floating + * point value, it must be between 0.0 and 1.0. * - * The accuracy parameter is a positive numeric literal - * which controls approximation accuracy at the cost of memory. - * Higher value of accuracy yields better accuracy, 1.0/accuracy - * is the relative error of the approximation. + * The accuracy parameter is a positive numeric literal which controls approximation accuracy at + * the cost of memory. Higher value of accuracy yields better accuracy, 1.0/accuracy is the + * relative error of the approximation. * * @group agg_funcs * @since 3.1.0 @@ -981,17 +998,16 @@ object functions { Column.fn("percentile_approx", e, percentage, accuracy) /** - * Aggregate function: returns the approximate `percentile` of the numeric column `col` which - * is the smallest value in the ordered `col` values (sorted from least to greatest) such that - * no more than `percentage` of `col` values is less than the value or equal to that value. + * Aggregate function: returns the approximate `percentile` of the numeric column `col` which is + * the smallest value in the ordered `col` values (sorted from least to greatest) such that no + * more than `percentage` of `col` values is less than the value or equal to that value. * - * If percentage is an array, each value must be between 0.0 and 1.0. - * If it is a single floating point value, it must be between 0.0 and 1.0. + * If percentage is an array, each value must be between 0.0 and 1.0. If it is a single floating + * point value, it must be between 0.0 and 1.0. * - * The accuracy parameter is a positive numeric literal - * which controls approximation accuracy at the cost of memory. - * Higher value of accuracy yields better accuracy, 1.0/accuracy - * is the relative error of the approximation. + * The accuracy parameter is a positive numeric literal which controls approximation accuracy at + * the cost of memory. Higher value of accuracy yields better accuracy, 1.0/accuracy is the + * relative error of the approximation. * * @group agg_funcs * @since 3.5.0 @@ -1049,8 +1065,7 @@ object functions { def stddev(columnName: String): Column = stddev(Column(columnName)) /** - * Aggregate function: returns the sample standard deviation of - * the expression in a group. + * Aggregate function: returns the sample standard deviation of the expression in a group. * * @group agg_funcs * @since 1.6.0 @@ -1058,8 +1073,7 @@ object functions { def stddev_samp(e: Column): Column = Column.fn("stddev_samp", e) /** - * Aggregate function: returns the sample standard deviation of - * the expression in a group. + * Aggregate function: returns the sample standard deviation of the expression in a group. * * @group agg_funcs * @since 1.6.0 @@ -1067,8 +1081,7 @@ object functions { def stddev_samp(columnName: String): Column = stddev_samp(Column(columnName)) /** - * Aggregate function: returns the population standard deviation of - * the expression in a group. + * Aggregate function: returns the population standard deviation of the expression in a group. * * @group agg_funcs * @since 1.6.0 @@ -1076,8 +1089,7 @@ object functions { def stddev_pop(e: Column): Column = Column.fn("stddev_pop", e) /** - * Aggregate function: returns the population standard deviation of - * the expression in a group. + * Aggregate function: returns the population standard deviation of the expression in a group. * * @group agg_funcs * @since 1.6.0 @@ -1175,8 +1187,8 @@ object functions { def var_pop(columnName: String): Column = var_pop(Column(columnName)) /** - * Aggregate function: returns the average of the independent variable for non-null pairs - * in a group, where `y` is the dependent variable and `x` is the independent variable. + * Aggregate function: returns the average of the independent variable for non-null pairs in a + * group, where `y` is the dependent variable and `x` is the independent variable. * * @group agg_funcs * @since 3.5.0 @@ -1184,8 +1196,8 @@ object functions { def regr_avgx(y: Column, x: Column): Column = Column.fn("regr_avgx", y, x) /** - * Aggregate function: returns the average of the independent variable for non-null pairs - * in a group, where `y` is the dependent variable and `x` is the independent variable. + * Aggregate function: returns the average of the independent variable for non-null pairs in a + * group, where `y` is the dependent variable and `x` is the independent variable. * * @group agg_funcs * @since 3.5.0 @@ -1193,8 +1205,8 @@ object functions { def regr_avgy(y: Column, x: Column): Column = Column.fn("regr_avgy", y, x) /** - * Aggregate function: returns the number of non-null number pairs - * in a group, where `y` is the dependent variable and `x` is the independent variable. + * Aggregate function: returns the number of non-null number pairs in a group, where `y` is the + * dependent variable and `x` is the independent variable. * * @group agg_funcs * @since 3.5.0 @@ -1202,9 +1214,9 @@ object functions { def regr_count(y: Column, x: Column): Column = Column.fn("regr_count", y, x) /** - * Aggregate function: returns the intercept of the univariate linear regression line - * for non-null pairs in a group, where `y` is the dependent variable and - * `x` is the independent variable. + * Aggregate function: returns the intercept of the univariate linear regression line for + * non-null pairs in a group, where `y` is the dependent variable and `x` is the independent + * variable. * * @group agg_funcs * @since 3.5.0 @@ -1212,8 +1224,8 @@ object functions { def regr_intercept(y: Column, x: Column): Column = Column.fn("regr_intercept", y, x) /** - * Aggregate function: returns the coefficient of determination for non-null pairs - * in a group, where `y` is the dependent variable and `x` is the independent variable. + * Aggregate function: returns the coefficient of determination for non-null pairs in a group, + * where `y` is the dependent variable and `x` is the independent variable. * * @group agg_funcs * @since 3.5.0 @@ -1221,8 +1233,8 @@ object functions { def regr_r2(y: Column, x: Column): Column = Column.fn("regr_r2", y, x) /** - * Aggregate function: returns the slope of the linear regression line for non-null pairs - * in a group, where `y` is the dependent variable and `x` is the independent variable. + * Aggregate function: returns the slope of the linear regression line for non-null pairs in a + * group, where `y` is the dependent variable and `x` is the independent variable. * * @group agg_funcs * @since 3.5.0 @@ -1230,8 +1242,8 @@ object functions { def regr_slope(y: Column, x: Column): Column = Column.fn("regr_slope", y, x) /** - * Aggregate function: returns REGR_COUNT(y, x) * VAR_POP(x) for non-null pairs - * in a group, where `y` is the dependent variable and `x` is the independent variable. + * Aggregate function: returns REGR_COUNT(y, x) * VAR_POP(x) for non-null pairs in a group, + * where `y` is the dependent variable and `x` is the independent variable. * * @group agg_funcs * @since 3.5.0 @@ -1239,8 +1251,8 @@ object functions { def regr_sxx(y: Column, x: Column): Column = Column.fn("regr_sxx", y, x) /** - * Aggregate function: returns REGR_COUNT(y, x) * COVAR_POP(y, x) for non-null pairs - * in a group, where `y` is the dependent variable and `x` is the independent variable. + * Aggregate function: returns REGR_COUNT(y, x) * COVAR_POP(y, x) for non-null pairs in a group, + * where `y` is the dependent variable and `x` is the independent variable. * * @group agg_funcs * @since 3.5.0 @@ -1248,8 +1260,8 @@ object functions { def regr_sxy(y: Column, x: Column): Column = Column.fn("regr_sxy", y, x) /** - * Aggregate function: returns REGR_COUNT(y, x) * VAR_POP(y) for non-null pairs - * in a group, where `y` is the dependent variable and `x` is the independent variable. + * Aggregate function: returns REGR_COUNT(y, x) * VAR_POP(y) for non-null pairs in a group, + * where `y` is the dependent variable and `x` is the independent variable. * * @group agg_funcs * @since 3.5.0 @@ -1265,8 +1277,8 @@ object functions { def any_value(e: Column): Column = Column.fn("any_value", e) /** - * Aggregate function: returns some value of `e` for a group of rows. - * If `isIgnoreNull` is true, returns only non-null values. + * Aggregate function: returns some value of `e` for a group of rows. If `isIgnoreNull` is true, + * returns only non-null values. * * @group agg_funcs * @since 3.5.0 @@ -1283,13 +1295,12 @@ object functions { def count_if(e: Column): Column = Column.fn("count_if", e) /** - * Aggregate function: computes a histogram on numeric 'expr' using nb bins. - * The return value is an array of (x,y) pairs representing the centers of the - * histogram's bins. As the value of 'nb' is increased, the histogram approximation - * gets finer-grained, but may yield artifacts around outliers. In practice, 20-40 - * histogram bins appear to work well, with more bins being required for skewed or - * smaller datasets. Note that this function creates a histogram with non-uniform - * bin widths. It offers no guarantees in terms of the mean-squared-error of the + * Aggregate function: computes a histogram on numeric 'expr' using nb bins. The return value is + * an array of (x,y) pairs representing the centers of the histogram's bins. As the value of + * 'nb' is increased, the histogram approximation gets finer-grained, but may yield artifacts + * around outliers. In practice, 20-40 histogram bins appear to work well, with more bins being + * required for skewed or smaller datasets. Note that this function creates a histogram with + * non-uniform bin widths. It offers no guarantees in terms of the mean-squared-error of the * histogram, but in practice is comparable to the histograms produced by the R/S-Plus * statistical computing packages. Note: the output type of the 'x' field in the return value is * propagated from the input value consumed in the aggregate function. @@ -1386,10 +1397,10 @@ object functions { * Window function: returns the rank of rows within a window partition, without any gaps. * * The difference between rank and dense_rank is that denseRank leaves no gaps in ranking - * sequence when there are ties. That is, if you were ranking a competition using dense_rank - * and had three people tie for second place, you would say that all three were in second - * place and that the next person came in third. Rank would give me sequential numbers, making - * the person that came in third place (after the ties) would register as coming in fifth. + * sequence when there are ties. That is, if you were ranking a competition using dense_rank and + * had three people tie for second place, you would say that all three were in second place and + * that the next person came in third. Rank would give me sequential numbers, making the person + * that came in third place (after the ties) would register as coming in fifth. * * This is equivalent to the DENSE_RANK function in SQL. * @@ -1399,9 +1410,9 @@ object functions { def dense_rank(): Column = Column.fn("dense_rank") /** - * Window function: returns the value that is `offset` rows before the current row, and - * `null` if there is less than `offset` rows before the current row. For example, - * an `offset` of one will return the previous row at any given point in the window partition. + * Window function: returns the value that is `offset` rows before the current row, and `null` + * if there is less than `offset` rows before the current row. For example, an `offset` of one + * will return the previous row at any given point in the window partition. * * This is equivalent to the LAG function in SQL. * @@ -1411,9 +1422,9 @@ object functions { def lag(e: Column, offset: Int): Column = lag(e, offset, null) /** - * Window function: returns the value that is `offset` rows before the current row, and - * `null` if there is less than `offset` rows before the current row. For example, - * an `offset` of one will return the previous row at any given point in the window partition. + * Window function: returns the value that is `offset` rows before the current row, and `null` + * if there is less than `offset` rows before the current row. For example, an `offset` of one + * will return the previous row at any given point in the window partition. * * This is equivalent to the LAG function in SQL. * @@ -1424,8 +1435,8 @@ object functions { /** * Window function: returns the value that is `offset` rows before the current row, and - * `defaultValue` if there is less than `offset` rows before the current row. For example, - * an `offset` of one will return the previous row at any given point in the window partition. + * `defaultValue` if there is less than `offset` rows before the current row. For example, an + * `offset` of one will return the previous row at any given point in the window partition. * * This is equivalent to the LAG function in SQL. * @@ -1438,8 +1449,8 @@ object functions { /** * Window function: returns the value that is `offset` rows before the current row, and - * `defaultValue` if there is less than `offset` rows before the current row. For example, - * an `offset` of one will return the previous row at any given point in the window partition. + * `defaultValue` if there is less than `offset` rows before the current row. For example, an + * `offset` of one will return the previous row at any given point in the window partition. * * This is equivalent to the LAG function in SQL. * @@ -1453,9 +1464,9 @@ object functions { /** * Window function: returns the value that is `offset` rows before the current row, and * `defaultValue` if there is less than `offset` rows before the current row. `ignoreNulls` - * determines whether null values of row are included in or eliminated from the calculation. - * For example, an `offset` of one will return the previous row at any given point in the - * window partition. + * determines whether null values of row are included in or eliminated from the calculation. For + * example, an `offset` of one will return the previous row at any given point in the window + * partition. * * This is equivalent to the LAG function in SQL. * @@ -1466,9 +1477,9 @@ object functions { Column.fn("lag", false, e, lit(offset), lit(defaultValue), lit(ignoreNulls)) /** - * Window function: returns the value that is `offset` rows after the current row, and - * `null` if there is less than `offset` rows after the current row. For example, - * an `offset` of one will return the next row at any given point in the window partition. + * Window function: returns the value that is `offset` rows after the current row, and `null` if + * there is less than `offset` rows after the current row. For example, an `offset` of one will + * return the next row at any given point in the window partition. * * This is equivalent to the LEAD function in SQL. * @@ -1478,9 +1489,9 @@ object functions { def lead(columnName: String, offset: Int): Column = { lead(columnName, offset, null) } /** - * Window function: returns the value that is `offset` rows after the current row, and - * `null` if there is less than `offset` rows after the current row. For example, - * an `offset` of one will return the next row at any given point in the window partition. + * Window function: returns the value that is `offset` rows after the current row, and `null` if + * there is less than `offset` rows after the current row. For example, an `offset` of one will + * return the next row at any given point in the window partition. * * This is equivalent to the LEAD function in SQL. * @@ -1491,8 +1502,8 @@ object functions { /** * Window function: returns the value that is `offset` rows after the current row, and - * `defaultValue` if there is less than `offset` rows after the current row. For example, - * an `offset` of one will return the next row at any given point in the window partition. + * `defaultValue` if there is less than `offset` rows after the current row. For example, an + * `offset` of one will return the next row at any given point in the window partition. * * This is equivalent to the LEAD function in SQL. * @@ -1505,8 +1516,8 @@ object functions { /** * Window function: returns the value that is `offset` rows after the current row, and - * `defaultValue` if there is less than `offset` rows after the current row. For example, - * an `offset` of one will return the next row at any given point in the window partition. + * `defaultValue` if there is less than `offset` rows after the current row. For example, an + * `offset` of one will return the next row at any given point in the window partition. * * This is equivalent to the LEAD function in SQL. * @@ -1520,9 +1531,9 @@ object functions { /** * Window function: returns the value that is `offset` rows after the current row, and * `defaultValue` if there is less than `offset` rows after the current row. `ignoreNulls` - * determines whether null values of row are included in or eliminated from the calculation. - * The default value of `ignoreNulls` is false. For example, an `offset` of one will return - * the next row at any given point in the window partition. + * determines whether null values of row are included in or eliminated from the calculation. The + * default value of `ignoreNulls` is false. For example, an `offset` of one will return the next + * row at any given point in the window partition. * * This is equivalent to the LEAD function in SQL. * @@ -1533,11 +1544,11 @@ object functions { Column.fn("lead", false, e, lit(offset), lit(defaultValue), lit(ignoreNulls)) /** - * Window function: returns the value that is the `offset`th row of the window frame - * (counting from 1), and `null` if the size of window frame is less than `offset` rows. + * Window function: returns the value that is the `offset`th row of the window frame (counting + * from 1), and `null` if the size of window frame is less than `offset` rows. * - * It will return the `offset`th non-null value it sees when ignoreNulls is set to true. - * If all values are null, then null is returned. + * It will return the `offset`th non-null value it sees when ignoreNulls is set to true. If all + * values are null, then null is returned. * * This is equivalent to the nth_value function in SQL. * @@ -1548,8 +1559,8 @@ object functions { Column.fn("nth_value", false, e, lit(offset), lit(ignoreNulls)) /** - * Window function: returns the value that is the `offset`th row of the window frame - * (counting from 1), and `null` if the size of window frame is less than `offset` rows. + * Window function: returns the value that is the `offset`th row of the window frame (counting + * from 1), and `null` if the size of window frame is less than `offset` rows. * * This is equivalent to the nth_value function in SQL. * @@ -1560,8 +1571,8 @@ object functions { /** * Window function: returns the ntile group id (from 1 to `n` inclusive) in an ordered window - * partition. For example, if `n` is 4, the first quarter of the rows will get value 1, the second - * quarter will get 2, the third quarter will get 3, and the last quarter will get 4. + * partition. For example, if `n` is 4, the first quarter of the rows will get value 1, the + * second quarter will get 2, the third quarter will get 3, and the last quarter will get 4. * * This is equivalent to the NTILE function in SQL. * @@ -1571,7 +1582,8 @@ object functions { def ntile(n: Int): Column = Column.fn("ntile", lit(n)) /** - * Window function: returns the relative rank (i.e. percentile) of rows within a window partition. + * Window function: returns the relative rank (i.e. percentile) of rows within a window + * partition. * * This is computed by: * {{{ @@ -1589,10 +1601,10 @@ object functions { * Window function: returns the rank of rows within a window partition. * * The difference between rank and dense_rank is that dense_rank leaves no gaps in ranking - * sequence when there are ties. That is, if you were ranking a competition using dense_rank - * and had three people tie for second place, you would say that all three were in second - * place and that the next person came in third. Rank would give me sequential numbers, making - * the person that came in third place (after the ties) would register as coming in fifth. + * sequence when there are ties. That is, if you were ranking a competition using dense_rank and + * had three people tie for second place, you would say that all three were in second place and + * that the next person came in third. Rank would give me sequential numbers, making the person + * that came in third place (after the ties) would register as coming in fifth. * * This is equivalent to the RANK function in SQL. * @@ -1630,13 +1642,13 @@ object functions { */ @scala.annotation.varargs def array(colName: String, colNames: String*): Column = { - array((colName +: colNames).map(col) : _*) + array((colName +: colNames).map(col): _*) } /** - * Creates a new map column. The input columns must be grouped as key-value pairs, e.g. - * (key1, value1, key2, value2, ...). The key columns must all have the same data type, and can't - * be null. The value columns must all have the same data type. + * Creates a new map column. The input columns must be grouped as key-value pairs, e.g. (key1, + * value1, key2, value2, ...). The key columns must all have the same data type, and can't be + * null. The value columns must all have the same data type. * * @group map_funcs * @since 2.0 @@ -1663,8 +1675,8 @@ object functions { Column.fn("map_from_arrays", keys, values) /** - * Creates a map after splitting the text into key/value pairs using delimiters. - * Both `pairDelim` and `keyValueDelim` are treated as regular expressions. + * Creates a map after splitting the text into key/value pairs using delimiters. Both + * `pairDelim` and `keyValueDelim` are treated as regular expressions. * * @group map_funcs * @since 3.5.0 @@ -1673,8 +1685,8 @@ object functions { Column.fn("str_to_map", text, pairDelim, keyValueDelim) /** - * Creates a map after splitting the text into key/value pairs using delimiters. - * The `pairDelim` is treated as regular expressions. + * Creates a map after splitting the text into key/value pairs using delimiters. The `pairDelim` + * is treated as regular expressions. * * @group map_funcs * @since 3.5.0 @@ -1709,8 +1721,8 @@ object functions { /** * Returns the first column that is not null, or null if all inputs are null. * - * For example, `coalesce(a, b, c)` will return a if a is not null, - * or b if a is null and b is not null, or c if both a and b are null but c is not null. + * For example, `coalesce(a, b, c)` will return a if a is not null, or b if a is null and b is + * not null, or c if both a and b are null but c is not null. * * @group conditional_funcs * @since 1.3.0 @@ -1745,13 +1757,13 @@ object functions { /** * A column expression that generates monotonically increasing 64-bit integers. * - * The generated ID is guaranteed to be monotonically increasing and unique, but not consecutive. - * The current implementation puts the partition ID in the upper 31 bits, and the record number - * within each partition in the lower 33 bits. The assumption is that the data frame has - * less than 1 billion partitions, and each partition has less than 8 billion records. + * The generated ID is guaranteed to be monotonically increasing and unique, but not + * consecutive. The current implementation puts the partition ID in the upper 31 bits, and the + * record number within each partition in the lower 33 bits. The assumption is that the data + * frame has less than 1 billion partitions, and each partition has less than 8 billion records. * - * As an example, consider a `DataFrame` with two partitions, each with 3 records. - * This expression would return the following IDs: + * As an example, consider a `DataFrame` with two partitions, each with 3 records. This + * expression would return the following IDs: * * {{{ * 0, 1, 2, 8589934592 (1L << 33), 8589934593, 8589934594. @@ -1766,13 +1778,13 @@ object functions { /** * A column expression that generates monotonically increasing 64-bit integers. * - * The generated ID is guaranteed to be monotonically increasing and unique, but not consecutive. - * The current implementation puts the partition ID in the upper 31 bits, and the record number - * within each partition in the lower 33 bits. The assumption is that the data frame has - * less than 1 billion partitions, and each partition has less than 8 billion records. + * The generated ID is guaranteed to be monotonically increasing and unique, but not + * consecutive. The current implementation puts the partition ID in the upper 31 bits, and the + * record number within each partition in the lower 33 bits. The assumption is that the data + * frame has less than 1 billion partitions, and each partition has less than 8 billion records. * - * As an example, consider a `DataFrame` with two partitions, each with 3 records. - * This expression would return the following IDs: + * As an example, consider a `DataFrame` with two partitions, each with 3 records. This + * expression would return the following IDs: * * {{{ * 0, 1, 2, 8589934592 (1L << 33), 8589934593, 8589934594. @@ -1828,7 +1840,8 @@ object functions { * Generate a random column with independent and identically distributed (i.i.d.) samples * uniformly distributed in [0.0, 1.0). * - * @note The function is non-deterministic in general case. + * @note + * The function is non-deterministic in general case. * * @group math_funcs * @since 1.4.0 @@ -1839,7 +1852,8 @@ object functions { * Generate a random column with independent and identically distributed (i.i.d.) samples * uniformly distributed in [0.0, 1.0). * - * @note The function is non-deterministic in general case. + * @note + * The function is non-deterministic in general case. * * @group math_funcs * @since 1.4.0 @@ -1847,10 +1861,11 @@ object functions { def rand(): Column = rand(SparkClassUtils.random.nextLong) /** - * Generate a column with independent and identically distributed (i.i.d.) samples from - * the standard normal distribution. + * Generate a column with independent and identically distributed (i.i.d.) samples from the + * standard normal distribution. * - * @note The function is non-deterministic in general case. + * @note + * The function is non-deterministic in general case. * * @group math_funcs * @since 1.4.0 @@ -1858,10 +1873,11 @@ object functions { def randn(seed: Long): Column = Column.fn("randn", lit(seed)) /** - * Generate a column with independent and identically distributed (i.i.d.) samples from - * the standard normal distribution. + * Generate a column with independent and identically distributed (i.i.d.) samples from the + * standard normal distribution. * - * @note The function is non-deterministic in general case. + * @note + * The function is non-deterministic in general case. * * @group math_funcs * @since 1.4.0 @@ -1871,7 +1887,8 @@ object functions { /** * Partition ID. * - * @note This is non-deterministic because it depends on data partitioning and task scheduling. + * @note + * This is non-deterministic because it depends on data partitioning and task scheduling. * * @group misc_funcs * @since 1.6.0 @@ -1921,8 +1938,7 @@ object functions { def try_divide(left: Column, right: Column): Column = Column.fn("try_divide", left, right) /** - * Returns the remainder of `dividend``/``divisor`. Its result is - * always null if `divisor` is 0. + * Returns the remainder of `dividend``/``divisor`. Its result is always null if `divisor` is 0. * * @group math_funcs * @since 4.0.0 @@ -1956,11 +1972,10 @@ object functions { def try_sum(e: Column): Column = Column.fn("try_sum", e) /** - * Creates a new struct column. - * If the input column is a column in a `DataFrame`, or a derived column expression - * that is named (i.e. aliased), its name would be retained as the StructField's name, - * otherwise, the newly generated StructField's name would be auto generated as - * `col` with a suffix `index + 1`, i.e. col1, col2, col3, ... + * Creates a new struct column. If the input column is a column in a `DataFrame`, or a derived + * column expression that is named (i.e. aliased), its name would be retained as the + * StructField's name, otherwise, the newly generated StructField's name would be auto generated + * as `col` with a suffix `index + 1`, i.e. col1, col2, col3, ... * * @group struct_funcs * @since 1.4.0 @@ -1976,12 +1991,12 @@ object functions { */ @scala.annotation.varargs def struct(colName: String, colNames: String*): Column = { - struct((colName +: colNames).map(col) : _*) + struct((colName +: colNames).map(col): _*) } /** - * Evaluates a list of conditions and returns one of multiple possible result expressions. - * If otherwise is not defined at the end, null is returned for unmatched conditions. + * Evaluates a list of conditions and returns one of multiple possible result expressions. If + * otherwise is not defined at the end, null is returned for unmatched conditions. * * {{{ * // Example: encoding gender string column into integer. @@ -2030,9 +2045,8 @@ object functions { def bit_count(e: Column): Column = Column.fn("bit_count", e) /** - * Returns the value of the bit (0 or 1) at the specified position. - * The positions are numbered from right to left, starting at zero. - * The position argument cannot be negative. + * Returns the value of the bit (0 or 1) at the specified position. The positions are numbered + * from right to left, starting at zero. The position argument cannot be negative. * * @group bitwise_funcs * @since 3.5.0 @@ -2040,9 +2054,8 @@ object functions { def bit_get(e: Column, pos: Column): Column = Column.fn("bit_get", e, pos) /** - * Returns the value of the bit (0 or 1) at the specified position. - * The positions are numbered from right to left, starting at zero. - * The position argument cannot be negative. + * Returns the value of the bit (0 or 1) at the specified position. The positions are numbered + * from right to left, starting at zero. The position argument cannot be negative. * * @group bitwise_funcs * @since 3.5.0 @@ -2074,7 +2087,8 @@ object functions { def abs(e: Column): Column = Column.fn("abs", e) /** - * @return inverse cosine of `e` in radians, as if computed by `java.lang.Math.acos` + * @return + * inverse cosine of `e` in radians, as if computed by `java.lang.Math.acos` * * @group math_funcs * @since 1.4.0 @@ -2082,7 +2096,8 @@ object functions { def acos(e: Column): Column = Column.fn("acos", e) /** - * @return inverse cosine of `columnName`, as if computed by `java.lang.Math.acos` + * @return + * inverse cosine of `columnName`, as if computed by `java.lang.Math.acos` * * @group math_funcs * @since 1.4.0 @@ -2090,7 +2105,8 @@ object functions { def acos(columnName: String): Column = acos(Column(columnName)) /** - * @return inverse hyperbolic cosine of `e` + * @return + * inverse hyperbolic cosine of `e` * * @group math_funcs * @since 3.1.0 @@ -2098,7 +2114,8 @@ object functions { def acosh(e: Column): Column = Column.fn("acosh", e) /** - * @return inverse hyperbolic cosine of `columnName` + * @return + * inverse hyperbolic cosine of `columnName` * * @group math_funcs * @since 3.1.0 @@ -2106,7 +2123,8 @@ object functions { def acosh(columnName: String): Column = acosh(Column(columnName)) /** - * @return inverse sine of `e` in radians, as if computed by `java.lang.Math.asin` + * @return + * inverse sine of `e` in radians, as if computed by `java.lang.Math.asin` * * @group math_funcs * @since 1.4.0 @@ -2114,7 +2132,8 @@ object functions { def asin(e: Column): Column = Column.fn("asin", e) /** - * @return inverse sine of `columnName`, as if computed by `java.lang.Math.asin` + * @return + * inverse sine of `columnName`, as if computed by `java.lang.Math.asin` * * @group math_funcs * @since 1.4.0 @@ -2122,7 +2141,8 @@ object functions { def asin(columnName: String): Column = asin(Column(columnName)) /** - * @return inverse hyperbolic sine of `e` + * @return + * inverse hyperbolic sine of `e` * * @group math_funcs * @since 3.1.0 @@ -2130,7 +2150,8 @@ object functions { def asinh(e: Column): Column = Column.fn("asinh", e) /** - * @return inverse hyperbolic sine of `columnName` + * @return + * inverse hyperbolic sine of `columnName` * * @group math_funcs * @since 3.1.0 @@ -2138,7 +2159,8 @@ object functions { def asinh(columnName: String): Column = asinh(Column(columnName)) /** - * @return inverse tangent of `e` as if computed by `java.lang.Math.atan` + * @return + * inverse tangent of `e` as if computed by `java.lang.Math.atan` * * @group math_funcs * @since 1.4.0 @@ -2146,7 +2168,8 @@ object functions { def atan(e: Column): Column = Column.fn("atan", e) /** - * @return inverse tangent of `columnName`, as if computed by `java.lang.Math.atan` + * @return + * inverse tangent of `columnName`, as if computed by `java.lang.Math.atan` * * @group math_funcs * @since 1.4.0 @@ -2154,13 +2177,14 @@ object functions { def atan(columnName: String): Column = atan(Column(columnName)) /** - * @param y coordinate on y-axis - * @param x coordinate on x-axis - * @return the theta component of the point - * (r, theta) - * in polar coordinates that corresponds to the point - * (x, y) in Cartesian coordinates, - * as if computed by `java.lang.Math.atan2` + * @param y + * coordinate on y-axis + * @param x + * coordinate on x-axis + * @return + * the theta component of the point (r, theta) in polar coordinates that + * corresponds to the point (x, y) in Cartesian coordinates, as if computed by + * `java.lang.Math.atan2` * * @group math_funcs * @since 1.4.0 @@ -2168,13 +2192,14 @@ object functions { def atan2(y: Column, x: Column): Column = Column.fn("atan2", y, x) /** - * @param y coordinate on y-axis - * @param xName coordinate on x-axis - * @return the theta component of the point - * (r, theta) - * in polar coordinates that corresponds to the point - * (x, y) in Cartesian coordinates, - * as if computed by `java.lang.Math.atan2` + * @param y + * coordinate on y-axis + * @param xName + * coordinate on x-axis + * @return + * the theta component of the point (r, theta) in polar coordinates that + * corresponds to the point (x, y) in Cartesian coordinates, as if computed by + * `java.lang.Math.atan2` * * @group math_funcs * @since 1.4.0 @@ -2182,13 +2207,14 @@ object functions { def atan2(y: Column, xName: String): Column = atan2(y, Column(xName)) /** - * @param yName coordinate on y-axis - * @param x coordinate on x-axis - * @return the theta component of the point - * (r, theta) - * in polar coordinates that corresponds to the point - * (x, y) in Cartesian coordinates, - * as if computed by `java.lang.Math.atan2` + * @param yName + * coordinate on y-axis + * @param x + * coordinate on x-axis + * @return + * the theta component of the point (r, theta) in polar coordinates that + * corresponds to the point (x, y) in Cartesian coordinates, as if computed by + * `java.lang.Math.atan2` * * @group math_funcs * @since 1.4.0 @@ -2196,13 +2222,14 @@ object functions { def atan2(yName: String, x: Column): Column = atan2(Column(yName), x) /** - * @param yName coordinate on y-axis - * @param xName coordinate on x-axis - * @return the theta component of the point - * (r, theta) - * in polar coordinates that corresponds to the point - * (x, y) in Cartesian coordinates, - * as if computed by `java.lang.Math.atan2` + * @param yName + * coordinate on y-axis + * @param xName + * coordinate on x-axis + * @return + * the theta component of the point (r, theta) in polar coordinates that + * corresponds to the point (x, y) in Cartesian coordinates, as if computed by + * `java.lang.Math.atan2` * * @group math_funcs * @since 1.4.0 @@ -2211,13 +2238,14 @@ object functions { atan2(Column(yName), Column(xName)) /** - * @param y coordinate on y-axis - * @param xValue coordinate on x-axis - * @return the theta component of the point - * (r, theta) - * in polar coordinates that corresponds to the point - * (x, y) in Cartesian coordinates, - * as if computed by `java.lang.Math.atan2` + * @param y + * coordinate on y-axis + * @param xValue + * coordinate on x-axis + * @return + * the theta component of the point (r, theta) in polar coordinates that + * corresponds to the point (x, y) in Cartesian coordinates, as if computed by + * `java.lang.Math.atan2` * * @group math_funcs * @since 1.4.0 @@ -2225,13 +2253,14 @@ object functions { def atan2(y: Column, xValue: Double): Column = atan2(y, lit(xValue)) /** - * @param yName coordinate on y-axis - * @param xValue coordinate on x-axis - * @return the theta component of the point - * (r, theta) - * in polar coordinates that corresponds to the point - * (x, y) in Cartesian coordinates, - * as if computed by `java.lang.Math.atan2` + * @param yName + * coordinate on y-axis + * @param xValue + * coordinate on x-axis + * @return + * the theta component of the point (r, theta) in polar coordinates that + * corresponds to the point (x, y) in Cartesian coordinates, as if computed by + * `java.lang.Math.atan2` * * @group math_funcs * @since 1.4.0 @@ -2239,13 +2268,14 @@ object functions { def atan2(yName: String, xValue: Double): Column = atan2(Column(yName), xValue) /** - * @param yValue coordinate on y-axis - * @param x coordinate on x-axis - * @return the theta component of the point - * (r, theta) - * in polar coordinates that corresponds to the point - * (x, y) in Cartesian coordinates, - * as if computed by `java.lang.Math.atan2` + * @param yValue + * coordinate on y-axis + * @param x + * coordinate on x-axis + * @return + * the theta component of the point (r, theta) in polar coordinates that + * corresponds to the point (x, y) in Cartesian coordinates, as if computed by + * `java.lang.Math.atan2` * * @group math_funcs * @since 1.4.0 @@ -2253,13 +2283,14 @@ object functions { def atan2(yValue: Double, x: Column): Column = atan2(lit(yValue), x) /** - * @param yValue coordinate on y-axis - * @param xName coordinate on x-axis - * @return the theta component of the point - * (r, theta) - * in polar coordinates that corresponds to the point - * (x, y) in Cartesian coordinates, - * as if computed by `java.lang.Math.atan2` + * @param yValue + * coordinate on y-axis + * @param xName + * coordinate on x-axis + * @return + * the theta component of the point (r, theta) in polar coordinates that + * corresponds to the point (x, y) in Cartesian coordinates, as if computed by + * `java.lang.Math.atan2` * * @group math_funcs * @since 1.4.0 @@ -2267,7 +2298,8 @@ object functions { def atan2(yValue: Double, xName: String): Column = atan2(yValue, Column(xName)) /** - * @return inverse hyperbolic tangent of `e` + * @return + * inverse hyperbolic tangent of `e` * * @group math_funcs * @since 3.1.0 @@ -2275,7 +2307,8 @@ object functions { def atanh(e: Column): Column = Column.fn("atanh", e) /** - * @return inverse hyperbolic tangent of `columnName` + * @return + * inverse hyperbolic tangent of `columnName` * * @group math_funcs * @since 3.1.0 @@ -2366,8 +2399,10 @@ object functions { Column.fn("conv", num, lit(fromBase), lit(toBase)) /** - * @param e angle in radians - * @return cosine of the angle, as if computed by `java.lang.Math.cos` + * @param e + * angle in radians + * @return + * cosine of the angle, as if computed by `java.lang.Math.cos` * * @group math_funcs * @since 1.4.0 @@ -2375,8 +2410,10 @@ object functions { def cos(e: Column): Column = Column.fn("cos", e) /** - * @param columnName angle in radians - * @return cosine of the angle, as if computed by `java.lang.Math.cos` + * @param columnName + * angle in radians + * @return + * cosine of the angle, as if computed by `java.lang.Math.cos` * * @group math_funcs * @since 1.4.0 @@ -2384,8 +2421,10 @@ object functions { def cos(columnName: String): Column = cos(Column(columnName)) /** - * @param e hyperbolic angle - * @return hyperbolic cosine of the angle, as if computed by `java.lang.Math.cosh` + * @param e + * hyperbolic angle + * @return + * hyperbolic cosine of the angle, as if computed by `java.lang.Math.cosh` * * @group math_funcs * @since 1.4.0 @@ -2393,8 +2432,10 @@ object functions { def cosh(e: Column): Column = Column.fn("cosh", e) /** - * @param columnName hyperbolic angle - * @return hyperbolic cosine of the angle, as if computed by `java.lang.Math.cosh` + * @param columnName + * hyperbolic angle + * @return + * hyperbolic cosine of the angle, as if computed by `java.lang.Math.cosh` * * @group math_funcs * @since 1.4.0 @@ -2402,8 +2443,10 @@ object functions { def cosh(columnName: String): Column = cosh(Column(columnName)) /** - * @param e angle in radians - * @return cotangent of the angle + * @param e + * angle in radians + * @return + * cotangent of the angle * * @group math_funcs * @since 3.3.0 @@ -2411,8 +2454,10 @@ object functions { def cot(e: Column): Column = Column.fn("cot", e) /** - * @param e angle in radians - * @return cosecant of the angle + * @param e + * angle in radians + * @return + * cosecant of the angle * * @group math_funcs * @since 3.3.0 @@ -2492,8 +2537,8 @@ object functions { def floor(columnName: String): Column = floor(Column(columnName)) /** - * Returns the greatest value of the list of values, skipping null values. - * This function takes at least 2 parameters. It will return null iff all parameters are null. + * Returns the greatest value of the list of values, skipping null values. This function takes + * at least 2 parameters. It will return null iff all parameters are null. * * @group math_funcs * @since 1.5.0 @@ -2502,8 +2547,8 @@ object functions { def greatest(exprs: Column*): Column = Column.fn("greatest", exprs: _*) /** - * Returns the greatest value of the list of column names, skipping null values. - * This function takes at least 2 parameters. It will return null iff all parameters are null. + * Returns the greatest value of the list of column names, skipping null values. This function + * takes at least 2 parameters. It will return null iff all parameters are null. * * @group math_funcs * @since 1.5.0 @@ -2522,8 +2567,8 @@ object functions { def hex(column: Column): Column = Column.fn("hex", column) /** - * Inverse of hex. Interprets each pair of characters as a hexadecimal number - * and converts to the byte representation of number. + * Inverse of hex. Interprets each pair of characters as a hexadecimal number and converts to + * the byte representation of number. * * @group math_funcs * @since 1.5.0 @@ -2596,8 +2641,8 @@ object functions { def hypot(l: Double, rightName: String): Column = hypot(l, Column(rightName)) /** - * Returns the least value of the list of values, skipping null values. - * This function takes at least 2 parameters. It will return null iff all parameters are null. + * Returns the least value of the list of values, skipping null values. This function takes at + * least 2 parameters. It will return null iff all parameters are null. * * @group math_funcs * @since 1.5.0 @@ -2606,8 +2651,8 @@ object functions { def least(exprs: Column*): Column = Column.fn("least", exprs: _*) /** - * Returns the least value of the list of column names, skipping null values. - * This function takes at least 2 parameters. It will return null iff all parameters are null. + * Returns the least value of the list of column names, skipping null values. This function + * takes at least 2 parameters. It will return null iff all parameters are null. * * @group math_funcs * @since 1.5.0 @@ -2810,8 +2855,8 @@ object functions { def pmod(dividend: Column, divisor: Column): Column = Column.fn("pmod", dividend, divisor) /** - * Returns the double value that is closest in value to the argument and - * is equal to a mathematical integer. + * Returns the double value that is closest in value to the argument and is equal to a + * mathematical integer. * * @group math_funcs * @since 1.4.0 @@ -2819,8 +2864,8 @@ object functions { def rint(e: Column): Column = Column.fn("rint", e) /** - * Returns the double value that is closest in value to the argument and - * is equal to a mathematical integer. + * Returns the double value that is closest in value to the argument and is equal to a + * mathematical integer. * * @group math_funcs * @since 1.4.0 @@ -2836,8 +2881,8 @@ object functions { def round(e: Column): Column = round(e, 0) /** - * Round the value of `e` to `scale` decimal places with HALF_UP round mode - * if `scale` is greater than or equal to 0 or at integral part when `scale` is less than 0. + * Round the value of `e` to `scale` decimal places with HALF_UP round mode if `scale` is + * greater than or equal to 0 or at integral part when `scale` is less than 0. * * @group math_funcs * @since 1.5.0 @@ -2845,8 +2890,8 @@ object functions { def round(e: Column, scale: Int): Column = Column.fn("round", e, lit(scale)) /** - * Round the value of `e` to `scale` decimal places with HALF_UP round mode - * if `scale` is greater than or equal to 0 or at integral part when `scale` is less than 0. + * Round the value of `e` to `scale` decimal places with HALF_UP round mode if `scale` is + * greater than or equal to 0 or at integral part when `scale` is less than 0. * * @group math_funcs * @since 4.0.0 @@ -2862,8 +2907,8 @@ object functions { def bround(e: Column): Column = bround(e, 0) /** - * Round the value of `e` to `scale` decimal places with HALF_EVEN round mode - * if `scale` is greater than or equal to 0 or at integral part when `scale` is less than 0. + * Round the value of `e` to `scale` decimal places with HALF_EVEN round mode if `scale` is + * greater than or equal to 0 or at integral part when `scale` is less than 0. * * @group math_funcs * @since 2.0.0 @@ -2871,8 +2916,8 @@ object functions { def bround(e: Column, scale: Int): Column = Column.fn("bround", e, lit(scale)) /** - * Round the value of `e` to `scale` decimal places with HALF_EVEN round mode - * if `scale` is greater than or equal to 0 or at integral part when `scale` is less than 0. + * Round the value of `e` to `scale` decimal places with HALF_EVEN round mode if `scale` is + * greater than or equal to 0 or at integral part when `scale` is less than 0. * * @group math_funcs * @since 4.0.0 @@ -2880,8 +2925,10 @@ object functions { def bround(e: Column, scale: Column): Column = Column.fn("bround", e, scale) /** - * @param e angle in radians - * @return secant of the angle + * @param e + * angle in radians + * @return + * secant of the angle * * @group math_funcs * @since 3.3.0 @@ -2889,8 +2936,8 @@ object functions { def sec(e: Column): Column = Column.fn("sec", e) /** - * Shift the given value numBits left. If the given value is a long value, this function - * will return a long value else it will return an integer value. + * Shift the given value numBits left. If the given value is a long value, this function will + * return a long value else it will return an integer value. * * @group bitwise_funcs * @since 1.5.0 @@ -2899,8 +2946,8 @@ object functions { def shiftLeft(e: Column, numBits: Int): Column = shiftleft(e, numBits) /** - * Shift the given value numBits left. If the given value is a long value, this function - * will return a long value else it will return an integer value. + * Shift the given value numBits left. If the given value is a long value, this function will + * return a long value else it will return an integer value. * * @group bitwise_funcs * @since 3.2.0 @@ -2927,8 +2974,8 @@ object functions { def shiftright(e: Column, numBits: Int): Column = Column.fn("shiftright", e, lit(numBits)) /** - * Unsigned shift the given value numBits right. If the given value is a long value, - * it will return a long value else it will return an integer value. + * Unsigned shift the given value numBits right. If the given value is a long value, it will + * return a long value else it will return an integer value. * * @group bitwise_funcs * @since 1.5.0 @@ -2937,8 +2984,8 @@ object functions { def shiftRightUnsigned(e: Column, numBits: Int): Column = shiftrightunsigned(e, numBits) /** - * Unsigned shift the given value numBits right. If the given value is a long value, - * it will return a long value else it will return an integer value. + * Unsigned shift the given value numBits right. If the given value is a long value, it will + * return a long value else it will return an integer value. * * @group bitwise_funcs * @since 3.2.0 @@ -2971,8 +3018,10 @@ object functions { def signum(columnName: String): Column = signum(Column(columnName)) /** - * @param e angle in radians - * @return sine of the angle, as if computed by `java.lang.Math.sin` + * @param e + * angle in radians + * @return + * sine of the angle, as if computed by `java.lang.Math.sin` * * @group math_funcs * @since 1.4.0 @@ -2980,8 +3029,10 @@ object functions { def sin(e: Column): Column = Column.fn("sin", e) /** - * @param columnName angle in radians - * @return sine of the angle, as if computed by `java.lang.Math.sin` + * @param columnName + * angle in radians + * @return + * sine of the angle, as if computed by `java.lang.Math.sin` * * @group math_funcs * @since 1.4.0 @@ -2989,8 +3040,10 @@ object functions { def sin(columnName: String): Column = sin(Column(columnName)) /** - * @param e hyperbolic angle - * @return hyperbolic sine of the given value, as if computed by `java.lang.Math.sinh` + * @param e + * hyperbolic angle + * @return + * hyperbolic sine of the given value, as if computed by `java.lang.Math.sinh` * * @group math_funcs * @since 1.4.0 @@ -2998,8 +3051,10 @@ object functions { def sinh(e: Column): Column = Column.fn("sinh", e) /** - * @param columnName hyperbolic angle - * @return hyperbolic sine of the given value, as if computed by `java.lang.Math.sinh` + * @param columnName + * hyperbolic angle + * @return + * hyperbolic sine of the given value, as if computed by `java.lang.Math.sinh` * * @group math_funcs * @since 1.4.0 @@ -3007,8 +3062,10 @@ object functions { def sinh(columnName: String): Column = sinh(Column(columnName)) /** - * @param e angle in radians - * @return tangent of the given value, as if computed by `java.lang.Math.tan` + * @param e + * angle in radians + * @return + * tangent of the given value, as if computed by `java.lang.Math.tan` * * @group math_funcs * @since 1.4.0 @@ -3016,8 +3073,10 @@ object functions { def tan(e: Column): Column = Column.fn("tan", e) /** - * @param columnName angle in radians - * @return tangent of the given value, as if computed by `java.lang.Math.tan` + * @param columnName + * angle in radians + * @return + * tangent of the given value, as if computed by `java.lang.Math.tan` * * @group math_funcs * @since 1.4.0 @@ -3025,8 +3084,10 @@ object functions { def tan(columnName: String): Column = tan(Column(columnName)) /** - * @param e hyperbolic angle - * @return hyperbolic tangent of the given value, as if computed by `java.lang.Math.tanh` + * @param e + * hyperbolic angle + * @return + * hyperbolic tangent of the given value, as if computed by `java.lang.Math.tanh` * * @group math_funcs * @since 1.4.0 @@ -3034,8 +3095,10 @@ object functions { def tanh(e: Column): Column = Column.fn("tanh", e) /** - * @param columnName hyperbolic angle - * @return hyperbolic tangent of the given value, as if computed by `java.lang.Math.tanh` + * @param columnName + * hyperbolic angle + * @return + * hyperbolic tangent of the given value, as if computed by `java.lang.Math.tanh` * * @group math_funcs * @since 1.4.0 @@ -3057,10 +3120,13 @@ object functions { def toDegrees(columnName: String): Column = degrees(Column(columnName)) /** - * Converts an angle measured in radians to an approximately equivalent angle measured in degrees. + * Converts an angle measured in radians to an approximately equivalent angle measured in + * degrees. * - * @param e angle in radians - * @return angle in degrees, as if computed by `java.lang.Math.toDegrees` + * @param e + * angle in radians + * @return + * angle in degrees, as if computed by `java.lang.Math.toDegrees` * * @group math_funcs * @since 2.1.0 @@ -3068,10 +3134,13 @@ object functions { def degrees(e: Column): Column = Column.fn("degrees", e) /** - * Converts an angle measured in radians to an approximately equivalent angle measured in degrees. + * Converts an angle measured in radians to an approximately equivalent angle measured in + * degrees. * - * @param columnName angle in radians - * @return angle in degrees, as if computed by `java.lang.Math.toDegrees` + * @param columnName + * angle in radians + * @return + * angle in degrees, as if computed by `java.lang.Math.toDegrees` * * @group math_funcs * @since 2.1.0 @@ -3093,10 +3162,13 @@ object functions { def toRadians(columnName: String): Column = radians(Column(columnName)) /** - * Converts an angle measured in degrees to an approximately equivalent angle measured in radians. + * Converts an angle measured in degrees to an approximately equivalent angle measured in + * radians. * - * @param e angle in degrees - * @return angle in radians, as if computed by `java.lang.Math.toRadians` + * @param e + * angle in degrees + * @return + * angle in radians, as if computed by `java.lang.Math.toRadians` * * @group math_funcs * @since 2.1.0 @@ -3104,10 +3176,13 @@ object functions { def radians(e: Column): Column = Column.fn("radians", e) /** - * Converts an angle measured in degrees to an approximately equivalent angle measured in radians. + * Converts an angle measured in degrees to an approximately equivalent angle measured in + * radians. * - * @param columnName angle in degrees - * @return angle in radians, as if computed by `java.lang.Math.toRadians` + * @param columnName + * angle in degrees + * @return + * angle in radians, as if computed by `java.lang.Math.toRadians` * * @group math_funcs * @since 2.1.0 @@ -3115,15 +3190,20 @@ object functions { def radians(columnName: String): Column = radians(Column(columnName)) /** - * Returns the bucket number into which the value of this expression would fall - * after being evaluated. Note that input arguments must follow conditions listed below; - * otherwise, the method will return null. + * Returns the bucket number into which the value of this expression would fall after being + * evaluated. Note that input arguments must follow conditions listed below; otherwise, the + * method will return null. * - * @param v value to compute a bucket number in the histogram - * @param min minimum value of the histogram - * @param max maximum value of the histogram - * @param numBucket the number of buckets - * @return the bucket number into which the value would fall after being evaluated + * @param v + * value to compute a bucket number in the histogram + * @param min + * minimum value of the histogram + * @param max + * maximum value of the histogram + * @param numBucket + * the number of buckets + * @return + * the bucket number into which the value would fall after being evaluated * @group math_funcs * @since 3.5.0 */ @@ -3167,8 +3247,8 @@ object functions { def current_user(): Column = Column.fn("current_user") /** - * Calculates the MD5 digest of a binary column and returns the value - * as a 32 character hex string. + * Calculates the MD5 digest of a binary column and returns the value as a 32 character hex + * string. * * @group hash_funcs * @since 1.5.0 @@ -3176,8 +3256,8 @@ object functions { def md5(e: Column): Column = Column.fn("md5", e) /** - * Calculates the SHA-1 digest of a binary column and returns the value - * as a 40 character hex string. + * Calculates the SHA-1 digest of a binary column and returns the value as a 40 character hex + * string. * * @group hash_funcs * @since 1.5.0 @@ -3185,11 +3265,13 @@ object functions { def sha1(e: Column): Column = Column.fn("sha1", e) /** - * Calculates the SHA-2 family of hash functions of a binary column and - * returns the value as a hex string. + * Calculates the SHA-2 family of hash functions of a binary column and returns the value as a + * hex string. * - * @param e column to compute SHA-2 on. - * @param numBits one of 224, 256, 384, or 512. + * @param e + * column to compute SHA-2 on. + * @param numBits + * one of 224, 256, 384, or 512. * * @group hash_funcs * @since 1.5.0 @@ -3202,8 +3284,8 @@ object functions { } /** - * Calculates the cyclic redundancy check value (CRC32) of a binary column and - * returns the value as a bigint. + * Calculates the cyclic redundancy check value (CRC32) of a binary column and returns the value + * as a bigint. * * @group hash_funcs * @since 1.5.0 @@ -3220,9 +3302,8 @@ object functions { def hash(cols: Column*): Column = Column.fn("hash", cols: _*) /** - * Calculates the hash code of given columns using the 64-bit - * variant of the xxHash algorithm, and returns the result as a long - * column. The hash computation uses an initial seed of 42. + * Calculates the hash code of given columns using the 64-bit variant of the xxHash algorithm, + * and returns the result as a long column. The hash computation uses an initial seed of 42. * * @group hash_funcs * @since 3.0.0 @@ -3255,8 +3336,8 @@ object functions { def raise_error(c: Column): Column = Column.fn("raise_error", c) /** - * Returns the estimated number of unique values given the binary representation - * of a Datasketches HllSketch. + * Returns the estimated number of unique values given the binary representation of a + * Datasketches HllSketch. * * @group misc_funcs * @since 3.5.0 @@ -3264,8 +3345,8 @@ object functions { def hll_sketch_estimate(c: Column): Column = Column.fn("hll_sketch_estimate", c) /** - * Returns the estimated number of unique values given the binary representation - * of a Datasketches HllSketch. + * Returns the estimated number of unique values given the binary representation of a + * Datasketches HllSketch. * * @group misc_funcs * @since 3.5.0 @@ -3275,9 +3356,8 @@ object functions { } /** - * Merges two binary representations of Datasketches HllSketch objects, using a - * Datasketches Union object. Throws an exception if sketches have different - * lgConfigK values. + * Merges two binary representations of Datasketches HllSketch objects, using a Datasketches + * Union object. Throws an exception if sketches have different lgConfigK values. * * @group misc_funcs * @since 3.5.0 @@ -3286,9 +3366,8 @@ object functions { Column.fn("hll_union", c1, c2) /** - * Merges two binary representations of Datasketches HllSketch objects, using a - * Datasketches Union object. Throws an exception if sketches have different - * lgConfigK values. + * Merges two binary representations of Datasketches HllSketch objects, using a Datasketches + * Union object. Throws an exception if sketches have different lgConfigK values. * * @group misc_funcs * @since 3.5.0 @@ -3298,9 +3377,9 @@ object functions { } /** - * Merges two binary representations of Datasketches HllSketch objects, using a - * Datasketches Union object. Throws an exception if sketches have different - * lgConfigK values and allowDifferentLgConfigK is set to false. + * Merges two binary representations of Datasketches HllSketch objects, using a Datasketches + * Union object. Throws an exception if sketches have different lgConfigK values and + * allowDifferentLgConfigK is set to false. * * @group misc_funcs * @since 3.5.0 @@ -3309,15 +3388,17 @@ object functions { Column.fn("hll_union", c1, c2, lit(allowDifferentLgConfigK)) /** - * Merges two binary representations of Datasketches HllSketch objects, using a - * Datasketches Union object. Throws an exception if sketches have different - * lgConfigK values and allowDifferentLgConfigK is set to false. + * Merges two binary representations of Datasketches HllSketch objects, using a Datasketches + * Union object. Throws an exception if sketches have different lgConfigK values and + * allowDifferentLgConfigK is set to false. * * @group misc_funcs * @since 3.5.0 */ - def hll_union(columnName1: String, columnName2: String, allowDifferentLgConfigK: Boolean): - Column = { + def hll_union( + columnName1: String, + columnName2: String, + allowDifferentLgConfigK: Boolean): Column = { hll_union(Column(columnName1), Column(columnName2), allowDifferentLgConfigK) } @@ -3684,8 +3765,8 @@ object functions { Column.fn("bitmap_bucket_number", col) /** - * Returns a bitmap with the positions of the bits set from all the values from the input column. - * The input column will most likely be bitmap_bit_position(). + * Returns a bitmap with the positions of the bits set from all the values from the input + * column. The input column will most likely be bitmap_bit_position(). * * @group agg_funcs * @since 3.5.0 @@ -3702,8 +3783,8 @@ object functions { def bitmap_count(col: Column): Column = Column.fn("bitmap_count", col) /** - * Returns a bitmap that is the bitwise OR of all of the bitmaps from the input column. - * The input column should be bitmaps created from bitmap_construct_agg(). + * Returns a bitmap that is the bitwise OR of all of the bitmaps from the input column. The + * input column should be bitmaps created from bitmap_construct_agg(). * * @group agg_funcs * @since 3.5.0 @@ -3724,8 +3805,8 @@ object functions { def ascii(e: Column): Column = Column.fn("ascii", e) /** - * Computes the BASE64 encoding of a binary column and returns it as a string column. - * This is the reverse of unbase64. + * Computes the BASE64 encoding of a binary column and returns it as a string column. This is + * the reverse of unbase64. * * @group string_funcs * @since 1.5.0 @@ -3741,10 +3822,11 @@ object functions { def bit_length(e: Column): Column = Column.fn("bit_length", e) /** - * Concatenates multiple input string columns together into a single string column, - * using the given separator. + * Concatenates multiple input string columns together into a single string column, using the + * given separator. * - * @note Input strings which are null are skipped. + * @note + * Input strings which are null are skipped. * * @group string_funcs * @since 1.5.0 @@ -3754,9 +3836,9 @@ object functions { Column.fn("concat_ws", lit(sep) +: exprs: _*) /** - * Computes the first argument into a string from a binary using the provided character set - * (one of 'US-ASCII', 'ISO-8859-1', 'UTF-8', 'UTF-16BE', 'UTF-16LE', 'UTF-16', 'UTF-32'). - * If either argument is null, the result will also be null. + * Computes the first argument into a string from a binary using the provided character set (one + * of 'US-ASCII', 'ISO-8859-1', 'UTF-8', 'UTF-16BE', 'UTF-16LE', 'UTF-16', 'UTF-32'). If either + * argument is null, the result will also be null. * * @group string_funcs * @since 1.5.0 @@ -3765,9 +3847,9 @@ object functions { Column.fn("decode", value, lit(charset)) /** - * Computes the first argument into a binary from a string using the provided character set - * (one of 'US-ASCII', 'ISO-8859-1', 'UTF-8', 'UTF-16BE', 'UTF-16LE', 'UTF-16', 'UTF-32'). - * If either argument is null, the result will also be null. + * Computes the first argument into a binary from a string using the provided character set (one + * of 'US-ASCII', 'ISO-8859-1', 'UTF-8', 'UTF-16BE', 'UTF-16LE', 'UTF-16', 'UTF-32'). If either + * argument is null, the result will also be null. * * @group string_funcs * @since 1.5.0 @@ -3776,11 +3858,11 @@ object functions { Column.fn("encode", value, lit(charset)) /** - * Formats numeric column x to a format like '#,###,###.##', rounded to d decimal places - * with HALF_EVEN round mode, and returns the result as a string column. + * Formats numeric column x to a format like '#,###,###.##', rounded to d decimal places with + * HALF_EVEN round mode, and returns the result as a string column. * - * If d is 0, the result has no decimal point or fractional part. - * If d is less than 0, the result will be null. + * If d is 0, the result has no decimal point or fractional part. If d is less than 0, the + * result will be null. * * @group string_funcs * @since 1.5.0 @@ -3798,8 +3880,8 @@ object functions { Column.fn("format_string", lit(format) +: arguments: _*) /** - * Returns a new string column by converting the first letter of each word to uppercase. - * Words are delimited by whitespace. + * Returns a new string column by converting the first letter of each word to uppercase. Words + * are delimited by whitespace. * * For example, "hello world" will become "Hello World". * @@ -3809,11 +3891,12 @@ object functions { def initcap(e: Column): Column = Column.fn("initcap", e) /** - * Locate the position of the first occurrence of substr column in the given string. - * Returns null if either of the arguments are null. + * Locate the position of the first occurrence of substr column in the given string. Returns + * null if either of the arguments are null. * - * @note The position is not zero based, but 1 based index. Returns 0 if substr - * could not be found in str. + * @note + * The position is not zero based, but 1 based index. Returns 0 if substr could not be found + * in str. * * @group string_funcs * @since 1.5.0 @@ -3821,8 +3904,8 @@ object functions { def instr(str: Column, substring: String): Column = Column.fn("instr", str, lit(substring)) /** - * Computes the character length of a given string or number of bytes of a binary string. - * The length of character strings include the trailing spaces. The length of binary strings + * Computes the character length of a given string or number of bytes of a binary string. The + * length of character strings include the trailing spaces. The length of binary strings * includes binary zeros. * * @group string_funcs @@ -3831,8 +3914,8 @@ object functions { def length(e: Column): Column = Column.fn("length", e) /** - * Computes the character length of a given string or number of bytes of a binary string. - * The length of character strings include the trailing spaces. The length of binary strings + * Computes the character length of a given string or number of bytes of a binary string. The + * length of character strings include the trailing spaces. The length of binary strings * includes binary zeros. * * @group string_funcs @@ -3849,9 +3932,10 @@ object functions { def lower(e: Column): Column = Column.fn("lower", e) /** - * Computes the Levenshtein distance of the two given string columns if it's less than or - * equal to a given threshold. - * @return result distance, or -1 + * Computes the Levenshtein distance of the two given string columns if it's less than or equal + * to a given threshold. + * @return + * result distance, or -1 * @group string_funcs * @since 3.5.0 */ @@ -3868,8 +3952,9 @@ object functions { /** * Locate the position of the first occurrence of substr. * - * @note The position is not zero based, but 1 based index. Returns 0 if substr - * could not be found in str. + * @note + * The position is not zero based, but 1 based index. Returns 0 if substr could not be found + * in str. * * @group string_funcs * @since 1.5.0 @@ -3879,8 +3964,9 @@ object functions { /** * Locate the position of the first occurrence of substr in a string column, after position pos. * - * @note The position is not zero based, but 1 based index. returns 0 if substr - * could not be found in str. + * @note + * The position is not zero based, but 1 based index. returns 0 if substr could not be found + * in str. * * @group string_funcs * @since 1.5.0 @@ -3889,8 +3975,8 @@ object functions { Column.fn("locate", lit(substr), str, lit(pos)) /** - * Left-pad the string column with pad to a length of len. If the string column is longer - * than len, the return value is shortened to len characters. + * Left-pad the string column with pad to a length of len. If the string column is longer than + * len, the return value is shortened to len characters. * * @group string_funcs * @since 1.5.0 @@ -3972,8 +4058,8 @@ object functions { def regexp_like(str: Column, regexp: Column): Column = Column.fn("regexp_like", str, regexp) /** - * Returns a count of the number of times that the regular expression pattern `regexp` - * is matched in the string `str`. + * Returns a count of the number of times that the regular expression pattern `regexp` is + * matched in the string `str`. * * @group string_funcs * @since 3.5.0 @@ -3981,10 +4067,10 @@ object functions { def regexp_count(str: Column, regexp: Column): Column = Column.fn("regexp_count", str, regexp) /** - * Extract a specific group matched by a Java regex, from the specified string column. - * If the regex did not match, or the specified group did not match, an empty string is returned. - * if the specified group index exceeds the group count of regex, an IllegalArgumentException - * will be thrown. + * Extract a specific group matched by a Java regex, from the specified string column. If the + * regex did not match, or the specified group did not match, an empty string is returned. if + * the specified group index exceeds the group count of regex, an IllegalArgumentException will + * be thrown. * * @group string_funcs * @since 1.5.0 @@ -3993,8 +4079,8 @@ object functions { Column.fn("regexp_extract", e, lit(exp), lit(groupIdx)) /** - * Extract all strings in the `str` that match the `regexp` expression and - * corresponding to the first regex group index. + * Extract all strings in the `str` that match the `regexp` expression and corresponding to the + * first regex group index. * * @group string_funcs * @since 3.5.0 @@ -4003,8 +4089,8 @@ object functions { Column.fn("regexp_extract_all", str, regexp) /** - * Extract all strings in the `str` that match the `regexp` expression and - * corresponding to the regex group index. + * Extract all strings in the `str` that match the `regexp` expression and corresponding to the + * regex group index. * * @group string_funcs * @since 3.5.0 @@ -4040,9 +4126,9 @@ object functions { def regexp_substr(str: Column, regexp: Column): Column = Column.fn("regexp_substr", str, regexp) /** - * Searches a string for a regular expression and returns an integer that indicates - * the beginning position of the matched substring. Positions are 1-based, not 0-based. - * If no match is found, returns 0. + * Searches a string for a regular expression and returns an integer that indicates the + * beginning position of the matched substring. Positions are 1-based, not 0-based. If no match + * is found, returns 0. * * @group string_funcs * @since 3.5.0 @@ -4050,9 +4136,9 @@ object functions { def regexp_instr(str: Column, regexp: Column): Column = Column.fn("regexp_instr", str, regexp) /** - * Searches a string for a regular expression and returns an integer that indicates - * the beginning position of the matched substring. Positions are 1-based, not 0-based. - * If no match is found, returns 0. + * Searches a string for a regular expression and returns an integer that indicates the + * beginning position of the matched substring. Positions are 1-based, not 0-based. If no match + * is found, returns 0. * * @group string_funcs * @since 3.5.0 @@ -4061,8 +4147,8 @@ object functions { Column.fn("regexp_instr", str, regexp, idx) /** - * Decodes a BASE64 encoded string column and returns it as a binary column. - * This is the reverse of base64. + * Decodes a BASE64 encoded string column and returns it as a binary column. This is the reverse + * of base64. * * @group string_funcs * @since 1.5.0 @@ -4070,8 +4156,8 @@ object functions { def unbase64(e: Column): Column = Column.fn("unbase64", e) /** - * Right-pad the string column with pad to a length of len. If the string column is longer - * than len, the return value is shortened to len characters. + * Right-pad the string column with pad to a length of len. If the string column is longer than + * len, the return value is shortened to len characters. * * @group string_funcs * @since 1.5.0 @@ -4199,11 +4285,11 @@ object functions { Column.fn("split", str, pattern, limit) /** - * Substring starts at `pos` and is of length `len` when str is String type or - * returns the slice of byte array that starts at `pos` in byte and is of length `len` - * when str is Binary type + * Substring starts at `pos` and is of length `len` when str is String type or returns the slice + * of byte array that starts at `pos` in byte and is of length `len` when str is Binary type * - * @note The position is not zero based, but 1 based index. + * @note + * The position is not zero based, but 1 based index. * * @group string_funcs * @since 1.5.0 @@ -4212,11 +4298,11 @@ object functions { Column.fn("substring", str, lit(pos), lit(len)) /** - * Substring starts at `pos` and is of length `len` when str is String type or - * returns the slice of byte array that starts at `pos` in byte and is of length `len` - * when str is Binary type + * Substring starts at `pos` and is of length `len` when str is String type or returns the slice + * of byte array that starts at `pos` in byte and is of length `len` when str is Binary type * - * @note The position is not zero based, but 1 based index. + * @note + * The position is not zero based, but 1 based index. * * @group string_funcs * @since 4.0.0 @@ -4225,8 +4311,8 @@ object functions { Column.fn("substring", str, pos, len) /** - * Returns the substring from string str before count occurrences of the delimiter delim. - * If count is positive, everything the left of the final delimiter (counting from left) is + * Returns the substring from string str before count occurrences of the delimiter delim. If + * count is positive, everything the left of the final delimiter (counting from left) is * returned. If count is negative, every to the right of the final delimiter (counting from the * right) is returned. substring_index performs a case-sensitive match when searching for delim. * @@ -4236,8 +4322,8 @@ object functions { Column.fn("substring_index", str, lit(delim), lit(count)) /** - * Overlay the specified portion of `src` with `replace`, - * starting from byte position `pos` of `src` and proceeding for `len` bytes. + * Overlay the specified portion of `src` with `replace`, starting from byte position `pos` of + * `src` and proceeding for `len` bytes. * * @group string_funcs * @since 3.0.0 @@ -4246,8 +4332,8 @@ object functions { Column.fn("overlay", src, replace, pos, len) /** - * Overlay the specified portion of `src` with `replace`, - * starting from byte position `pos` of `src`. + * Overlay the specified portion of `src` with `replace`, starting from byte position `pos` of + * `src`. * * @group string_funcs * @since 3.0.0 @@ -4264,18 +4350,17 @@ object functions { Column.fn("sentences", string, language, country) /** - * Splits a string into arrays of sentences, where each sentence is an array of words. - * The default locale is used. + * Splits a string into arrays of sentences, where each sentence is an array of words. The + * default locale is used. * @group string_funcs * @since 3.2.0 */ def sentences(string: Column): Column = Column.fn("sentences", string) /** - * Translate any character in the src by a character in replaceString. - * The characters in replaceString correspond to the characters in matchingString. - * The translate will happen when any character in the string matches the character - * in the `matchingString`. + * Translate any character in the src by a character in replaceString. The characters in + * replaceString correspond to the characters in matchingString. The translate will happen when + * any character in the string matches the character in the `matchingString`. * * @group string_funcs * @since 1.5.0 @@ -4307,10 +4392,10 @@ object functions { def upper(e: Column): Column = Column.fn("upper", e) /** - * Converts the input `e` to a binary value based on the supplied `format`. - * The `format` can be a case-insensitive string literal of "hex", "utf-8", "utf8", or "base64". - * By default, the binary format for conversion is "hex" if `format` is omitted. - * The function returns NULL if at least one of the input parameters is NULL. + * Converts the input `e` to a binary value based on the supplied `format`. The `format` can be + * a case-insensitive string literal of "hex", "utf-8", "utf8", or "base64". By default, the + * binary format for conversion is "hex" if `format` is omitted. The function returns NULL if at + * least one of the input parameters is NULL. * * @group string_funcs * @since 3.5.0 @@ -4318,8 +4403,8 @@ object functions { def to_binary(e: Column, f: Column): Column = Column.fn("to_binary", e, f) /** - * Converts the input `e` to a binary value based on the default format "hex". - * The function returns NULL if at least one of the input parameters is NULL. + * Converts the input `e` to a binary value based on the default format "hex". The function + * returns NULL if at least one of the input parameters is NULL. * * @group string_funcs * @since 3.5.0 @@ -4328,32 +4413,27 @@ object functions { // scalastyle:off line.size.limit /** - * Convert `e` to a string based on the `format`. - * Throws an exception if the conversion fails. The format can consist of the following - * characters, case insensitive: - * '0' or '9': Specifies an expected digit between 0 and 9. A sequence of 0 or 9 in the format - * string matches a sequence of digits in the input value, generating a result string of the - * same length as the corresponding sequence in the format string. The result string is - * left-padded with zeros if the 0/9 sequence comprises more digits than the matching part of - * the decimal value, starts with 0, and is before the decimal point. Otherwise, it is - * padded with spaces. - * '.' or 'D': Specifies the position of the decimal point (optional, only allowed once). - * ',' or 'G': Specifies the position of the grouping (thousands) separator (,). There must be - * a 0 or 9 to the left and right of each grouping separator. - * '$': Specifies the location of the $ currency sign. This character may only be specified - * once. - * 'S' or 'MI': Specifies the position of a '-' or '+' sign (optional, only allowed once at - * the beginning or end of the format string). Note that 'S' prints '+' for positive values - * but 'MI' prints a space. - * 'PR': Only allowed at the end of the format string; specifies that the result string will be - * wrapped by angle brackets if the input value is negative. - * - * If `e` is a datetime, `format` shall be a valid datetime pattern, see - * Datetime Patterns. - * If `e` is a binary, it is converted to a string in one of the formats: - * 'base64': a base 64 string. - * 'hex': a string in the hexadecimal format. - * 'utf-8': the input binary is decoded to UTF-8 string. + * Convert `e` to a string based on the `format`. Throws an exception if the conversion fails. + * The format can consist of the following characters, case insensitive: '0' or '9': Specifies + * an expected digit between 0 and 9. A sequence of 0 or 9 in the format string matches a + * sequence of digits in the input value, generating a result string of the same length as the + * corresponding sequence in the format string. The result string is left-padded with zeros if + * the 0/9 sequence comprises more digits than the matching part of the decimal value, starts + * with 0, and is before the decimal point. Otherwise, it is padded with spaces. '.' or 'D': + * Specifies the position of the decimal point (optional, only allowed once). ',' or 'G': + * Specifies the position of the grouping (thousands) separator (,). There must be a 0 or 9 to + * the left and right of each grouping separator. '$': Specifies the location of the $ currency + * sign. This character may only be specified once. 'S' or 'MI': Specifies the position of a '-' + * or '+' sign (optional, only allowed once at the beginning or end of the format string). Note + * that 'S' prints '+' for positive values but 'MI' prints a space. 'PR': Only allowed at the + * end of the format string; specifies that the result string will be wrapped by angle brackets + * if the input value is negative. + * + * If `e` is a datetime, `format` shall be a valid datetime pattern, see Datetime + * Patterns. If `e` is a binary, it is converted to a string in one of the formats: + * 'base64': a base 64 string. 'hex': a string in the hexadecimal format. 'utf-8': the input + * binary is decoded to UTF-8 string. * * @group string_funcs * @since 3.5.0 @@ -4363,32 +4443,27 @@ object functions { // scalastyle:off line.size.limit /** - * Convert `e` to a string based on the `format`. - * Throws an exception if the conversion fails. The format can consist of the following - * characters, case insensitive: - * '0' or '9': Specifies an expected digit between 0 and 9. A sequence of 0 or 9 in the format - * string matches a sequence of digits in the input value, generating a result string of the - * same length as the corresponding sequence in the format string. The result string is - * left-padded with zeros if the 0/9 sequence comprises more digits than the matching part of - * the decimal value, starts with 0, and is before the decimal point. Otherwise, it is - * padded with spaces. - * '.' or 'D': Specifies the position of the decimal point (optional, only allowed once). - * ',' or 'G': Specifies the position of the grouping (thousands) separator (,). There must be - * a 0 or 9 to the left and right of each grouping separator. - * '$': Specifies the location of the $ currency sign. This character may only be specified - * once. - * 'S' or 'MI': Specifies the position of a '-' or '+' sign (optional, only allowed once at - * the beginning or end of the format string). Note that 'S' prints '+' for positive values - * but 'MI' prints a space. - * 'PR': Only allowed at the end of the format string; specifies that the result string will be - * wrapped by angle brackets if the input value is negative. - * - * If `e` is a datetime, `format` shall be a valid datetime pattern, see - * Datetime Patterns. - * If `e` is a binary, it is converted to a string in one of the formats: - * 'base64': a base 64 string. - * 'hex': a string in the hexadecimal format. - * 'utf-8': the input binary is decoded to UTF-8 string. + * Convert `e` to a string based on the `format`. Throws an exception if the conversion fails. + * The format can consist of the following characters, case insensitive: '0' or '9': Specifies + * an expected digit between 0 and 9. A sequence of 0 or 9 in the format string matches a + * sequence of digits in the input value, generating a result string of the same length as the + * corresponding sequence in the format string. The result string is left-padded with zeros if + * the 0/9 sequence comprises more digits than the matching part of the decimal value, starts + * with 0, and is before the decimal point. Otherwise, it is padded with spaces. '.' or 'D': + * Specifies the position of the decimal point (optional, only allowed once). ',' or 'G': + * Specifies the position of the grouping (thousands) separator (,). There must be a 0 or 9 to + * the left and right of each grouping separator. '$': Specifies the location of the $ currency + * sign. This character may only be specified once. 'S' or 'MI': Specifies the position of a '-' + * or '+' sign (optional, only allowed once at the beginning or end of the format string). Note + * that 'S' prints '+' for positive values but 'MI' prints a space. 'PR': Only allowed at the + * end of the format string; specifies that the result string will be wrapped by angle brackets + * if the input value is negative. + * + * If `e` is a datetime, `format` shall be a valid datetime pattern, see Datetime + * Patterns. If `e` is a binary, it is converted to a string in one of the formats: + * 'base64': a base 64 string. 'hex': a string in the hexadecimal format. 'utf-8': the input + * binary is decoded to UTF-8 string. * * @group string_funcs * @since 3.5.0 @@ -4397,24 +4472,21 @@ object functions { def to_varchar(e: Column, format: Column): Column = Column.fn("to_varchar", e, format) /** - * Convert string 'e' to a number based on the string format 'format'. - * Throws an exception if the conversion fails. The format can consist of the following - * characters, case insensitive: - * '0' or '9': Specifies an expected digit between 0 and 9. A sequence of 0 or 9 in the format - * string matches a sequence of digits in the input string. If the 0/9 sequence starts with - * 0 and is before the decimal point, it can only match a digit sequence of the same size. - * Otherwise, if the sequence starts with 9 or is after the decimal point, it can match a - * digit sequence that has the same or smaller size. - * '.' or 'D': Specifies the position of the decimal point (optional, only allowed once). - * ',' or 'G': Specifies the position of the grouping (thousands) separator (,). There must be - * a 0 or 9 to the left and right of each grouping separator. 'expr' must match the - * grouping separator relevant for the size of the number. - * '$': Specifies the location of the $ currency sign. This character may only be specified - * once. - * 'S' or 'MI': Specifies the position of a '-' or '+' sign (optional, only allowed once at - * the beginning or end of the format string). Note that 'S' allows '-' but 'MI' does not. - * 'PR': Only allowed at the end of the format string; specifies that 'expr' indicates a - * negative number with wrapping angled brackets. + * Convert string 'e' to a number based on the string format 'format'. Throws an exception if + * the conversion fails. The format can consist of the following characters, case insensitive: + * '0' or '9': Specifies an expected digit between 0 and 9. A sequence of 0 or 9 in the format + * string matches a sequence of digits in the input string. If the 0/9 sequence starts with 0 + * and is before the decimal point, it can only match a digit sequence of the same size. + * Otherwise, if the sequence starts with 9 or is after the decimal point, it can match a digit + * sequence that has the same or smaller size. '.' or 'D': Specifies the position of the decimal + * point (optional, only allowed once). ',' or 'G': Specifies the position of the grouping + * (thousands) separator (,). There must be a 0 or 9 to the left and right of each grouping + * separator. 'expr' must match the grouping separator relevant for the size of the number. '$': + * Specifies the location of the $ currency sign. This character may only be specified once. 'S' + * or 'MI': Specifies the position of a '-' or '+' sign (optional, only allowed once at the + * beginning or end of the format string). Note that 'S' allows '-' but 'MI' does not. 'PR': + * Only allowed at the end of the format string; specifies that 'expr' indicates a negative + * number with wrapping angled brackets. * * @group string_funcs * @since 3.5.0 @@ -4452,11 +4524,10 @@ object functions { def replace(src: Column, search: Column): Column = Column.fn("replace", src, search) /** - * Splits `str` by delimiter and return requested part of the split (1-based). - * If any input is null, returns null. if `partNum` is out of range of split parts, - * returns empty string. If `partNum` is 0, throws an error. If `partNum` is negative, - * the parts are counted backward from the end of the string. - * If the `delimiter` is an empty string, the `str` is not split. + * Splits `str` by delimiter and return requested part of the split (1-based). If any input is + * null, returns null. if `partNum` is out of range of split parts, returns empty string. If + * `partNum` is 0, throws an error. If `partNum` is negative, the parts are counted backward + * from the end of the string. If the `delimiter` is an empty string, the `str` is not split. * * @group string_funcs * @since 3.5.0 @@ -4465,8 +4536,8 @@ object functions { Column.fn("split_part", str, delimiter, partNum) /** - * Returns the substring of `str` that starts at `pos` and is of length `len`, - * or the slice of byte array that starts at `pos` and is of length `len`. + * Returns the substring of `str` that starts at `pos` and is of length `len`, or the slice of + * byte array that starts at `pos` and is of length `len`. * * @group string_funcs * @since 3.5.0 @@ -4475,8 +4546,8 @@ object functions { Column.fn("substr", str, pos, len) /** - * Returns the substring of `str` that starts at `pos`, - * or the slice of byte array that starts at `pos`. + * Returns the substring of `str` that starts at `pos`, or the slice of byte array that starts + * at `pos`. * * @group string_funcs * @since 3.5.0 @@ -4511,8 +4582,8 @@ object functions { Column.fn("printf", (format +: arguments): _*) /** - * Decodes a `str` in 'application/x-www-form-urlencoded' format - * using a specific encoding scheme. + * Decodes a `str` in 'application/x-www-form-urlencoded' format using a specific encoding + * scheme. * * @group url_funcs * @since 3.5.0 @@ -4520,8 +4591,8 @@ object functions { def url_decode(str: Column): Column = Column.fn("url_decode", str) /** - * This is a special version of `url_decode` that performs the same operation, but returns - * a NULL value instead of raising an error if the decoding cannot be performed. + * This is a special version of `url_decode` that performs the same operation, but returns a + * NULL value instead of raising an error if the decoding cannot be performed. * * @group url_funcs * @since 4.0.0 @@ -4529,8 +4600,8 @@ object functions { def try_url_decode(str: Column): Column = Column.fn("try_url_decode", str) /** - * Translates a string into 'application/x-www-form-urlencoded' format - * using a specific encoding scheme. + * Translates a string into 'application/x-www-form-urlencoded' format using a specific encoding + * scheme. * * @group url_funcs * @since 3.5.0 @@ -4538,8 +4609,8 @@ object functions { def url_encode(str: Column): Column = Column.fn("url_encode", str) /** - * Returns the position of the first occurrence of `substr` in `str` after position `start`. - * The given `start` and return value are 1-based. + * Returns the position of the first occurrence of `substr` in `str` after position `start`. The + * given `start` and return value are 1-based. * * @group string_funcs * @since 3.5.0 @@ -4548,8 +4619,8 @@ object functions { Column.fn("position", substr, str, start) /** - * Returns the position of the first occurrence of `substr` in `str` after position `1`. - * The return value are 1-based. + * Returns the position of the first occurrence of `substr` in `str` after position `1`. The + * return value are 1-based. * * @group string_funcs * @since 3.5.0 @@ -4558,9 +4629,9 @@ object functions { Column.fn("position", substr, str) /** - * Returns a boolean. The value is True if str ends with suffix. - * Returns NULL if either input expression is NULL. Otherwise, returns False. - * Both str or suffix must be of STRING or BINARY type. + * Returns a boolean. The value is True if str ends with suffix. Returns NULL if either input + * expression is NULL. Otherwise, returns False. Both str or suffix must be of STRING or BINARY + * type. * * @group string_funcs * @since 3.5.0 @@ -4569,9 +4640,9 @@ object functions { Column.fn("endswith", str, suffix) /** - * Returns a boolean. The value is True if str starts with prefix. - * Returns NULL if either input expression is NULL. Otherwise, returns False. - * Both str or prefix must be of STRING or BINARY type. + * Returns a boolean. The value is True if str starts with prefix. Returns NULL if either input + * expression is NULL. Otherwise, returns False. Both str or prefix must be of STRING or BINARY + * type. * * @group string_funcs * @since 3.5.0 @@ -4580,8 +4651,8 @@ object functions { Column.fn("startswith", str, prefix) /** - * Returns the ASCII character having the binary equivalent to `n`. - * If n is larger than 256 the result is equivalent to char(n % 256) + * Returns the ASCII character having the binary equivalent to `n`. If n is larger than 256 the + * result is equivalent to char(n % 256) * * @group string_funcs * @since 3.5.0 @@ -4633,9 +4704,8 @@ object functions { def try_to_number(e: Column, format: Column): Column = Column.fn("try_to_number", e, format) /** - * Returns the character length of string data or number of bytes of binary data. - * The length of string data includes the trailing spaces. - * The length of binary data includes binary zeros. + * Returns the character length of string data or number of bytes of binary data. The length of + * string data includes the trailing spaces. The length of binary data includes binary zeros. * * @group string_funcs * @since 3.5.0 @@ -4643,9 +4713,8 @@ object functions { def char_length(str: Column): Column = Column.fn("char_length", str) /** - * Returns the character length of string data or number of bytes of binary data. - * The length of string data includes the trailing spaces. - * The length of binary data includes binary zeros. + * Returns the character length of string data or number of bytes of binary data. The length of + * string data includes the trailing spaces. The length of binary data includes binary zeros. * * @group string_funcs * @since 3.5.0 @@ -4653,8 +4722,8 @@ object functions { def character_length(str: Column): Column = Column.fn("character_length", str) /** - * Returns the ASCII character having the binary equivalent to `n`. - * If n is larger than 256 the result is equivalent to chr(n % 256) + * Returns the ASCII character having the binary equivalent to `n`. If n is larger than 256 the + * result is equivalent to chr(n % 256) * * @group string_funcs * @since 3.5.0 @@ -4662,9 +4731,9 @@ object functions { def chr(n: Column): Column = Column.fn("chr", n) /** - * Returns a boolean. The value is True if right is found inside left. - * Returns NULL if either input expression is NULL. Otherwise, returns False. - * Both left or right must be of STRING or BINARY type. + * Returns a boolean. The value is True if right is found inside left. Returns NULL if either + * input expression is NULL. Otherwise, returns False. Both left or right must be of STRING or + * BINARY type. * * @group string_funcs * @since 3.5.0 @@ -4672,10 +4741,10 @@ object functions { def contains(left: Column, right: Column): Column = Column.fn("contains", left, right) /** - * Returns the `n`-th input, e.g., returns `input2` when `n` is 2. - * The function returns NULL if the index exceeds the length of the array - * and `spark.sql.ansi.enabled` is set to false. If `spark.sql.ansi.enabled` is set to true, - * it throws ArrayIndexOutOfBoundsException for invalid indices. + * Returns the `n`-th input, e.g., returns `input2` when `n` is 2. The function returns NULL if + * the index exceeds the length of the array and `spark.sql.ansi.enabled` is set to false. If + * `spark.sql.ansi.enabled` is set to true, it throws ArrayIndexOutOfBoundsException for invalid + * indices. * * @group string_funcs * @since 3.5.0 @@ -4684,9 +4753,9 @@ object functions { def elt(inputs: Column*): Column = Column.fn("elt", inputs: _*) /** - * Returns the index (1-based) of the given string (`str`) in the comma-delimited - * list (`strArray`). Returns 0, if the string was not found or if the given string (`str`) - * contains a comma. + * Returns the index (1-based) of the given string (`str`) in the comma-delimited list + * (`strArray`). Returns 0, if the string was not found or if the given string (`str`) contains + * a comma. * * @group string_funcs * @since 3.5.0 @@ -4748,8 +4817,8 @@ object functions { def ucase(str: Column): Column = Column.fn("ucase", str) /** - * Returns the leftmost `len`(`len` can be string type) characters from the string `str`, - * if `len` is less or equal than 0 the result is an empty string. + * Returns the leftmost `len`(`len` can be string type) characters from the string `str`, if + * `len` is less or equal than 0 the result is an empty string. * * @group string_funcs * @since 3.5.0 @@ -4757,8 +4826,8 @@ object functions { def left(str: Column, len: Column): Column = Column.fn("left", str, len) /** - * Returns the rightmost `len`(`len` can be string type) characters from the string `str`, - * if `len` is less or equal than 0 the result is an empty string. + * Returns the rightmost `len`(`len` can be string type) characters from the string `str`, if + * `len` is less or equal than 0 the result is an empty string. * * @group string_funcs * @since 3.5.0 @@ -4772,23 +4841,29 @@ object functions { /** * Returns the date that is `numMonths` after `startDate`. * - * @param startDate A date, timestamp or string. If a string, the data must be in a format that - * can be cast to a date, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` - * @param numMonths The number of months to add to `startDate`, can be negative to subtract months - * @return A date, or null if `startDate` was a string that could not be cast to a date + * @param startDate + * A date, timestamp or string. If a string, the data must be in a format that can be cast to + * a date, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` + * @param numMonths + * The number of months to add to `startDate`, can be negative to subtract months + * @return + * A date, or null if `startDate` was a string that could not be cast to a date * @group datetime_funcs * @since 1.5.0 */ - def add_months(startDate: Column, numMonths: Int): Column = add_months(startDate, lit(numMonths)) + def add_months(startDate: Column, numMonths: Int): Column = + add_months(startDate, lit(numMonths)) /** * Returns the date that is `numMonths` after `startDate`. * - * @param startDate A date, timestamp or string. If a string, the data must be in a format that - * can be cast to a date, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` - * @param numMonths A column of the number of months to add to `startDate`, can be negative to - * subtract months - * @return A date, or null if `startDate` was a string that could not be cast to a date + * @param startDate + * A date, timestamp or string. If a string, the data must be in a format that can be cast to + * a date, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` + * @param numMonths + * A column of the number of months to add to `startDate`, can be negative to subtract months + * @return + * A date, or null if `startDate` was a string that could not be cast to a date * @group datetime_funcs * @since 3.0.0 */ @@ -4796,8 +4871,8 @@ object functions { Column.fn("add_months", startDate, numMonths) /** - * Returns the current date at the start of query evaluation as a date column. - * All calls of current_date within the same query return the same value. + * Returns the current date at the start of query evaluation as a date column. All calls of + * current_date within the same query return the same value. * * @group datetime_funcs * @since 3.5.0 @@ -4805,8 +4880,8 @@ object functions { def curdate(): Column = Column.fn("curdate") /** - * Returns the current date at the start of query evaluation as a date column. - * All calls of current_date within the same query return the same value. + * Returns the current date at the start of query evaluation as a date column. All calls of + * current_date within the same query return the same value. * * @group datetime_funcs * @since 1.5.0 @@ -4822,8 +4897,8 @@ object functions { def current_timezone(): Column = Column.fn("current_timezone") /** - * Returns the current timestamp at the start of query evaluation as a timestamp column. - * All calls of current_timestamp within the same query return the same value. + * Returns the current timestamp at the start of query evaluation as a timestamp column. All + * calls of current_timestamp within the same query return the same value. * * @group datetime_funcs * @since 1.5.0 @@ -4839,9 +4914,9 @@ object functions { def now(): Column = Column.fn("now") /** - * Returns the current timestamp without time zone at the start of query evaluation - * as a timestamp without time zone column. - * All calls of localtimestamp within the same query return the same value. + * Returns the current timestamp without time zone at the start of query evaluation as a + * timestamp without time zone column. All calls of localtimestamp within the same query return + * the same value. * * @group datetime_funcs * @since 3.3.0 @@ -4852,17 +4927,21 @@ object functions { * Converts a date/timestamp/string to a value of string in the format specified by the date * format given by the second argument. * - * See - * Datetime Patterns - * for valid date and time format patterns - * - * @param dateExpr A date, timestamp or string. If a string, the data must be in a format that - * can be cast to a timestamp, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` - * @param format A pattern `dd.MM.yyyy` would return a string like `18.03.1993` - * @return A string, or null if `dateExpr` was a string that could not be cast to a timestamp - * @note Use specialized functions like [[year]] whenever possible as they benefit from a - * specialized implementation. - * @throws IllegalArgumentException if the `format` pattern is invalid + * See Datetime + * Patterns for valid date and time format patterns + * + * @param dateExpr + * A date, timestamp or string. If a string, the data must be in a format that can be cast to + * a timestamp, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` + * @param format + * A pattern `dd.MM.yyyy` would return a string like `18.03.1993` + * @return + * A string, or null if `dateExpr` was a string that could not be cast to a timestamp + * @note + * Use specialized functions like [[year]] whenever possible as they benefit from a + * specialized implementation. + * @throws IllegalArgumentException + * if the `format` pattern is invalid * @group datetime_funcs * @since 1.5.0 */ @@ -4872,10 +4951,13 @@ object functions { /** * Returns the date that is `days` days after `start` * - * @param start A date, timestamp or string. If a string, the data must be in a format that - * can be cast to a date, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` - * @param days The number of days to add to `start`, can be negative to subtract days - * @return A date, or null if `start` was a string that could not be cast to a date + * @param start + * A date, timestamp or string. If a string, the data must be in a format that can be cast to + * a date, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` + * @param days + * The number of days to add to `start`, can be negative to subtract days + * @return + * A date, or null if `start` was a string that could not be cast to a date * @group datetime_funcs * @since 1.5.0 */ @@ -4884,10 +4966,13 @@ object functions { /** * Returns the date that is `days` days after `start` * - * @param start A date, timestamp or string. If a string, the data must be in a format that - * can be cast to a date, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` - * @param days A column of the number of days to add to `start`, can be negative to subtract days - * @return A date, or null if `start` was a string that could not be cast to a date + * @param start + * A date, timestamp or string. If a string, the data must be in a format that can be cast to + * a date, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` + * @param days + * A column of the number of days to add to `start`, can be negative to subtract days + * @return + * A date, or null if `start` was a string that could not be cast to a date * @group datetime_funcs * @since 3.0.0 */ @@ -4896,10 +4981,13 @@ object functions { /** * Returns the date that is `days` days after `start` * - * @param start A date, timestamp or string. If a string, the data must be in a format that - * can be cast to a date, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` - * @param days A column of the number of days to add to `start`, can be negative to subtract days - * @return A date, or null if `start` was a string that could not be cast to a date + * @param start + * A date, timestamp or string. If a string, the data must be in a format that can be cast to + * a date, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` + * @param days + * A column of the number of days to add to `start`, can be negative to subtract days + * @return + * A date, or null if `start` was a string that could not be cast to a date * @group datetime_funcs * @since 3.5.0 */ @@ -4908,10 +4996,13 @@ object functions { /** * Returns the date that is `days` days before `start` * - * @param start A date, timestamp or string. If a string, the data must be in a format that - * can be cast to a date, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` - * @param days The number of days to subtract from `start`, can be negative to add days - * @return A date, or null if `start` was a string that could not be cast to a date + * @param start + * A date, timestamp or string. If a string, the data must be in a format that can be cast to + * a date, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` + * @param days + * The number of days to subtract from `start`, can be negative to add days + * @return + * A date, or null if `start` was a string that could not be cast to a date * @group datetime_funcs * @since 1.5.0 */ @@ -4920,11 +5011,13 @@ object functions { /** * Returns the date that is `days` days before `start` * - * @param start A date, timestamp or string. If a string, the data must be in a format that - * can be cast to a date, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` - * @param days A column of the number of days to subtract from `start`, can be negative to add - * days - * @return A date, or null if `start` was a string that could not be cast to a date + * @param start + * A date, timestamp or string. If a string, the data must be in a format that can be cast to + * a date, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` + * @param days + * A column of the number of days to subtract from `start`, can be negative to add days + * @return + * A date, or null if `start` was a string that could not be cast to a date * @group datetime_funcs * @since 3.0.0 */ @@ -4940,12 +5033,15 @@ object functions { * // returns 1 * }}} * - * @param end A date, timestamp or string. If a string, the data must be in a format that - * can be cast to a date, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` - * @param start A date, timestamp or string. If a string, the data must be in a format that - * can be cast to a date, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` - * @return An integer, or null if either `end` or `start` were strings that could not be cast to - * a date. Negative if `end` is before `start` + * @param end + * A date, timestamp or string. If a string, the data must be in a format that can be cast to + * a date, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` + * @param start + * A date, timestamp or string. If a string, the data must be in a format that can be cast to + * a date, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` + * @return + * An integer, or null if either `end` or `start` were strings that could not be cast to a + * date. Negative if `end` is before `start` * @group datetime_funcs * @since 1.5.0 */ @@ -4960,12 +5056,15 @@ object functions { * // returns 1 * }}} * - * @param end A date, timestamp or string. If a string, the data must be in a format that - * can be cast to a date, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` - * @param start A date, timestamp or string. If a string, the data must be in a format that - * can be cast to a date, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` - * @return An integer, or null if either `end` or `start` were strings that could not be cast to - * a date. Negative if `end` is before `start` + * @param end + * A date, timestamp or string. If a string, the data must be in a format that can be cast to + * a date, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` + * @param start + * A date, timestamp or string. If a string, the data must be in a format that can be cast to + * a date, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` + * @return + * An integer, or null if either `end` or `start` were strings that could not be cast to a + * date. Negative if `end` is before `start` * @group datetime_funcs * @since 3.5.0 */ @@ -4981,7 +5080,8 @@ object functions { /** * Extracts the year as an integer from a given date/timestamp/string. - * @return An integer, or null if the input was a string that could not be cast to a date + * @return + * An integer, or null if the input was a string that could not be cast to a date * @group datetime_funcs * @since 1.5.0 */ @@ -4989,7 +5089,8 @@ object functions { /** * Extracts the quarter as an integer from a given date/timestamp/string. - * @return An integer, or null if the input was a string that could not be cast to a date + * @return + * An integer, or null if the input was a string that could not be cast to a date * @group datetime_funcs * @since 1.5.0 */ @@ -4997,16 +5098,18 @@ object functions { /** * Extracts the month as an integer from a given date/timestamp/string. - * @return An integer, or null if the input was a string that could not be cast to a date + * @return + * An integer, or null if the input was a string that could not be cast to a date * @group datetime_funcs * @since 1.5.0 */ def month(e: Column): Column = Column.fn("month", e) /** - * Extracts the day of the week as an integer from a given date/timestamp/string. - * Ranges from 1 for a Sunday through to 7 for a Saturday - * @return An integer, or null if the input was a string that could not be cast to a date + * Extracts the day of the week as an integer from a given date/timestamp/string. Ranges from 1 + * for a Sunday through to 7 for a Saturday + * @return + * An integer, or null if the input was a string that could not be cast to a date * @group datetime_funcs * @since 2.3.0 */ @@ -5014,7 +5117,8 @@ object functions { /** * Extracts the day of the month as an integer from a given date/timestamp/string. - * @return An integer, or null if the input was a string that could not be cast to a date + * @return + * An integer, or null if the input was a string that could not be cast to a date * @group datetime_funcs * @since 1.5.0 */ @@ -5022,7 +5126,8 @@ object functions { /** * Extracts the day of the month as an integer from a given date/timestamp/string. - * @return An integer, or null if the input was a string that could not be cast to a date + * @return + * An integer, or null if the input was a string that could not be cast to a date * @group datetime_funcs * @since 3.5.0 */ @@ -5030,7 +5135,8 @@ object functions { /** * Extracts the day of the year as an integer from a given date/timestamp/string. - * @return An integer, or null if the input was a string that could not be cast to a date + * @return + * An integer, or null if the input was a string that could not be cast to a date * @group datetime_funcs * @since 1.5.0 */ @@ -5038,7 +5144,8 @@ object functions { /** * Extracts the hours as an integer from a given date/timestamp/string. - * @return An integer, or null if the input was a string that could not be cast to a date + * @return + * An integer, or null if the input was a string that could not be cast to a date * @group datetime_funcs * @since 1.5.0 */ @@ -5047,9 +5154,12 @@ object functions { /** * Extracts a part of the date/timestamp or interval source. * - * @param field selects which part of the source should be extracted. - * @param source a date/timestamp or interval column from where `field` should be extracted. - * @return a part of the date/timestamp or interval source + * @param field + * selects which part of the source should be extracted. + * @param source + * a date/timestamp or interval column from where `field` should be extracted. + * @return + * a part of the date/timestamp or interval source * @group datetime_funcs * @since 3.5.0 */ @@ -5060,10 +5170,13 @@ object functions { /** * Extracts a part of the date/timestamp or interval source. * - * @param field selects which part of the source should be extracted, and supported string values - * are as same as the fields of the equivalent function `extract`. - * @param source a date/timestamp or interval column from where `field` should be extracted. - * @return a part of the date/timestamp or interval source + * @param field + * selects which part of the source should be extracted, and supported string values are as + * same as the fields of the equivalent function `extract`. + * @param source + * a date/timestamp or interval column from where `field` should be extracted. + * @return + * a part of the date/timestamp or interval source * @group datetime_funcs * @since 3.5.0 */ @@ -5074,10 +5187,13 @@ object functions { /** * Extracts a part of the date/timestamp or interval source. * - * @param field selects which part of the source should be extracted, and supported string values - * are as same as the fields of the equivalent function `EXTRACT`. - * @param source a date/timestamp or interval column from where `field` should be extracted. - * @return a part of the date/timestamp or interval source + * @param field + * selects which part of the source should be extracted, and supported string values are as + * same as the fields of the equivalent function `EXTRACT`. + * @param source + * a date/timestamp or interval column from where `field` should be extracted. + * @return + * a part of the date/timestamp or interval source * @group datetime_funcs * @since 3.5.0 */ @@ -5086,13 +5202,14 @@ object functions { } /** - * Returns the last day of the month which the given date belongs to. - * For example, input "2015-07-27" returns "2015-07-31" since July 31 is the last day of the - * month in July 2015. + * Returns the last day of the month which the given date belongs to. For example, input + * "2015-07-27" returns "2015-07-31" since July 31 is the last day of the month in July 2015. * - * @param e A date, timestamp or string. If a string, the data must be in a format that can be - * cast to a date, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` - * @return A date, or null if the input was a string that could not be cast to a date + * @param e + * A date, timestamp or string. If a string, the data must be in a format that can be cast to + * a date, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` + * @return + * A date, or null if the input was a string that could not be cast to a date * @group datetime_funcs * @since 1.5.0 */ @@ -5100,7 +5217,8 @@ object functions { /** * Extracts the minutes as an integer from a given date/timestamp/string. - * @return An integer, or null if the input was a string that could not be cast to a date + * @return + * An integer, or null if the input was a string that could not be cast to a date * @group datetime_funcs * @since 1.5.0 */ @@ -5115,7 +5233,8 @@ object functions { def weekday(e: Column): Column = Column.fn("weekday", e) /** - * @return A date created from year, month and day fields. + * @return + * A date created from year, month and day fields. * @group datetime_funcs * @since 3.3.0 */ @@ -5126,7 +5245,8 @@ object functions { * Returns number of months between dates `start` and `end`. * * A whole number is returned if both inputs have the same day of month or both are the last day - * of their respective months. Otherwise, the difference is calculated assuming 31 days per month. + * of their respective months. Otherwise, the difference is calculated assuming 31 days per + * month. * * For example: * {{{ @@ -5135,12 +5255,15 @@ object functions { * months_between("2017-06-01", "2017-06-16 12:00:00") // returns -0.5 * }}} * - * @param end A date, timestamp or string. If a string, the data must be in a format that can - * be cast to a timestamp, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` - * @param start A date, timestamp or string. If a string, the data must be in a format that can - * cast to a timestamp, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` - * @return A double, or null if either `end` or `start` were strings that could not be cast to a - * timestamp. Negative if `end` is before `start` + * @param end + * A date, timestamp or string. If a string, the data must be in a format that can be cast to + * a timestamp, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` + * @param start + * A date, timestamp or string. If a string, the data must be in a format that can cast to a + * timestamp, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` + * @return + * A double, or null if either `end` or `start` were strings that could not be cast to a + * timestamp. Negative if `end` is before `start` * @group datetime_funcs * @since 1.5.0 */ @@ -5163,11 +5286,14 @@ object functions { * For example, `next_day('2015-07-27', "Sunday")` returns 2015-08-02 because that is the first * Sunday after 2015-07-27. * - * @param date A date, timestamp or string. If a string, the data must be in a format that - * can be cast to a date, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` - * @param dayOfWeek Case insensitive, and accepts: "Mon", "Tue", "Wed", "Thu", "Fri", "Sat", "Sun" - * @return A date, or null if `date` was a string that could not be cast to a date or if - * `dayOfWeek` was an invalid value + * @param date + * A date, timestamp or string. If a string, the data must be in a format that can be cast to + * a date, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` + * @param dayOfWeek + * Case insensitive, and accepts: "Mon", "Tue", "Wed", "Thu", "Fri", "Sat", "Sun" + * @return + * A date, or null if `date` was a string that could not be cast to a date or if `dayOfWeek` + * was an invalid value * @group datetime_funcs * @since 1.5.0 */ @@ -5180,12 +5306,15 @@ object functions { * For example, `next_day('2015-07-27', "Sunday")` returns 2015-08-02 because that is the first * Sunday after 2015-07-27. * - * @param date A date, timestamp or string. If a string, the data must be in a format that - * can be cast to a date, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` - * @param dayOfWeek A column of the day of week. Case insensitive, and accepts: "Mon", "Tue", - * "Wed", "Thu", "Fri", "Sat", "Sun" - * @return A date, or null if `date` was a string that could not be cast to a date or if - * `dayOfWeek` was an invalid value + * @param date + * A date, timestamp or string. If a string, the data must be in a format that can be cast to + * a date, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` + * @param dayOfWeek + * A column of the day of week. Case insensitive, and accepts: "Mon", "Tue", "Wed", "Thu", + * "Fri", "Sat", "Sun" + * @return + * A date, or null if `date` was a string that could not be cast to a date or if `dayOfWeek` + * was an invalid value * @group datetime_funcs * @since 3.2.0 */ @@ -5194,7 +5323,8 @@ object functions { /** * Extracts the seconds as an integer from a given date/timestamp/string. - * @return An integer, or null if the input was a string that could not be cast to a timestamp + * @return + * An integer, or null if the input was a string that could not be cast to a timestamp * @group datetime_funcs * @since 1.5.0 */ @@ -5206,7 +5336,8 @@ object functions { * A week is considered to start on a Monday and week 1 is the first week with more than 3 days, * as defined by ISO 8601 * - * @return An integer, or null if the input was a string that could not be cast to a date + * @return + * An integer, or null if the input was a string that could not be cast to a date * @group datetime_funcs * @since 1.5.0 */ @@ -5214,12 +5345,14 @@ object functions { /** * Converts the number of seconds from unix epoch (1970-01-01 00:00:00 UTC) to a string - * representing the timestamp of that moment in the current system time zone in the - * yyyy-MM-dd HH:mm:ss format. - * - * @param ut A number of a type that is castable to a long, such as string or integer. Can be - * negative for timestamps before the unix epoch - * @return A string, or null if the input was a string that could not be cast to a long + * representing the timestamp of that moment in the current system time zone in the yyyy-MM-dd + * HH:mm:ss format. + * + * @param ut + * A number of a type that is castable to a long, such as string or integer. Can be negative + * for timestamps before the unix epoch + * @return + * A string, or null if the input was a string that could not be cast to a long * @group datetime_funcs * @since 1.5.0 */ @@ -5230,15 +5363,17 @@ object functions { * representing the timestamp of that moment in the current system time zone in the given * format. * - * See - * Datetime Patterns - * for valid date and time format patterns - * - * @param ut A number of a type that is castable to a long, such as string or integer. Can be - * negative for timestamps before the unix epoch - * @param f A date time pattern that the input will be formatted to - * @return A string, or null if `ut` was a string that could not be cast to a long or `f` was - * an invalid date time pattern + * See Datetime + * Patterns for valid date and time format patterns + * + * @param ut + * A number of a type that is castable to a long, such as string or integer. Can be negative + * for timestamps before the unix epoch + * @param f + * A date time pattern that the input will be formatted to + * @return + * A string, or null if `ut` was a string that could not be cast to a long or `f` was an + * invalid date time pattern * @group datetime_funcs * @since 1.5.0 */ @@ -5248,8 +5383,9 @@ object functions { /** * Returns the current Unix timestamp (in seconds) as a long. * - * @note All calls of `unix_timestamp` within the same query return the same value - * (i.e. the current timestamp is calculated at the start of query evaluation). + * @note + * All calls of `unix_timestamp` within the same query return the same value (i.e. the current + * timestamp is calculated at the start of query evaluation). * * @group datetime_funcs * @since 1.5.0 @@ -5257,12 +5393,14 @@ object functions { def unix_timestamp(): Column = unix_timestamp(current_timestamp()) /** - * Converts time string in format yyyy-MM-dd HH:mm:ss to Unix timestamp (in seconds), - * using the default timezone and the default locale. + * Converts time string in format yyyy-MM-dd HH:mm:ss to Unix timestamp (in seconds), using the + * default timezone and the default locale. * - * @param s A date, timestamp or string. If a string, the data must be in the - * `yyyy-MM-dd HH:mm:ss` format - * @return A long, or null if the input was a string not of the correct format + * @param s + * A date, timestamp or string. If a string, the data must be in the `yyyy-MM-dd HH:mm:ss` + * format + * @return + * A long, or null if the input was a string not of the correct format * @group datetime_funcs * @since 1.5.0 */ @@ -5271,15 +5409,17 @@ object functions { /** * Converts time string with given pattern to Unix timestamp (in seconds). * - * See - * Datetime Patterns - * for valid date and time format patterns - * - * @param s A date, timestamp or string. If a string, the data must be in a format that can be - * cast to a date, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` - * @param p A date time pattern detailing the format of `s` when `s` is a string - * @return A long, or null if `s` was a string that could not be cast to a date or `p` was - * an invalid format + * See Datetime + * Patterns for valid date and time format patterns + * + * @param s + * A date, timestamp or string. If a string, the data must be in a format that can be cast to + * a date, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` + * @param p + * A date time pattern detailing the format of `s` when `s` is a string + * @return + * A long, or null if `s` was a string that could not be cast to a date or `p` was an invalid + * format * @group datetime_funcs * @since 1.5.0 */ @@ -5289,9 +5429,11 @@ object functions { /** * Converts to a timestamp by casting rules to `TimestampType`. * - * @param s A date, timestamp or string. If a string, the data must be in a format that can be - * cast to a timestamp, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` - * @return A timestamp, or null if the input was a string that could not be cast to a timestamp + * @param s + * A date, timestamp or string. If a string, the data must be in a format that can be cast to + * a timestamp, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` + * @return + * A timestamp, or null if the input was a string that could not be cast to a timestamp * @group datetime_funcs * @since 2.2.0 */ @@ -5300,15 +5442,17 @@ object functions { /** * Converts time string with the given pattern to timestamp. * - * See - * Datetime Patterns - * for valid date and time format patterns - * - * @param s A date, timestamp or string. If a string, the data must be in a format that can be - * cast to a timestamp, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` - * @param fmt A date time pattern detailing the format of `s` when `s` is a string - * @return A timestamp, or null if `s` was a string that could not be cast to a timestamp or - * `fmt` was an invalid format + * See Datetime + * Patterns for valid date and time format patterns + * + * @param s + * A date, timestamp or string. If a string, the data must be in a format that can be cast to + * a timestamp, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` + * @param fmt + * A date time pattern detailing the format of `s` when `s` is a string + * @return + * A timestamp, or null if `s` was a string that could not be cast to a timestamp or `fmt` was + * an invalid format * @group datetime_funcs * @since 2.2.0 */ @@ -5326,9 +5470,9 @@ object functions { Column.fn("try_to_timestamp", s, format) /** - * Parses the `s` to a timestamp. The function always returns null on an invalid - * input with`/`without ANSI SQL mode enabled. It follows casting rules to a timestamp. The - * result data type is consistent with the value of configuration `spark.sql.timestampType`. + * Parses the `s` to a timestamp. The function always returns null on an invalid input + * with`/`without ANSI SQL mode enabled. It follows casting rules to a timestamp. The result + * data type is consistent with the value of configuration `spark.sql.timestampType`. * * @group datetime_funcs * @since 3.5.0 @@ -5346,15 +5490,17 @@ object functions { /** * Converts the column into a `DateType` with a specified format * - * See - * Datetime Patterns - * for valid date and time format patterns - * - * @param e A date, timestamp or string. If a string, the data must be in a format that can be - * cast to a date, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` - * @param fmt A date time pattern detailing the format of `e` when `e`is a string - * @return A date, or null if `e` was a string that could not be cast to a date or `fmt` was an - * invalid format + * See Datetime + * Patterns for valid date and time format patterns + * + * @param e + * A date, timestamp or string. If a string, the data must be in a format that can be cast to + * a date, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` + * @param fmt + * A date time pattern detailing the format of `e` when `e`is a string + * @return + * A date, or null if `e` was a string that could not be cast to a date or `fmt` was an + * invalid format * @group datetime_funcs * @since 2.2.0 */ @@ -5377,8 +5523,8 @@ object functions { def unix_micros(e: Column): Column = Column.fn("unix_micros", e) /** - * Returns the number of milliseconds since 1970-01-01 00:00:00 UTC. - * Truncates higher levels of precision. + * Returns the number of milliseconds since 1970-01-01 00:00:00 UTC. Truncates higher levels of + * precision. * * @group datetime_funcs * @since 3.5.0 @@ -5386,8 +5532,8 @@ object functions { def unix_millis(e: Column): Column = Column.fn("unix_millis", e) /** - * Returns the number of seconds since 1970-01-01 00:00:00 UTC. - * Truncates higher levels of precision. + * Returns the number of seconds since 1970-01-01 00:00:00 UTC. Truncates higher levels of + * precision. * * @group datetime_funcs * @since 3.5.0 @@ -5399,14 +5545,16 @@ object functions { * * For example, `trunc("2018-11-19 12:01:19", "year")` returns 2018-01-01 * - * @param date A date, timestamp or string. If a string, the data must be in a format that can be - * cast to a date, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` - * @param format: 'year', 'yyyy', 'yy' to truncate by year, - * or 'month', 'mon', 'mm' to truncate by month - * Other options are: 'week', 'quarter' + * @param date + * A date, timestamp or string. If a string, the data must be in a format that can be cast to + * a date, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` + * @param format: + * 'year', 'yyyy', 'yy' to truncate by year, or 'month', 'mon', 'mm' to truncate by month + * Other options are: 'week', 'quarter' * - * @return A date, or null if `date` was a string that could not be cast to a date or `format` - * was an invalid value + * @return + * A date, or null if `date` was a string that could not be cast to a date or `format` was an + * invalid value * @group datetime_funcs * @since 1.5.0 */ @@ -5417,15 +5565,16 @@ object functions { * * For example, `date_trunc("year", "2018-11-19 12:01:19")` returns 2018-01-01 00:00:00 * - * @param format: 'year', 'yyyy', 'yy' to truncate by year, - * 'month', 'mon', 'mm' to truncate by month, - * 'day', 'dd' to truncate by day, - * Other options are: - * 'microsecond', 'millisecond', 'second', 'minute', 'hour', 'week', 'quarter' - * @param timestamp A date, timestamp or string. If a string, the data must be in a format that - * can be cast to a timestamp, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` - * @return A timestamp, or null if `timestamp` was a string that could not be cast to a timestamp - * or `format` was an invalid value + * @param format: + * 'year', 'yyyy', 'yy' to truncate by year, 'month', 'mon', 'mm' to truncate by month, 'day', + * 'dd' to truncate by day, Other options are: 'microsecond', 'millisecond', 'second', + * 'minute', 'hour', 'week', 'quarter' + * @param timestamp + * A date, timestamp or string. If a string, the data must be in a format that can be cast to + * a timestamp, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` + * @return + * A timestamp, or null if `timestamp` was a string that could not be cast to a timestamp or + * `format` was an invalid value * @group datetime_funcs * @since 2.3.0 */ @@ -5434,19 +5583,21 @@ object functions { /** * Given a timestamp like '2017-07-14 02:40:00.0', interprets it as a time in UTC, and renders - * that time as a timestamp in the given time zone. For example, 'GMT+1' would yield - * '2017-07-14 03:40:00.0'. - * - * @param ts A date, timestamp or string. If a string, the data must be in a format that can be - * cast to a timestamp, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` - * @param tz A string detailing the time zone ID that the input should be adjusted to. It should - * be in the format of either region-based zone IDs or zone offsets. Region IDs must - * have the form 'area/city', such as 'America/Los_Angeles'. Zone offsets must be in - * the format '(+|-)HH:mm', for example '-08:00' or '+01:00'. Also 'UTC' and 'Z' are - * supported as aliases of '+00:00'. Other short names are not recommended to use - * because they can be ambiguous. - * @return A timestamp, or null if `ts` was a string that could not be cast to a timestamp or - * `tz` was an invalid value + * that time as a timestamp in the given time zone. For example, 'GMT+1' would yield '2017-07-14 + * 03:40:00.0'. + * + * @param ts + * A date, timestamp or string. If a string, the data must be in a format that can be cast to + * a timestamp, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` + * @param tz + * A string detailing the time zone ID that the input should be adjusted to. It should be in + * the format of either region-based zone IDs or zone offsets. Region IDs must have the form + * 'area/city', such as 'America/Los_Angeles'. Zone offsets must be in the format + * '(+|-)HH:mm', for example '-08:00' or '+01:00'. Also 'UTC' and 'Z' are supported as aliases + * of '+00:00'. Other short names are not recommended to use because they can be ambiguous. + * @return + * A timestamp, or null if `ts` was a string that could not be cast to a timestamp or `tz` was + * an invalid value * @group datetime_funcs * @since 1.5.0 */ @@ -5454,8 +5605,8 @@ object functions { /** * Given a timestamp like '2017-07-14 02:40:00.0', interprets it as a time in UTC, and renders - * that time as a timestamp in the given time zone. For example, 'GMT+1' would yield - * '2017-07-14 03:40:00.0'. + * that time as a timestamp in the given time zone. For example, 'GMT+1' would yield '2017-07-14 + * 03:40:00.0'. * @group datetime_funcs * @since 2.4.0 */ @@ -5467,16 +5618,18 @@ object functions { * zone, and renders that time as a timestamp in UTC. For example, 'GMT+1' would yield * '2017-07-14 01:40:00.0'. * - * @param ts A date, timestamp or string. If a string, the data must be in a format that can be - * cast to a timestamp, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` - * @param tz A string detailing the time zone ID that the input should be adjusted to. It should - * be in the format of either region-based zone IDs or zone offsets. Region IDs must - * have the form 'area/city', such as 'America/Los_Angeles'. Zone offsets must be in - * the format '(+|-)HH:mm', for example '-08:00' or '+01:00'. Also 'UTC' and 'Z' are - * supported as aliases of '+00:00'. Other short names are not recommended to use - * because they can be ambiguous. - * @return A timestamp, or null if `ts` was a string that could not be cast to a timestamp or - * `tz` was an invalid value + * @param ts + * A date, timestamp or string. If a string, the data must be in a format that can be cast to + * a timestamp, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` + * @param tz + * A string detailing the time zone ID that the input should be adjusted to. It should be in + * the format of either region-based zone IDs or zone offsets. Region IDs must have the form + * 'area/city', such as 'America/Los_Angeles'. Zone offsets must be in the format + * '(+|-)HH:mm', for example '-08:00' or '+01:00'. Also 'UTC' and 'Z' are supported as aliases + * of '+00:00'. Other short names are not recommended to use because they can be ambiguous. + * @return + * A timestamp, or null if `ts` was a string that could not be cast to a timestamp or `tz` was + * an invalid value * @group datetime_funcs * @since 1.5.0 */ @@ -5495,8 +5648,8 @@ object functions { * Bucketize rows into one or more time windows given a timestamp specifying column. Window * starts are inclusive but the window ends are exclusive, e.g. 12:05 will be in the window * [12:05,12:10) but not in [12:00,12:05). Windows can support microsecond precision. Windows in - * the order of months are not supported. The following example takes the average stock price for - * a one minute window every 10 seconds starting 5 seconds after the hour: + * the order of months are not supported. The following example takes the average stock price + * for a one minute window every 10 seconds starting 5 seconds after the hour: * * {{{ * val df = ... // schema => timestamp: TimestampType, stockId: StringType, price: DoubleType @@ -5515,23 +5668,23 @@ object functions { * For a streaming query, you may use the function `current_timestamp` to generate windows on * processing time. * - * @param timeColumn The column or the expression to use as the timestamp for windowing by time. - * The time column must be of TimestampType or TimestampNTZType. - * @param windowDuration A string specifying the width of the window, e.g. `10 minutes`, - * `1 second`. Check `org.apache.spark.unsafe.types.CalendarInterval` for - * valid duration identifiers. Note that the duration is a fixed length of - * time, and does not vary over time according to a calendar. For example, - * `1 day` always means 86,400,000 milliseconds, not a calendar day. - * @param slideDuration A string specifying the sliding interval of the window, e.g. `1 minute`. - * A new window will be generated every `slideDuration`. Must be less than - * or equal to the `windowDuration`. Check - * `org.apache.spark.unsafe.types.CalendarInterval` for valid duration - * identifiers. This duration is likewise absolute, and does not vary - * according to a calendar. - * @param startTime The offset with respect to 1970-01-01 00:00:00 UTC with which to start - * window intervals. For example, in order to have hourly tumbling windows that - * start 15 minutes past the hour, e.g. 12:15-13:15, 13:15-14:15... provide - * `startTime` as `15 minutes`. + * @param timeColumn + * The column or the expression to use as the timestamp for windowing by time. The time column + * must be of TimestampType or TimestampNTZType. + * @param windowDuration + * A string specifying the width of the window, e.g. `10 minutes`, `1 second`. Check + * `org.apache.spark.unsafe.types.CalendarInterval` for valid duration identifiers. Note that + * the duration is a fixed length of time, and does not vary over time according to a + * calendar. For example, `1 day` always means 86,400,000 milliseconds, not a calendar day. + * @param slideDuration + * A string specifying the sliding interval of the window, e.g. `1 minute`. A new window will + * be generated every `slideDuration`. Must be less than or equal to the `windowDuration`. + * Check `org.apache.spark.unsafe.types.CalendarInterval` for valid duration identifiers. This + * duration is likewise absolute, and does not vary according to a calendar. + * @param startTime + * The offset with respect to 1970-01-01 00:00:00 UTC with which to start window intervals. + * For example, in order to have hourly tumbling windows that start 15 minutes past the hour, + * e.g. 12:15-13:15, 13:15-14:15... provide `startTime` as `15 minutes`. * * @group datetime_funcs * @since 2.0.0 @@ -5547,8 +5700,9 @@ object functions { * Bucketize rows into one or more time windows given a timestamp specifying column. Window * starts are inclusive but the window ends are exclusive, e.g. 12:05 will be in the window * [12:05,12:10) but not in [12:00,12:05). Windows can support microsecond precision. Windows in - * the order of months are not supported. The windows start beginning at 1970-01-01 00:00:00 UTC. - * The following example takes the average stock price for a one minute window every 10 seconds: + * the order of months are not supported. The windows start beginning at 1970-01-01 00:00:00 + * UTC. The following example takes the average stock price for a one minute window every 10 + * seconds: * * {{{ * val df = ... // schema => timestamp: TimestampType, stockId: StringType, price: DoubleType @@ -5567,19 +5721,19 @@ object functions { * For a streaming query, you may use the function `current_timestamp` to generate windows on * processing time. * - * @param timeColumn The column or the expression to use as the timestamp for windowing by time. - * The time column must be of TimestampType or TimestampNTZType. - * @param windowDuration A string specifying the width of the window, e.g. `10 minutes`, - * `1 second`. Check `org.apache.spark.unsafe.types.CalendarInterval` for - * valid duration identifiers. Note that the duration is a fixed length of - * time, and does not vary over time according to a calendar. For example, - * `1 day` always means 86,400,000 milliseconds, not a calendar day. - * @param slideDuration A string specifying the sliding interval of the window, e.g. `1 minute`. - * A new window will be generated every `slideDuration`. Must be less than - * or equal to the `windowDuration`. Check - * `org.apache.spark.unsafe.types.CalendarInterval` for valid duration - * identifiers. This duration is likewise absolute, and does not vary - * according to a calendar. + * @param timeColumn + * The column or the expression to use as the timestamp for windowing by time. The time column + * must be of TimestampType or TimestampNTZType. + * @param windowDuration + * A string specifying the width of the window, e.g. `10 minutes`, `1 second`. Check + * `org.apache.spark.unsafe.types.CalendarInterval` for valid duration identifiers. Note that + * the duration is a fixed length of time, and does not vary over time according to a + * calendar. For example, `1 day` always means 86,400,000 milliseconds, not a calendar day. + * @param slideDuration + * A string specifying the sliding interval of the window, e.g. `1 minute`. A new window will + * be generated every `slideDuration`. Must be less than or equal to the `windowDuration`. + * Check `org.apache.spark.unsafe.types.CalendarInterval` for valid duration identifiers. This + * duration is likewise absolute, and does not vary according to a calendar. * * @group datetime_funcs * @since 2.0.0 @@ -5589,11 +5743,11 @@ object functions { } /** - * Generates tumbling time windows given a timestamp specifying column. Window - * starts are inclusive but the window ends are exclusive, e.g. 12:05 will be in the window - * [12:05,12:10) but not in [12:00,12:05). Windows can support microsecond precision. Windows in - * the order of months are not supported. The windows start beginning at 1970-01-01 00:00:00 UTC. - * The following example takes the average stock price for a one minute tumbling window: + * Generates tumbling time windows given a timestamp specifying column. Window starts are + * inclusive but the window ends are exclusive, e.g. 12:05 will be in the window [12:05,12:10) + * but not in [12:00,12:05). Windows can support microsecond precision. Windows in the order of + * months are not supported. The windows start beginning at 1970-01-01 00:00:00 UTC. The + * following example takes the average stock price for a one minute tumbling window: * * {{{ * val df = ... // schema => timestamp: TimestampType, stockId: StringType, price: DoubleType @@ -5612,11 +5766,12 @@ object functions { * For a streaming query, you may use the function `current_timestamp` to generate windows on * processing time. * - * @param timeColumn The column or the expression to use as the timestamp for windowing by time. - * The time column must be of TimestampType or TimestampNTZType. - * @param windowDuration A string specifying the width of the window, e.g. `10 minutes`, - * `1 second`. Check `org.apache.spark.unsafe.types.CalendarInterval` for - * valid duration identifiers. + * @param timeColumn + * The column or the expression to use as the timestamp for windowing by time. The time column + * must be of TimestampType or TimestampNTZType. + * @param windowDuration + * A string specifying the width of the window, e.g. `10 minutes`, `1 second`. Check + * `org.apache.spark.unsafe.types.CalendarInterval` for valid duration identifiers. * * @group datetime_funcs * @since 2.0.0 @@ -5632,8 +5787,9 @@ object functions { * inclusive and end is exclusive. Since event time can support microsecond precision, * window_time(window) = window.end - 1 microsecond. * - * @param windowColumn The window column (typically produced by window aggregation) of type - * StructType { start: Timestamp, end: Timestamp } + * @param windowColumn + * The window column (typically produced by window aggregation) of type StructType { start: + * Timestamp, end: Timestamp } * * @group datetime_funcs * @since 3.4.0 @@ -5644,10 +5800,9 @@ object functions { * Generates session window given a timestamp specifying column. * * Session window is one of dynamic windows, which means the length of window is varying - * according to the given inputs. The length of session window is defined as "the timestamp - * of latest input of the session + gap duration", so when the new inputs are bound to the - * current session window, the end time of session window can be expanded according to the new - * inputs. + * according to the given inputs. The length of session window is defined as "the timestamp of + * latest input of the session + gap duration", so when the new inputs are bound to the current + * session window, the end time of session window can be expanded according to the new inputs. * * Windows can support microsecond precision. gapDuration in the order of months are not * supported. @@ -5655,11 +5810,12 @@ object functions { * For a streaming query, you may use the function `current_timestamp` to generate windows on * processing time. * - * @param timeColumn The column or the expression to use as the timestamp for windowing by time. - * The time column must be of TimestampType or TimestampNTZType. - * @param gapDuration A string specifying the timeout of the session, e.g. `10 minutes`, - * `1 second`. Check `org.apache.spark.unsafe.types.CalendarInterval` for - * valid duration identifiers. + * @param timeColumn + * The column or the expression to use as the timestamp for windowing by time. The time column + * must be of TimestampType or TimestampNTZType. + * @param gapDuration + * A string specifying the timeout of the session, e.g. `10 minutes`, `1 second`. Check + * `org.apache.spark.unsafe.types.CalendarInterval` for valid duration identifiers. * * @group datetime_funcs * @since 3.2.0 @@ -5671,17 +5827,17 @@ object functions { * Generates session window given a timestamp specifying column. * * Session window is one of dynamic windows, which means the length of window is varying - * according to the given inputs. For static gap duration, the length of session window - * is defined as "the timestamp of latest input of the session + gap duration", so when - * the new inputs are bound to the current session window, the end time of session window - * can be expanded according to the new inputs. - * - * Besides a static gap duration value, users can also provide an expression to specify - * gap duration dynamically based on the input row. With dynamic gap duration, the closing - * of a session window does not depend on the latest input anymore. A session window's range - * is the union of all events' ranges which are determined by event start time and evaluated - * gap duration during the query execution. Note that the rows with negative or zero gap - * duration will be filtered out from the aggregation. + * according to the given inputs. For static gap duration, the length of session window is + * defined as "the timestamp of latest input of the session + gap duration", so when the new + * inputs are bound to the current session window, the end time of session window can be + * expanded according to the new inputs. + * + * Besides a static gap duration value, users can also provide an expression to specify gap + * duration dynamically based on the input row. With dynamic gap duration, the closing of a + * session window does not depend on the latest input anymore. A session window's range is the + * union of all events' ranges which are determined by event start time and evaluated gap + * duration during the query execution. Note that the rows with negative or zero gap duration + * will be filtered out from the aggregation. * * Windows can support microsecond precision. gapDuration in the order of months are not * supported. @@ -5689,11 +5845,13 @@ object functions { * For a streaming query, you may use the function `current_timestamp` to generate windows on * processing time. * - * @param timeColumn The column or the expression to use as the timestamp for windowing by time. - * The time column must be of TimestampType or TimestampNTZType. - * @param gapDuration A column specifying the timeout of the session. It could be static value, - * e.g. `10 minutes`, `1 second`, or an expression/UDF that specifies gap - * duration dynamically based on the input row. + * @param timeColumn + * The column or the expression to use as the timestamp for windowing by time. The time column + * must be of TimestampType or TimestampNTZType. + * @param gapDuration + * A column specifying the timeout of the session. It could be static value, e.g. `10 + * minutes`, `1 second`, or an expression/UDF that specifies gap duration dynamically based on + * the input row. * * @group datetime_funcs * @since 3.2.0 @@ -5702,8 +5860,7 @@ object functions { Column.fn("session_window", timeColumn, gapDuration) /** - * Converts the number of seconds from the Unix epoch (1970-01-01T00:00:00Z) - * to a timestamp. + * Converts the number of seconds from the Unix epoch (1970-01-01T00:00:00Z) to a timestamp. * @group datetime_funcs * @since 3.1.0 */ @@ -5726,8 +5883,8 @@ object functions { def timestamp_micros(e: Column): Column = Column.fn("timestamp_micros", e) /** - * Gets the difference between the timestamps in the specified units by truncating - * the fraction part. + * Gets the difference between the timestamps in the specified units by truncating the fraction + * part. * * @group datetime_funcs * @since 4.0.0 @@ -5745,8 +5902,8 @@ object functions { Column.internalFn("timestampadd", lit(unit), quantity, ts) /** - * Parses the `timestamp` expression with the `format` expression - * to a timestamp without time zone. Returns null with invalid input. + * Parses the `timestamp` expression with the `format` expression to a timestamp without time + * zone. Returns null with invalid input. * * @group datetime_funcs * @since 3.5.0 @@ -5765,8 +5922,8 @@ object functions { Column.fn("to_timestamp_ltz", timestamp) /** - * Parses the `timestamp_str` expression with the `format` expression - * to a timestamp without time zone. Returns null with invalid input. + * Parses the `timestamp_str` expression with the `format` expression to a timestamp without + * time zone. Returns null with invalid input. * * @group datetime_funcs * @since 3.5.0 @@ -5855,9 +6012,12 @@ object functions { * Returns an array containing all the elements in `x` from index `start` (or starting from the * end if `start` is negative) with the specified `length`. * - * @param x the array column to be sliced - * @param start the starting index - * @param length the length of the slice + * @param x + * the array column to be sliced + * @param start + * the starting index + * @param length + * the length of the slice * * @group array_funcs * @since 2.4.0 @@ -5869,9 +6029,12 @@ object functions { * Returns an array containing all the elements in `x` from index `start` (or starting from the * end if `start` is negative) with the specified `length`. * - * @param x the array column to be sliced - * @param start the starting index - * @param length the length of the slice + * @param x + * the array column to be sliced + * @param start + * the starting index + * @param length + * the length of the slice * * @group array_funcs * @since 3.1.0 @@ -5897,10 +6060,11 @@ object functions { Column.fn("array_join", column, lit(delimiter)) /** - * Concatenates multiple input columns together into a single column. - * The function works with strings, binary and compatible array columns. + * Concatenates multiple input columns together into a single column. The function works with + * strings, binary and compatible array columns. * - * @note Returns null if any of the input columns are null. + * @note + * Returns null if any of the input columns are null. * * @group collection_funcs * @since 1.5.0 @@ -5909,11 +6073,12 @@ object functions { def concat(exprs: Column*): Column = Column.fn("concat", exprs: _*) /** - * Locates the position of the first occurrence of the value in the given array as long. - * Returns null if either of the arguments are null. + * Locates the position of the first occurrence of the value in the given array as long. Returns + * null if either of the arguments are null. * - * @note The position is not zero based, but 1 based index. Returns 0 if value - * could not be found in array. + * @note + * The position is not zero based, but 1 based index. Returns 0 if value could not be found in + * array. * * @group array_funcs * @since 2.4.0 @@ -5922,8 +6087,8 @@ object functions { Column.fn("array_position", column, lit(value)) /** - * Returns element of array at given index in value if column is array. Returns value for - * the given key in value if column is map. + * Returns element of array at given index in value if column is array. Returns value for the + * given key in value if column is map. * * @group collection_funcs * @since 2.4.0 @@ -5945,8 +6110,8 @@ object functions { Column.fn("try_element_at", column, value) /** - * Returns element of array at given (0-based) index. If the index points - * outside of the array boundaries, then this function returns NULL. + * Returns element of array at given (0-based) index. If the index points outside of the array + * boundaries, then this function returns NULL. * * @group array_funcs * @since 3.4.0 @@ -5955,8 +6120,8 @@ object functions { /** * Sorts the input array in ascending order. The elements of the input array must be orderable. - * NaN is greater than any non-NaN elements for double/float type. - * Null elements will be placed at the end of the returned array. + * NaN is greater than any non-NaN elements for double/float type. Null elements will be placed + * at the end of the returned array. * * @group collection_funcs * @since 2.4.0 @@ -6010,8 +6175,8 @@ object functions { def array_distinct(e: Column): Column = Column.fn("array_distinct", e) /** - * Returns an array of the elements in the intersection of the given two arrays, - * without duplicates. + * Returns an array of the elements in the intersection of the given two arrays, without + * duplicates. * * @group array_funcs * @since 2.4.0 @@ -6038,8 +6203,8 @@ object functions { Column.fn("array_union", col1, col2) /** - * Returns an array of the elements in the first array but not in the second array, - * without duplicates. The order of elements in the result is not determined + * Returns an array of the elements in the first array but not in the second array, without + * duplicates. The order of elements in the result is not determined * * @group array_funcs * @since 2.4.0 @@ -6047,7 +6212,6 @@ object functions { def array_except(col1: Column, col2: Column): Column = Column.fn("array_except", col1, col2) - private def createLambda(f: Column => Column) = { val x = internal.UnresolvedNamedLambdaVariable("x") val function = f(Column(x)).node @@ -6070,14 +6234,16 @@ object functions { } /** - * Returns an array of elements after applying a transformation to each element - * in the input array. + * Returns an array of elements after applying a transformation to each element in the input + * array. * {{{ * df.select(transform(col("i"), x => x + 1)) * }}} * - * @param column the input array column - * @param f col => transformed_col, the lambda function to transform the input column + * @param column + * the input array column + * @param f + * col => transformed_col, the lambda function to transform the input column * * @group collection_funcs * @since 3.0.0 @@ -6086,15 +6252,17 @@ object functions { Column.fn("transform", column, createLambda(f)) /** - * Returns an array of elements after applying a transformation to each element - * in the input array. + * Returns an array of elements after applying a transformation to each element in the input + * array. * {{{ * df.select(transform(col("i"), (x, i) => x + i)) * }}} * - * @param column the input array column - * @param f (col, index) => transformed_col, the lambda function to transform the input - * column given the index. Indices start at 0. + * @param column + * the input array column + * @param f + * (col, index) => transformed_col, the lambda function to transform the input column given + * the index. Indices start at 0. * * @group collection_funcs * @since 3.0.0 @@ -6108,8 +6276,10 @@ object functions { * df.select(exists(col("i"), _ % 2 === 0)) * }}} * - * @param column the input array column - * @param f col => predicate, the Boolean predicate to check the input column + * @param column + * the input array column + * @param f + * col => predicate, the Boolean predicate to check the input column * * @group collection_funcs * @since 3.0.0 @@ -6123,8 +6293,10 @@ object functions { * df.select(forall(col("i"), x => x % 2 === 0)) * }}} * - * @param column the input array column - * @param f col => predicate, the Boolean predicate to check the input column + * @param column + * the input array column + * @param f + * col => predicate, the Boolean predicate to check the input column * * @group collection_funcs * @since 3.0.0 @@ -6138,8 +6310,10 @@ object functions { * df.select(filter(col("s"), x => x % 2 === 0)) * }}} * - * @param column the input array column - * @param f col => predicate, the Boolean predicate to filter the input column + * @param column + * the input array column + * @param f + * col => predicate, the Boolean predicate to filter the input column * * @group collection_funcs * @since 3.0.0 @@ -6153,9 +6327,11 @@ object functions { * df.select(filter(col("s"), (x, i) => i % 2 === 0)) * }}} * - * @param column the input array column - * @param f (col, index) => predicate, the Boolean predicate to filter the input column - * given the index. Indices start at 0. + * @param column + * the input array column + * @param f + * (col, index) => predicate, the Boolean predicate to filter the input column given the + * index. Indices start at 0. * * @group collection_funcs * @since 3.0.0 @@ -6164,19 +6340,23 @@ object functions { Column.fn("filter", column, createLambda(f)) /** - * Applies a binary operator to an initial state and all elements in the array, - * and reduces this to a single state. The final state is converted into the final result - * by applying a finish function. + * Applies a binary operator to an initial state and all elements in the array, and reduces this + * to a single state. The final state is converted into the final result by applying a finish + * function. * {{{ * df.select(aggregate(col("i"), lit(0), (acc, x) => acc + x, _ * 10)) * }}} * - * @param expr the input array column - * @param initialValue the initial value - * @param merge (combined_value, input_value) => combined_value, the merge function to merge - * an input value to the combined_value - * @param finish combined_value => final_value, the lambda function to convert the combined value - * of all inputs to final result + * @param expr + * the input array column + * @param initialValue + * the initial value + * @param merge + * (combined_value, input_value) => combined_value, the merge function to merge an input value + * to the combined_value + * @param finish + * combined_value => final_value, the lambda function to convert the combined value of all + * inputs to final result * * @group collection_funcs * @since 3.0.0 @@ -6189,16 +6369,19 @@ object functions { Column.fn("aggregate", expr, initialValue, createLambda(merge), createLambda(finish)) /** - * Applies a binary operator to an initial state and all elements in the array, - * and reduces this to a single state. + * Applies a binary operator to an initial state and all elements in the array, and reduces this + * to a single state. * {{{ * df.select(aggregate(col("i"), lit(0), (acc, x) => acc + x)) * }}} * - * @param expr the input array column - * @param initialValue the initial value - * @param merge (combined_value, input_value) => combined_value, the merge function to merge - * an input value to the combined_value + * @param expr + * the input array column + * @param initialValue + * the initial value + * @param merge + * (combined_value, input_value) => combined_value, the merge function to merge an input value + * to the combined_value * @group collection_funcs * @since 3.0.0 */ @@ -6206,19 +6389,23 @@ object functions { aggregate(expr, initialValue, merge, c => c) /** - * Applies a binary operator to an initial state and all elements in the array, - * and reduces this to a single state. The final state is converted into the final result - * by applying a finish function. + * Applies a binary operator to an initial state and all elements in the array, and reduces this + * to a single state. The final state is converted into the final result by applying a finish + * function. * {{{ * df.select(aggregate(col("i"), lit(0), (acc, x) => acc + x, _ * 10)) * }}} * - * @param expr the input array column - * @param initialValue the initial value - * @param merge (combined_value, input_value) => combined_value, the merge function to merge - * an input value to the combined_value - * @param finish combined_value => final_value, the lambda function to convert the combined value - * of all inputs to final result + * @param expr + * the input array column + * @param initialValue + * the initial value + * @param merge + * (combined_value, input_value) => combined_value, the merge function to merge an input value + * to the combined_value + * @param finish + * combined_value => final_value, the lambda function to convert the combined value of all + * inputs to final result * * @group collection_funcs * @since 3.5.0 @@ -6231,16 +6418,19 @@ object functions { Column.fn("reduce", expr, initialValue, createLambda(merge), createLambda(finish)) /** - * Applies a binary operator to an initial state and all elements in the array, - * and reduces this to a single state. + * Applies a binary operator to an initial state and all elements in the array, and reduces this + * to a single state. * {{{ * df.select(aggregate(col("i"), lit(0), (acc, x) => acc + x)) * }}} * - * @param expr the input array column - * @param initialValue the initial value - * @param merge (combined_value, input_value) => combined_value, the merge function to merge - * an input value to the combined_value + * @param expr + * the input array column + * @param initialValue + * the initial value + * @param merge + * (combined_value, input_value) => combined_value, the merge function to merge an input value + * to the combined_value * @group collection_funcs * @since 3.5.0 */ @@ -6248,16 +6438,19 @@ object functions { reduce(expr, initialValue, merge, c => c) /** - * Merge two given arrays, element-wise, into a single array using a function. - * If one array is shorter, nulls are appended at the end to match the length of the longer - * array, before applying the function. + * Merge two given arrays, element-wise, into a single array using a function. If one array is + * shorter, nulls are appended at the end to match the length of the longer array, before + * applying the function. * {{{ * df.select(zip_with(df1("val1"), df1("val2"), (x, y) => x + y)) * }}} * - * @param left the left input array column - * @param right the right input array column - * @param f (lCol, rCol) => col, the lambda function to merge two input columns into one column + * @param left + * the left input array column + * @param right + * the right input array column + * @param f + * (lCol, rCol) => col, the lambda function to merge two input columns into one column * * @group collection_funcs * @since 3.0.0 @@ -6266,14 +6459,16 @@ object functions { Column.fn("zip_with", left, right, createLambda(f)) /** - * Applies a function to every key-value pair in a map and returns - * a map with the results of those applications as the new keys for the pairs. + * Applies a function to every key-value pair in a map and returns a map with the results of + * those applications as the new keys for the pairs. * {{{ * df.select(transform_keys(col("i"), (k, v) => k + v)) * }}} * - * @param expr the input map column - * @param f (key, value) => new_key, the lambda function to transform the key of input map column + * @param expr + * the input map column + * @param f + * (key, value) => new_key, the lambda function to transform the key of input map column * * @group collection_funcs * @since 3.0.0 @@ -6282,15 +6477,16 @@ object functions { Column.fn("transform_keys", expr, createLambda(f)) /** - * Applies a function to every key-value pair in a map and returns - * a map with the results of those applications as the new values for the pairs. + * Applies a function to every key-value pair in a map and returns a map with the results of + * those applications as the new values for the pairs. * {{{ * df.select(transform_values(col("i"), (k, v) => k + v)) * }}} * - * @param expr the input map column - * @param f (key, value) => new_value, the lambda function to transform the value of input map - * column + * @param expr + * the input map column + * @param f + * (key, value) => new_value, the lambda function to transform the value of input map column * * @group collection_funcs * @since 3.0.0 @@ -6304,8 +6500,10 @@ object functions { * df.select(map_filter(col("m"), (k, v) => k * 10 === v)) * }}} * - * @param expr the input map column - * @param f (key, value) => predicate, the Boolean predicate to filter the input map column + * @param expr + * the input map column + * @param f + * (key, value) => predicate, the Boolean predicate to filter the input map column * * @group collection_funcs * @since 3.0.0 @@ -6319,9 +6517,12 @@ object functions { * df.select(map_zip_with(df("m1"), df("m2"), (k, v1, v2) => k === v1 + v2)) * }}} * - * @param left the left input map column - * @param right the right input map column - * @param f (key, value1, value2) => new_value, the lambda function to merge the map values + * @param left + * the left input map column + * @param right + * the right input map column + * @param f + * (key, value1, value2) => new_value, the lambda function to merge the map values * * @group collection_funcs * @since 3.0.0 @@ -6330,9 +6531,9 @@ object functions { Column.fn("map_zip_with", left, right, createLambda(f)) /** - * Creates a new row for each element in the given array or map column. - * Uses the default column name `col` for elements in the array and - * `key` and `value` for elements in the map unless specified otherwise. + * Creates a new row for each element in the given array or map column. Uses the default column + * name `col` for elements in the array and `key` and `value` for elements in the map unless + * specified otherwise. * * @group generator_funcs * @since 1.3.0 @@ -6340,10 +6541,9 @@ object functions { def explode(e: Column): Column = Column.fn("explode", e) /** - * Creates a new row for each element in the given array or map column. - * Uses the default column name `col` for elements in the array and - * `key` and `value` for elements in the map unless specified otherwise. - * Unlike explode, if the array/map is null or empty then null is produced. + * Creates a new row for each element in the given array or map column. Uses the default column + * name `col` for elements in the array and `key` and `value` for elements in the map unless + * specified otherwise. Unlike explode, if the array/map is null or empty then null is produced. * * @group generator_funcs * @since 2.2.0 @@ -6351,9 +6551,9 @@ object functions { def explode_outer(e: Column): Column = Column.fn("explode_outer", e) /** - * Creates a new row for each element with position in the given array or map column. - * Uses the default column name `pos` for position, and `col` for elements in the array - * and `key` and `value` for elements in the map unless specified otherwise. + * Creates a new row for each element with position in the given array or map column. Uses the + * default column name `pos` for position, and `col` for elements in the array and `key` and + * `value` for elements in the map unless specified otherwise. * * @group generator_funcs * @since 2.1.0 @@ -6361,17 +6561,17 @@ object functions { def posexplode(e: Column): Column = Column.fn("posexplode", e) /** - * Creates a new row for each element with position in the given array or map column. - * Uses the default column name `pos` for position, and `col` for elements in the array - * and `key` and `value` for elements in the map unless specified otherwise. - * Unlike posexplode, if the array/map is null or empty then the row (null, null) is produced. + * Creates a new row for each element with position in the given array or map column. Uses the + * default column name `pos` for position, and `col` for elements in the array and `key` and + * `value` for elements in the map unless specified otherwise. Unlike posexplode, if the + * array/map is null or empty then the row (null, null) is produced. * * @group generator_funcs * @since 2.2.0 */ def posexplode_outer(e: Column): Column = Column.fn("posexplode_outer", e) - /** + /** * Creates a new row for each element in the given array of structs. * * @group generator_funcs @@ -6380,8 +6580,8 @@ object functions { def inline(e: Column): Column = Column.fn("inline", e) /** - * Creates a new row for each element in the given array of structs. - * Unlike inline, if the array is null or empty then null is produced for each nested column. + * Creates a new row for each element in the given array of structs. Unlike inline, if the array + * is null or empty then null is produced for each nested column. * * @group generator_funcs * @since 3.4.0 @@ -6415,14 +6615,15 @@ object functions { * (Scala-specific) Parses a column containing a JSON string into a `StructType` with the * specified schema. Returns `null`, in the case of an unparseable string. * - * @param e a string column containing JSON data. - * @param schema the schema to use when parsing the json string - * @param options options to control how the json is parsed. Accepts the same options as the - * json data source. - * See - * - * Data Source Option in the version you use. + * @param e + * a string column containing JSON data. + * @param schema + * the schema to use when parsing the json string + * @param options + * options to control how the json is parsed. Accepts the same options as the json data + * source. See Data + * Source Option in the version you use. * * @group json_funcs * @since 2.1.0 @@ -6434,17 +6635,18 @@ object functions { // scalastyle:off line.size.limit /** * (Scala-specific) Parses a column containing a JSON string into a `MapType` with `StringType` - * as keys type, `StructType` or `ArrayType` with the specified schema. - * Returns `null`, in the case of an unparseable string. - * - * @param e a string column containing JSON data. - * @param schema the schema to use when parsing the json string - * @param options options to control how the json is parsed. accepts the same options and the - * json data source. - * See - * - * Data Source Option in the version you use. + * as keys type, `StructType` or `ArrayType` with the specified schema. Returns `null`, in the + * case of an unparseable string. + * + * @param e + * a string column containing JSON data. + * @param schema + * the schema to use when parsing the json string + * @param options + * options to control how the json is parsed. accepts the same options and the json data + * source. See Data + * Source Option in the version you use. * * @group json_funcs * @since 2.2.0 @@ -6459,14 +6661,15 @@ object functions { * (Java-specific) Parses a column containing a JSON string into a `StructType` with the * specified schema. Returns `null`, in the case of an unparseable string. * - * @param e a string column containing JSON data. - * @param schema the schema to use when parsing the json string - * @param options options to control how the json is parsed. accepts the same options and the - * json data source. - * See - * - * Data Source Option in the version you use. + * @param e + * a string column containing JSON data. + * @param schema + * the schema to use when parsing the json string + * @param options + * options to control how the json is parsed. accepts the same options and the json data + * source. See Data + * Source Option in the version you use. * * @group json_funcs * @since 2.1.0 @@ -6478,17 +6681,18 @@ object functions { // scalastyle:off line.size.limit /** * (Java-specific) Parses a column containing a JSON string into a `MapType` with `StringType` - * as keys type, `StructType` or `ArrayType` with the specified schema. - * Returns `null`, in the case of an unparseable string. - * - * @param e a string column containing JSON data. - * @param schema the schema to use when parsing the json string - * @param options options to control how the json is parsed. accepts the same options and the - * json data source. - * See - * - * Data Source Option in the version you use. + * as keys type, `StructType` or `ArrayType` with the specified schema. Returns `null`, in the + * case of an unparseable string. + * + * @param e + * a string column containing JSON data. + * @param schema + * the schema to use when parsing the json string + * @param options + * options to control how the json is parsed. accepts the same options and the json data + * source. See Data + * Source Option in the version you use. * * @group json_funcs * @since 2.2.0 @@ -6502,8 +6706,10 @@ object functions { * Parses a column containing a JSON string into a `StructType` with the specified schema. * Returns `null`, in the case of an unparseable string. * - * @param e a string column containing JSON data. - * @param schema the schema to use when parsing the json string + * @param e + * a string column containing JSON data. + * @param schema + * the schema to use when parsing the json string * * @group json_funcs * @since 2.1.0 @@ -6513,11 +6719,13 @@ object functions { /** * Parses a column containing a JSON string into a `MapType` with `StringType` as keys type, - * `StructType` or `ArrayType` with the specified schema. - * Returns `null`, in the case of an unparseable string. + * `StructType` or `ArrayType` with the specified schema. Returns `null`, in the case of an + * unparseable string. * - * @param e a string column containing JSON data. - * @param schema the schema to use when parsing the json string + * @param e + * a string column containing JSON data. + * @param schema + * the schema to use when parsing the json string * * @group json_funcs * @since 2.2.0 @@ -6528,17 +6736,18 @@ object functions { // scalastyle:off line.size.limit /** * (Java-specific) Parses a column containing a JSON string into a `MapType` with `StringType` - * as keys type, `StructType` or `ArrayType` with the specified schema. - * Returns `null`, in the case of an unparseable string. - * - * @param e a string column containing JSON data. - * @param schema the schema as a DDL-formatted string. - * @param options options to control how the json is parsed. accepts the same options and the - * json data source. - * See - * - * Data Source Option in the version you use. + * as keys type, `StructType` or `ArrayType` with the specified schema. Returns `null`, in the + * case of an unparseable string. + * + * @param e + * a string column containing JSON data. + * @param schema + * the schema as a DDL-formatted string. + * @param options + * options to control how the json is parsed. accepts the same options and the json data + * source. See Data + * Source Option in the version you use. * * @group json_funcs * @since 2.1.0 @@ -6551,17 +6760,18 @@ object functions { // scalastyle:off line.size.limit /** * (Scala-specific) Parses a column containing a JSON string into a `MapType` with `StringType` - * as keys type, `StructType` or `ArrayType` with the specified schema. - * Returns `null`, in the case of an unparseable string. - * - * @param e a string column containing JSON data. - * @param schema the schema as a DDL-formatted string. - * @param options options to control how the json is parsed. accepts the same options and the - * json data source. - * See - * - * Data Source Option in the version you use. + * as keys type, `StructType` or `ArrayType` with the specified schema. Returns `null`, in the + * case of an unparseable string. + * + * @param e + * a string column containing JSON data. + * @param schema + * the schema as a DDL-formatted string. + * @param options + * options to control how the json is parsed. accepts the same options and the json data + * source. See Data + * Source Option in the version you use. * * @group json_funcs * @since 2.3.0 @@ -6573,11 +6783,13 @@ object functions { /** * (Scala-specific) Parses a column containing a JSON string into a `MapType` with `StringType` - * as keys type, `StructType` or `ArrayType` of `StructType`s with the specified schema. - * Returns `null`, in the case of an unparseable string. + * as keys type, `StructType` or `ArrayType` of `StructType`s with the specified schema. Returns + * `null`, in the case of an unparseable string. * - * @param e a string column containing JSON data. - * @param schema the schema to use when parsing the json string + * @param e + * a string column containing JSON data. + * @param schema + * the schema to use when parsing the json string * * @group json_funcs * @since 2.4.0 @@ -6589,17 +6801,18 @@ object functions { // scalastyle:off line.size.limit /** * (Java-specific) Parses a column containing a JSON string into a `MapType` with `StringType` - * as keys type, `StructType` or `ArrayType` of `StructType`s with the specified schema. - * Returns `null`, in the case of an unparseable string. - * - * @param e a string column containing JSON data. - * @param schema the schema to use when parsing the json string - * @param options options to control how the json is parsed. accepts the same options and the - * json data source. - * See - * - * Data Source Option in the version you use. + * as keys type, `StructType` or `ArrayType` of `StructType`s with the specified schema. Returns + * `null`, in the case of an unparseable string. + * + * @param e + * a string column containing JSON data. + * @param schema + * the schema to use when parsing the json string + * @param options + * options to control how the json is parsed. accepts the same options and the json data + * source. See Data + * Source Option in the version you use. * * @group json_funcs * @since 2.4.0 @@ -6620,7 +6833,8 @@ object functions { * Parses a JSON string and constructs a Variant value. Returns null if the input string is not * a valid JSON value. * - * @param json a string column that contains JSON data. + * @param json + * a string column that contains JSON data. * * @group variant_funcs * @since 4.0.0 @@ -6705,7 +6919,8 @@ object functions { /** * Parses a JSON string and infers its schema in DDL format. * - * @param json a JSON string. + * @param json + * a JSON string. * * @group json_funcs * @since 2.4.0 @@ -6715,7 +6930,8 @@ object functions { /** * Parses a JSON string and infers its schema in DDL format. * - * @param json a foldable string column containing a JSON string. + * @param json + * a foldable string column containing a JSON string. * * @group json_funcs * @since 2.4.0 @@ -6726,14 +6942,15 @@ object functions { /** * Parses a JSON string and infers its schema in DDL format using options. * - * @param json a foldable string column containing JSON data. - * @param options options to control how the json is parsed. accepts the same options and the - * json data source. - * See - * - * Data Source Option in the version you use. - * @return a column with string literal containing schema in DDL format. + * @param json + * a foldable string column containing JSON data. + * @param options + * options to control how the json is parsed. accepts the same options and the json data + * source. See Data + * Source Option in the version you use. + * @return + * a column with string literal containing schema in DDL format. * * @group json_funcs * @since 3.0.0 @@ -6743,8 +6960,8 @@ object functions { Column.fnWithOptions("schema_of_json", options.asScala.iterator, json) /** - * Returns the number of elements in the outermost JSON array. `NULL` is returned in case of - * any other valid JSON string, `NULL` or an invalid JSON. + * Returns the number of elements in the outermost JSON array. `NULL` is returned in case of any + * other valid JSON string, `NULL` or an invalid JSON. * * @group json_funcs * @since 3.5.0 @@ -6753,8 +6970,8 @@ object functions { /** * Returns all the keys of the outermost JSON object as an array. If a valid JSON object is - * given, all the keys of the outermost object will be returned as an array. If it is any - * other valid JSON string, an invalid JSON string or an empty string, the function returns null. + * given, all the keys of the outermost object will be returned as an array. If it is any other + * valid JSON string, an invalid JSON string or an empty string, the function returns null. * * @group json_funcs * @since 3.5.0 @@ -6763,19 +6980,18 @@ object functions { // scalastyle:off line.size.limit /** - * (Scala-specific) Converts a column containing a `StructType`, `ArrayType` or - * a `MapType` into a JSON string with the specified schema. - * Throws an exception, in the case of an unsupported type. + * (Scala-specific) Converts a column containing a `StructType`, `ArrayType` or a `MapType` into + * a JSON string with the specified schema. Throws an exception, in the case of an unsupported + * type. * - * @param e a column containing a struct, an array or a map. - * @param options options to control how the struct column is converted into a json string. - * accepts the same options and the json data source. - * See - * - * Data Source Option in the version you use. - * Additionally the function supports the `pretty` option which enables - * pretty JSON generation. + * @param e + * a column containing a struct, an array or a map. + * @param options + * options to control how the struct column is converted into a json string. accepts the same + * options and the json data source. See Data + * Source Option in the version you use. Additionally the function supports the `pretty` + * option which enables pretty JSON generation. * * @group json_funcs * @since 2.1.0 @@ -6786,19 +7002,18 @@ object functions { // scalastyle:off line.size.limit /** - * (Java-specific) Converts a column containing a `StructType`, `ArrayType` or - * a `MapType` into a JSON string with the specified schema. - * Throws an exception, in the case of an unsupported type. + * (Java-specific) Converts a column containing a `StructType`, `ArrayType` or a `MapType` into + * a JSON string with the specified schema. Throws an exception, in the case of an unsupported + * type. * - * @param e a column containing a struct, an array or a map. - * @param options options to control how the struct column is converted into a json string. - * accepts the same options and the json data source. - * See - * - * Data Source Option in the version you use. - * Additionally the function supports the `pretty` option which enables - * pretty JSON generation. + * @param e + * a column containing a struct, an array or a map. + * @param options + * options to control how the struct column is converted into a json string. accepts the same + * options and the json data source. See Data + * Source Option in the version you use. Additionally the function supports the `pretty` + * option which enables pretty JSON generation. * * @group json_funcs * @since 2.1.0 @@ -6808,11 +7023,11 @@ object functions { to_json(e, options.asScala.toMap) /** - * Converts a column containing a `StructType`, `ArrayType` or - * a `MapType` into a JSON string with the specified schema. - * Throws an exception, in the case of an unsupported type. + * Converts a column containing a `StructType`, `ArrayType` or a `MapType` into a JSON string + * with the specified schema. Throws an exception, in the case of an unsupported type. * - * @param e a column containing a struct, an array or a map. + * @param e + * a column containing a struct, an array or a map. * * @group json_funcs * @since 2.1.0 @@ -6822,10 +7037,11 @@ object functions { /** * Masks the given string value. The function replaces characters with 'X' or 'x', and numbers - * with 'n'. - * This can be useful for creating copies of tables with sensitive information removed. + * with 'n'. This can be useful for creating copies of tables with sensitive information + * removed. * - * @param input string value to mask. Supported types: STRING, VARCHAR, CHAR + * @param input + * string value to mask. Supported types: STRING, VARCHAR, CHAR * * @group string_funcs * @since 3.5.0 @@ -6834,8 +7050,8 @@ object functions { /** * Masks the given string value. The function replaces upper-case characters with specific - * character, lower-case characters with 'x', and numbers with 'n'. - * This can be useful for creating copies of tables with sensitive information removed. + * character, lower-case characters with 'x', and numbers with 'n'. This can be useful for + * creating copies of tables with sensitive information removed. * * @param input * string value to mask. Supported types: STRING, VARCHAR, CHAR @@ -6850,8 +7066,8 @@ object functions { /** * Masks the given string value. The function replaces upper-case and lower-case characters with - * the characters specified respectively, and numbers with 'n'. - * This can be useful for creating copies of tables with sensitive information removed. + * the characters specified respectively, and numbers with 'n'. This can be useful for creating + * copies of tables with sensitive information removed. * * @param input * string value to mask. Supported types: STRING, VARCHAR, CHAR @@ -6868,8 +7084,8 @@ object functions { /** * Masks the given string value. The function replaces upper-case, lower-case characters and - * numbers with the characters specified respectively. - * This can be useful for creating copies of tables with sensitive information removed. + * numbers with the characters specified respectively. This can be useful for creating copies of + * tables with sensitive information removed. * * @param input * string value to mask. Supported types: STRING, VARCHAR, CHAR @@ -6916,8 +7132,8 @@ object functions { * Returns length of array or map. * * This function returns -1 for null input only if spark.sql.ansi.enabled is false and - * spark.sql.legacy.sizeOfNull is true. Otherwise, it returns null for null input. - * With the default settings, the function returns null for null input. + * spark.sql.legacy.sizeOfNull is true. Otherwise, it returns null for null input. With the + * default settings, the function returns null for null input. * * @group collection_funcs * @since 1.5.0 @@ -6928,8 +7144,8 @@ object functions { * Returns length of array or map. This is an alias of `size` function. * * This function returns -1 for null input only if spark.sql.ansi.enabled is false and - * spark.sql.legacy.sizeOfNull is true. Otherwise, it returns null for null input. - * With the default settings, the function returns null for null input. + * spark.sql.legacy.sizeOfNull is true. Otherwise, it returns null for null input. With the + * default settings, the function returns null for null input. * * @group collection_funcs * @since 3.5.0 @@ -6937,9 +7153,9 @@ object functions { def cardinality(e: Column): Column = Column.fn("cardinality", e) /** - * Sorts the input array for the given column in ascending order, - * according to the natural ordering of the array elements. - * Null elements will be placed at the beginning of the returned array. + * Sorts the input array for the given column in ascending order, according to the natural + * ordering of the array elements. Null elements will be placed at the beginning of the returned + * array. * * @group array_funcs * @since 1.5.0 @@ -6947,11 +7163,10 @@ object functions { def sort_array(e: Column): Column = sort_array(e, asc = true) /** - * Sorts the input array for the given column in ascending or descending order, - * according to the natural ordering of the array elements. NaN is greater than any non-NaN - * elements for double/float type. Null elements will be placed at the beginning of the returned - * array in ascending order or - * at the end of the returned array in descending order. + * Sorts the input array for the given column in ascending or descending order, according to the + * natural ordering of the array elements. NaN is greater than any non-NaN elements for + * double/float type. Null elements will be placed at the beginning of the returned array in + * ascending order or at the end of the returned array in descending order. * * @group array_funcs * @since 1.5.0 @@ -6987,8 +7202,9 @@ object functions { /** * Aggregate function: returns a list of objects with duplicates. * - * @note The function is non-deterministic because the order of collected results depends - * on the order of the rows which may be non-deterministic after a shuffle. + * @note + * The function is non-deterministic because the order of collected results depends on the + * order of the rows which may be non-deterministic after a shuffle. * @group agg_funcs * @since 3.5.0 */ @@ -6997,7 +7213,8 @@ object functions { /** * Returns a random permutation of the given array. * - * @note The function is non-deterministic. + * @note + * The function is non-deterministic. * * @group array_funcs * @since 2.4.0 @@ -7012,8 +7229,8 @@ object functions { def reverse(e: Column): Column = Column.fn("reverse", e) /** - * Creates a single array from an array of arrays. If a structure of nested arrays is deeper than - * two levels, only one level of nesting is removed. + * Creates a single array from an array of arrays. If a structure of nested arrays is deeper + * than two levels, only one level of nesting is removed. * @group array_funcs * @since 2.4.0 */ @@ -7029,8 +7246,8 @@ object functions { Column.fn("sequence", start, stop, step) /** - * Generate a sequence of integers from start to stop, - * incrementing by 1 if start is less than or equal to stop, otherwise -1. + * Generate a sequence of integers from start to stop, incrementing by 1 if start is less than + * or equal to stop, otherwise -1. * * @group array_funcs * @since 2.4.0 @@ -7038,8 +7255,8 @@ object functions { def sequence(start: Column, stop: Column): Column = Column.fn("sequence", start, stop) /** - * Creates an array containing the left argument repeated the number of times given by the - * right argument. + * Creates an array containing the left argument repeated the number of times given by the right + * argument. * * @group array_funcs * @since 2.4.0 @@ -7047,8 +7264,8 @@ object functions { def array_repeat(left: Column, right: Column): Column = Column.fn("array_repeat", left, right) /** - * Creates an array containing the left argument repeated the number of times given by the - * right argument. + * Creates an array containing the left argument repeated the number of times given by the right + * argument. * * @group array_funcs * @since 2.4.0 @@ -7113,14 +7330,15 @@ object functions { * Parses a column containing a CSV string into a `StructType` with the specified schema. * Returns `null`, in the case of an unparseable string. * - * @param e a string column containing CSV data. - * @param schema the schema to use when parsing the CSV string - * @param options options to control how the CSV is parsed. accepts the same options and the - * CSV data source. - * See - * - * Data Source Option in the version you use. + * @param e + * a string column containing CSV data. + * @param schema + * the schema to use when parsing the CSV string + * @param options + * options to control how the CSV is parsed. accepts the same options and the CSV data source. + * See Data + * Source Option in the version you use. * * @group csv_funcs * @since 3.0.0 @@ -7131,17 +7349,18 @@ object functions { // scalastyle:off line.size.limit /** - * (Java-specific) Parses a column containing a CSV string into a `StructType` - * with the specified schema. Returns `null`, in the case of an unparseable string. + * (Java-specific) Parses a column containing a CSV string into a `StructType` with the + * specified schema. Returns `null`, in the case of an unparseable string. * - * @param e a string column containing CSV data. - * @param schema the schema to use when parsing the CSV string - * @param options options to control how the CSV is parsed. accepts the same options and the - * CSV data source. - * See - * - * Data Source Option in the version you use. + * @param e + * a string column containing CSV data. + * @param schema + * the schema to use when parsing the CSV string + * @param options + * options to control how the CSV is parsed. accepts the same options and the CSV data source. + * See Data + * Source Option in the version you use. * * @group csv_funcs * @since 3.0.0 @@ -7156,7 +7375,8 @@ object functions { /** * Parses a CSV string and infers its schema in DDL format. * - * @param csv a CSV string. + * @param csv + * a CSV string. * * @group csv_funcs * @since 3.0.0 @@ -7166,7 +7386,8 @@ object functions { /** * Parses a CSV string and infers its schema in DDL format. * - * @param csv a foldable string column containing a CSV string. + * @param csv + * a foldable string column containing a CSV string. * * @group csv_funcs * @since 3.0.0 @@ -7177,14 +7398,15 @@ object functions { /** * Parses a CSV string and infers its schema in DDL format using options. * - * @param csv a foldable string column containing a CSV string. - * @param options options to control how the CSV is parsed. accepts the same options and the - * CSV data source. - * See - * - * Data Source Option in the version you use. - * @return a column with string literal containing schema in DDL format. + * @param csv + * a foldable string column containing a CSV string. + * @param options + * options to control how the CSV is parsed. accepts the same options and the CSV data source. + * See Data + * Source Option in the version you use. + * @return + * a column with string literal containing schema in DDL format. * * @group csv_funcs * @since 3.0.0 @@ -7195,16 +7417,16 @@ object functions { // scalastyle:off line.size.limit /** - * (Java-specific) Converts a column containing a `StructType` into a CSV string with - * the specified schema. Throws an exception, in the case of an unsupported type. + * (Java-specific) Converts a column containing a `StructType` into a CSV string with the + * specified schema. Throws an exception, in the case of an unsupported type. * - * @param e a column containing a struct. - * @param options options to control how the struct column is converted into a CSV string. - * It accepts the same options and the CSV data source. - * See - * - * Data Source Option in the version you use. + * @param e + * a column containing a struct. + * @param options + * options to control how the struct column is converted into a CSV string. It accepts the + * same options and the CSV data source. See Data + * Source Option in the version you use. * * @group csv_funcs * @since 3.0.0 @@ -7217,7 +7439,8 @@ object functions { * Converts a column containing a `StructType` into a CSV string with the specified schema. * Throws an exception, in the case of an unsupported type. * - * @param e a column containing a struct. + * @param e + * a column containing a struct. * * @group csv_funcs * @since 3.0.0 @@ -7226,17 +7449,18 @@ object functions { // scalastyle:off line.size.limit /** - * Parses a column containing a XML string into the data type corresponding to the specified schema. - * Returns `null`, in the case of an unparseable string. - * - * @param e a string column containing XML data. - * @param schema the schema to use when parsing the XML string - * @param options options to control how the XML is parsed. accepts the same options and the - * XML data source. - * See - * - * Data Source Option in the version you use. + * Parses a column containing a XML string into the data type corresponding to the specified + * schema. Returns `null`, in the case of an unparseable string. + * + * @param e + * a string column containing XML data. + * @param schema + * the schema to use when parsing the XML string + * @param options + * options to control how the XML is parsed. accepts the same options and the XML data source. + * See Data + * Source Option in the version you use. * @group xml_funcs * @since 4.0.0 */ @@ -7246,18 +7470,18 @@ object functions { // scalastyle:off line.size.limit /** - * (Java-specific) Parses a column containing a XML string into a `StructType` - * with the specified schema. - * Returns `null`, in the case of an unparseable string. + * (Java-specific) Parses a column containing a XML string into a `StructType` with the + * specified schema. Returns `null`, in the case of an unparseable string. * - * @param e a string column containing XML data. - * @param schema the schema as a DDL-formatted string. - * @param options options to control how the XML is parsed. accepts the same options and the - * xml data source. - * See - * - * Data Source Option in the version you use. + * @param e + * a string column containing XML data. + * @param schema + * the schema as a DDL-formatted string. + * @param options + * options to control how the XML is parsed. accepts the same options and the xml data source. + * See Data + * Source Option in the version you use. * @group xml_funcs * @since 4.0.0 */ @@ -7268,11 +7492,13 @@ object functions { // scalastyle:off line.size.limit /** - * (Java-specific) Parses a column containing a XML string into a `StructType` - * with the specified schema. Returns `null`, in the case of an unparseable string. + * (Java-specific) Parses a column containing a XML string into a `StructType` with the + * specified schema. Returns `null`, in the case of an unparseable string. * - * @param e a string column containing XML data. - * @param schema the schema to use when parsing the XML string + * @param e + * a string column containing XML data. + * @param schema + * the schema to use when parsing the XML string * @group xml_funcs * @since 4.0.0 */ @@ -7283,17 +7509,18 @@ object functions { // scalastyle:off line.size.limit /** - * (Java-specific) Parses a column containing a XML string into a `StructType` - * with the specified schema. Returns `null`, in the case of an unparseable string. - * - * @param e a string column containing XML data. - * @param schema the schema to use when parsing the XML string - * @param options options to control how the XML is parsed. accepts the same options and the - * XML data source. - * See - * - * Data Source Option in the version you use. + * (Java-specific) Parses a column containing a XML string into a `StructType` with the + * specified schema. Returns `null`, in the case of an unparseable string. + * + * @param e + * a string column containing XML data. + * @param schema + * the schema to use when parsing the XML string + * @param options + * options to control how the XML is parsed. accepts the same options and the XML data source. + * See Data + * Source Option in the version you use. * @group xml_funcs * @since 4.0.0 */ @@ -7302,13 +7529,14 @@ object functions { from_xml(e, schema, options.asScala.iterator) /** - * Parses a column containing a XML string into the data type - * corresponding to the specified schema. - * Returns `null`, in the case of an unparseable string. + * Parses a column containing a XML string into the data type corresponding to the specified + * schema. Returns `null`, in the case of an unparseable string. + * + * @param e + * a string column containing XML data. + * @param schema + * the schema to use when parsing the XML string * - * @param e a string column containing XML data. - * @param schema the schema to use when parsing the XML string - * @group xml_funcs * @since 4.0.0 */ @@ -7322,7 +7550,8 @@ object functions { /** * Parses a XML string and infers its schema in DDL format. * - * @param xml a XML string. + * @param xml + * a XML string. * @group xml_funcs * @since 4.0.0 */ @@ -7331,7 +7560,8 @@ object functions { /** * Parses a XML string and infers its schema in DDL format. * - * @param xml a foldable string column containing a XML string. + * @param xml + * a foldable string column containing a XML string. * @group xml_funcs * @since 4.0.0 */ @@ -7342,14 +7572,15 @@ object functions { /** * Parses a XML string and infers its schema in DDL format using options. * - * @param xml a foldable string column containing XML data. - * @param options options to control how the xml is parsed. accepts the same options and the - * XML data source. - * See - * - * Data Source Option in the version you use. - * @return a column with string literal containing schema in DDL format. + * @param xml + * a foldable string column containing XML data. + * @param options + * options to control how the xml is parsed. accepts the same options and the XML data source. + * See Data + * Source Option in the version you use. + * @return + * a column with string literal containing schema in DDL format. * @group xml_funcs * @since 4.0.0 */ @@ -7360,16 +7591,16 @@ object functions { // scalastyle:off line.size.limit /** - * (Java-specific) Converts a column containing a `StructType` into a XML string with - * the specified schema. Throws an exception, in the case of an unsupported type. + * (Java-specific) Converts a column containing a `StructType` into a XML string with the + * specified schema. Throws an exception, in the case of an unsupported type. * - * @param e a column containing a struct. - * @param options options to control how the struct column is converted into a XML string. - * It accepts the same options as the XML data source. - * See - * - * Data Source Option in the version you use. + * @param e + * a column containing a struct. + * @param options + * options to control how the struct column is converted into a XML string. It accepts the + * same options as the XML data source. See Data + * Source Option in the version you use. * @group xml_funcs * @since 4.0.0 */ @@ -7381,7 +7612,8 @@ object functions { * Converts a column containing a `StructType` into a XML string with the specified schema. * Throws an exception, in the case of an unsupported type. * - * @param e a column containing a struct. + * @param e + * a column containing a struct. * @group xml_funcs * @since 4.0.0 */ @@ -7430,8 +7662,8 @@ object functions { Column.fn("xpath_boolean", xml, path) /** - * Returns a double value, the value zero if no match is found, - * or NaN if a match is found but the value is non-numeric. + * Returns a double value, the value zero if no match is found, or NaN if a match is found but + * the value is non-numeric. * * @group xml_funcs * @since 3.5.0 @@ -7440,8 +7672,8 @@ object functions { Column.fn("xpath_double", xml, path) /** - * Returns a double value, the value zero if no match is found, - * or NaN if a match is found but the value is non-numeric. + * Returns a double value, the value zero if no match is found, or NaN if a match is found but + * the value is non-numeric. * * @group xml_funcs * @since 3.5.0 @@ -7450,8 +7682,8 @@ object functions { Column.fn("xpath_number", xml, path) /** - * Returns a float value, the value zero if no match is found, - * or NaN if a match is found but the value is non-numeric. + * Returns a float value, the value zero if no match is found, or NaN if a match is found but + * the value is non-numeric. * * @group xml_funcs * @since 3.5.0 @@ -7460,8 +7692,8 @@ object functions { Column.fn("xpath_float", xml, path) /** - * Returns an integer value, or the value zero if no match is found, - * or a match is found but the value is non-numeric. + * Returns an integer value, or the value zero if no match is found, or a match is found but the + * value is non-numeric. * * @group xml_funcs * @since 3.5.0 @@ -7470,8 +7702,8 @@ object functions { Column.fn("xpath_int", xml, path) /** - * Returns a long integer value, or the value zero if no match is found, - * or a match is found but the value is non-numeric. + * Returns a long integer value, or the value zero if no match is found, or a match is found but + * the value is non-numeric. * * @group xml_funcs * @since 3.5.0 @@ -7480,8 +7712,8 @@ object functions { Column.fn("xpath_long", xml, path) /** - * Returns a short integer value, or the value zero if no match is found, - * or a match is found but the value is non-numeric. + * Returns a short integer value, or the value zero if no match is found, or a match is found + * but the value is non-numeric. * * @group xml_funcs * @since 3.5.0 @@ -7507,13 +7739,16 @@ object functions { def hours(e: Column): Column = partitioning.hours(e) /** - * Converts the timestamp without time zone `sourceTs` - * from the `sourceTz` time zone to `targetTz`. + * Converts the timestamp without time zone `sourceTs` from the `sourceTz` time zone to + * `targetTz`. * - * @param sourceTz the time zone for the input timestamp. If it is missed, - * the current session time zone is used as the source time zone. - * @param targetTz the time zone to which the input timestamp should be converted. - * @param sourceTs a timestamp without time zone. + * @param sourceTz + * the time zone for the input timestamp. If it is missed, the current session time zone is + * used as the source time zone. + * @param targetTz + * the time zone to which the input timestamp should be converted. + * @param sourceTs + * a timestamp without time zone. * @group datetime_funcs * @since 3.5.0 */ @@ -7521,11 +7756,12 @@ object functions { Column.fn("convert_timezone", sourceTz, targetTz, sourceTs) /** - * Converts the timestamp without time zone `sourceTs` - * from the current time zone to `targetTz`. + * Converts the timestamp without time zone `sourceTs` from the current time zone to `targetTz`. * - * @param targetTz the time zone to which the input timestamp should be converted. - * @param sourceTs a timestamp without time zone. + * @param targetTz + * the time zone to which the input timestamp should be converted. + * @param sourceTs + * a timestamp without time zone. * @group datetime_funcs * @since 3.5.0 */ @@ -7818,8 +8054,8 @@ object functions { def isnotnull(col: Column): Column = Column.fn("isnotnull", col) /** - * Returns same result as the EQUAL(=) operator for non-null operands, - * but returns true if both are null, false if one of the them is null. + * Returns same result as the EQUAL(=) operator for non-null operands, but returns true if both + * are null, false if one of the them is null. * * @group predicate_funcs * @since 3.5.0 @@ -7908,15 +8144,15 @@ object functions { |}""".stripMargin) } - */ + */ ////////////////////////////////////////////////////////////////////////////////////////////// // Scala UDF functions ////////////////////////////////////////////////////////////////////////////////////////////// /** - * Obtains a `UserDefinedFunction` that wraps the given `Aggregator` - * so that it may be used with untyped Data Frames. + * Obtains a `UserDefinedFunction` that wraps the given `Aggregator` so that it may be used with + * untyped Data Frames. * {{{ * val agg = // Aggregator[IN, BUF, OUT] * @@ -7928,24 +8164,30 @@ object functions { * spark.udf.register("myAggName", udaf(agg)) * }}} * - * @tparam IN the aggregator input type - * @tparam BUF the aggregating buffer type - * @tparam OUT the finalized output type + * @tparam IN + * the aggregator input type + * @tparam BUF + * the aggregating buffer type + * @tparam OUT + * the finalized output type * - * @param agg the typed Aggregator + * @param agg + * the typed Aggregator * - * @return a UserDefinedFunction that can be used as an aggregating expression. + * @return + * a UserDefinedFunction that can be used as an aggregating expression. * * @group udf_funcs - * @note The input encoder is inferred from the input type IN. + * @note + * The input encoder is inferred from the input type IN. */ def udaf[IN: TypeTag, BUF, OUT](agg: Aggregator[IN, BUF, OUT]): UserDefinedFunction = { udaf(agg, ScalaReflection.encoderFor[IN]) } /** - * Obtains a `UserDefinedFunction` that wraps the given `Aggregator` - * so that it may be used with untyped Data Frames. + * Obtains a `UserDefinedFunction` that wraps the given `Aggregator` so that it may be used with + * untyped Data Frames. * {{{ * Aggregator agg = // custom Aggregator * Encoder enc = // input encoder @@ -7958,18 +8200,24 @@ object functions { * spark.udf.register("myAggName", udaf(agg, enc)) * }}} * - * @tparam IN the aggregator input type - * @tparam BUF the aggregating buffer type - * @tparam OUT the finalized output type + * @tparam IN + * the aggregator input type + * @tparam BUF + * the aggregating buffer type + * @tparam OUT + * the finalized output type * - * @param agg the typed Aggregator - * @param inputEncoder a specific input encoder to use + * @param agg + * the typed Aggregator + * @param inputEncoder + * a specific input encoder to use * - * @return a UserDefinedFunction that can be used as an aggregating expression + * @return + * a UserDefinedFunction that can be used as an aggregating expression * * @group udf_funcs - * @note This overloading takes an explicit input encoder, to support UDAF - * declarations in Java. + * @note + * This overloading takes an explicit input encoder, to support UDAF declarations in Java. */ def udaf[IN, BUF, OUT]( agg: Aggregator[IN, BUF, OUT], @@ -7978,10 +8226,10 @@ object functions { } /** - * Defines a Scala closure of 0 arguments as user-defined function (UDF). - * The data types are automatically inferred based on the Scala closure's - * signature. By default the returned UDF is deterministic. To change it to - * nondeterministic, call the API `UserDefinedFunction.asNondeterministic()`. + * Defines a Scala closure of 0 arguments as user-defined function (UDF). The data types are + * automatically inferred based on the Scala closure's signature. By default the returned UDF is + * deterministic. To change it to nondeterministic, call the API + * `UserDefinedFunction.asNondeterministic()`. * * @group udf_funcs * @since 1.3.0 @@ -7991,10 +8239,10 @@ object functions { } /** - * Defines a Scala closure of 1 arguments as user-defined function (UDF). - * The data types are automatically inferred based on the Scala closure's - * signature. By default the returned UDF is deterministic. To change it to - * nondeterministic, call the API `UserDefinedFunction.asNondeterministic()`. + * Defines a Scala closure of 1 arguments as user-defined function (UDF). The data types are + * automatically inferred based on the Scala closure's signature. By default the returned UDF is + * deterministic. To change it to nondeterministic, call the API + * `UserDefinedFunction.asNondeterministic()`. * * @group udf_funcs * @since 1.3.0 @@ -8004,120 +8252,242 @@ object functions { } /** - * Defines a Scala closure of 2 arguments as user-defined function (UDF). - * The data types are automatically inferred based on the Scala closure's - * signature. By default the returned UDF is deterministic. To change it to - * nondeterministic, call the API `UserDefinedFunction.asNondeterministic()`. + * Defines a Scala closure of 2 arguments as user-defined function (UDF). The data types are + * automatically inferred based on the Scala closure's signature. By default the returned UDF is + * deterministic. To change it to nondeterministic, call the API + * `UserDefinedFunction.asNondeterministic()`. * * @group udf_funcs * @since 1.3.0 */ - def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag](f: Function2[A1, A2, RT]): UserDefinedFunction = { - SparkUserDefinedFunction(f, implicitly[TypeTag[RT]], implicitly[TypeTag[A1]], implicitly[TypeTag[A2]]) + def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag]( + f: Function2[A1, A2, RT]): UserDefinedFunction = { + SparkUserDefinedFunction( + f, + implicitly[TypeTag[RT]], + implicitly[TypeTag[A1]], + implicitly[TypeTag[A2]]) } /** - * Defines a Scala closure of 3 arguments as user-defined function (UDF). - * The data types are automatically inferred based on the Scala closure's - * signature. By default the returned UDF is deterministic. To change it to - * nondeterministic, call the API `UserDefinedFunction.asNondeterministic()`. + * Defines a Scala closure of 3 arguments as user-defined function (UDF). The data types are + * automatically inferred based on the Scala closure's signature. By default the returned UDF is + * deterministic. To change it to nondeterministic, call the API + * `UserDefinedFunction.asNondeterministic()`. * * @group udf_funcs * @since 1.3.0 */ - def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag](f: Function3[A1, A2, A3, RT]): UserDefinedFunction = { - SparkUserDefinedFunction(f, implicitly[TypeTag[RT]], implicitly[TypeTag[A1]], implicitly[TypeTag[A2]], implicitly[TypeTag[A3]]) + def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag]( + f: Function3[A1, A2, A3, RT]): UserDefinedFunction = { + SparkUserDefinedFunction( + f, + implicitly[TypeTag[RT]], + implicitly[TypeTag[A1]], + implicitly[TypeTag[A2]], + implicitly[TypeTag[A3]]) } /** - * Defines a Scala closure of 4 arguments as user-defined function (UDF). - * The data types are automatically inferred based on the Scala closure's - * signature. By default the returned UDF is deterministic. To change it to - * nondeterministic, call the API `UserDefinedFunction.asNondeterministic()`. + * Defines a Scala closure of 4 arguments as user-defined function (UDF). The data types are + * automatically inferred based on the Scala closure's signature. By default the returned UDF is + * deterministic. To change it to nondeterministic, call the API + * `UserDefinedFunction.asNondeterministic()`. * * @group udf_funcs * @since 1.3.0 */ - def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag](f: Function4[A1, A2, A3, A4, RT]): UserDefinedFunction = { - SparkUserDefinedFunction(f, implicitly[TypeTag[RT]], implicitly[TypeTag[A1]], implicitly[TypeTag[A2]], implicitly[TypeTag[A3]], implicitly[TypeTag[A4]]) + def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag]( + f: Function4[A1, A2, A3, A4, RT]): UserDefinedFunction = { + SparkUserDefinedFunction( + f, + implicitly[TypeTag[RT]], + implicitly[TypeTag[A1]], + implicitly[TypeTag[A2]], + implicitly[TypeTag[A3]], + implicitly[TypeTag[A4]]) } /** - * Defines a Scala closure of 5 arguments as user-defined function (UDF). - * The data types are automatically inferred based on the Scala closure's - * signature. By default the returned UDF is deterministic. To change it to - * nondeterministic, call the API `UserDefinedFunction.asNondeterministic()`. + * Defines a Scala closure of 5 arguments as user-defined function (UDF). The data types are + * automatically inferred based on the Scala closure's signature. By default the returned UDF is + * deterministic. To change it to nondeterministic, call the API + * `UserDefinedFunction.asNondeterministic()`. * * @group udf_funcs * @since 1.3.0 */ - def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag](f: Function5[A1, A2, A3, A4, A5, RT]): UserDefinedFunction = { - SparkUserDefinedFunction(f, implicitly[TypeTag[RT]], implicitly[TypeTag[A1]], implicitly[TypeTag[A2]], implicitly[TypeTag[A3]], implicitly[TypeTag[A4]], implicitly[TypeTag[A5]]) + def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag]( + f: Function5[A1, A2, A3, A4, A5, RT]): UserDefinedFunction = { + SparkUserDefinedFunction( + f, + implicitly[TypeTag[RT]], + implicitly[TypeTag[A1]], + implicitly[TypeTag[A2]], + implicitly[TypeTag[A3]], + implicitly[TypeTag[A4]], + implicitly[TypeTag[A5]]) } /** - * Defines a Scala closure of 6 arguments as user-defined function (UDF). - * The data types are automatically inferred based on the Scala closure's - * signature. By default the returned UDF is deterministic. To change it to - * nondeterministic, call the API `UserDefinedFunction.asNondeterministic()`. + * Defines a Scala closure of 6 arguments as user-defined function (UDF). The data types are + * automatically inferred based on the Scala closure's signature. By default the returned UDF is + * deterministic. To change it to nondeterministic, call the API + * `UserDefinedFunction.asNondeterministic()`. * * @group udf_funcs * @since 1.3.0 */ - def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag](f: Function6[A1, A2, A3, A4, A5, A6, RT]): UserDefinedFunction = { - SparkUserDefinedFunction(f, implicitly[TypeTag[RT]], implicitly[TypeTag[A1]], implicitly[TypeTag[A2]], implicitly[TypeTag[A3]], implicitly[TypeTag[A4]], implicitly[TypeTag[A5]], implicitly[TypeTag[A6]]) + def udf[ + RT: TypeTag, + A1: TypeTag, + A2: TypeTag, + A3: TypeTag, + A4: TypeTag, + A5: TypeTag, + A6: TypeTag](f: Function6[A1, A2, A3, A4, A5, A6, RT]): UserDefinedFunction = { + SparkUserDefinedFunction( + f, + implicitly[TypeTag[RT]], + implicitly[TypeTag[A1]], + implicitly[TypeTag[A2]], + implicitly[TypeTag[A3]], + implicitly[TypeTag[A4]], + implicitly[TypeTag[A5]], + implicitly[TypeTag[A6]]) } /** - * Defines a Scala closure of 7 arguments as user-defined function (UDF). - * The data types are automatically inferred based on the Scala closure's - * signature. By default the returned UDF is deterministic. To change it to - * nondeterministic, call the API `UserDefinedFunction.asNondeterministic()`. + * Defines a Scala closure of 7 arguments as user-defined function (UDF). The data types are + * automatically inferred based on the Scala closure's signature. By default the returned UDF is + * deterministic. To change it to nondeterministic, call the API + * `UserDefinedFunction.asNondeterministic()`. * * @group udf_funcs * @since 1.3.0 */ - def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag](f: Function7[A1, A2, A3, A4, A5, A6, A7, RT]): UserDefinedFunction = { - SparkUserDefinedFunction(f, implicitly[TypeTag[RT]], implicitly[TypeTag[A1]], implicitly[TypeTag[A2]], implicitly[TypeTag[A3]], implicitly[TypeTag[A4]], implicitly[TypeTag[A5]], implicitly[TypeTag[A6]], implicitly[TypeTag[A7]]) + def udf[ + RT: TypeTag, + A1: TypeTag, + A2: TypeTag, + A3: TypeTag, + A4: TypeTag, + A5: TypeTag, + A6: TypeTag, + A7: TypeTag](f: Function7[A1, A2, A3, A4, A5, A6, A7, RT]): UserDefinedFunction = { + SparkUserDefinedFunction( + f, + implicitly[TypeTag[RT]], + implicitly[TypeTag[A1]], + implicitly[TypeTag[A2]], + implicitly[TypeTag[A3]], + implicitly[TypeTag[A4]], + implicitly[TypeTag[A5]], + implicitly[TypeTag[A6]], + implicitly[TypeTag[A7]]) } /** - * Defines a Scala closure of 8 arguments as user-defined function (UDF). - * The data types are automatically inferred based on the Scala closure's - * signature. By default the returned UDF is deterministic. To change it to - * nondeterministic, call the API `UserDefinedFunction.asNondeterministic()`. + * Defines a Scala closure of 8 arguments as user-defined function (UDF). The data types are + * automatically inferred based on the Scala closure's signature. By default the returned UDF is + * deterministic. To change it to nondeterministic, call the API + * `UserDefinedFunction.asNondeterministic()`. * * @group udf_funcs * @since 1.3.0 */ - def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag](f: Function8[A1, A2, A3, A4, A5, A6, A7, A8, RT]): UserDefinedFunction = { - SparkUserDefinedFunction(f, implicitly[TypeTag[RT]], implicitly[TypeTag[A1]], implicitly[TypeTag[A2]], implicitly[TypeTag[A3]], implicitly[TypeTag[A4]], implicitly[TypeTag[A5]], implicitly[TypeTag[A6]], implicitly[TypeTag[A7]], implicitly[TypeTag[A8]]) + def udf[ + RT: TypeTag, + A1: TypeTag, + A2: TypeTag, + A3: TypeTag, + A4: TypeTag, + A5: TypeTag, + A6: TypeTag, + A7: TypeTag, + A8: TypeTag](f: Function8[A1, A2, A3, A4, A5, A6, A7, A8, RT]): UserDefinedFunction = { + SparkUserDefinedFunction( + f, + implicitly[TypeTag[RT]], + implicitly[TypeTag[A1]], + implicitly[TypeTag[A2]], + implicitly[TypeTag[A3]], + implicitly[TypeTag[A4]], + implicitly[TypeTag[A5]], + implicitly[TypeTag[A6]], + implicitly[TypeTag[A7]], + implicitly[TypeTag[A8]]) } /** - * Defines a Scala closure of 9 arguments as user-defined function (UDF). - * The data types are automatically inferred based on the Scala closure's - * signature. By default the returned UDF is deterministic. To change it to - * nondeterministic, call the API `UserDefinedFunction.asNondeterministic()`. + * Defines a Scala closure of 9 arguments as user-defined function (UDF). The data types are + * automatically inferred based on the Scala closure's signature. By default the returned UDF is + * deterministic. To change it to nondeterministic, call the API + * `UserDefinedFunction.asNondeterministic()`. * * @group udf_funcs * @since 1.3.0 */ - def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag](f: Function9[A1, A2, A3, A4, A5, A6, A7, A8, A9, RT]): UserDefinedFunction = { - SparkUserDefinedFunction(f, implicitly[TypeTag[RT]], implicitly[TypeTag[A1]], implicitly[TypeTag[A2]], implicitly[TypeTag[A3]], implicitly[TypeTag[A4]], implicitly[TypeTag[A5]], implicitly[TypeTag[A6]], implicitly[TypeTag[A7]], implicitly[TypeTag[A8]], implicitly[TypeTag[A9]]) + def udf[ + RT: TypeTag, + A1: TypeTag, + A2: TypeTag, + A3: TypeTag, + A4: TypeTag, + A5: TypeTag, + A6: TypeTag, + A7: TypeTag, + A8: TypeTag, + A9: TypeTag](f: Function9[A1, A2, A3, A4, A5, A6, A7, A8, A9, RT]): UserDefinedFunction = { + SparkUserDefinedFunction( + f, + implicitly[TypeTag[RT]], + implicitly[TypeTag[A1]], + implicitly[TypeTag[A2]], + implicitly[TypeTag[A3]], + implicitly[TypeTag[A4]], + implicitly[TypeTag[A5]], + implicitly[TypeTag[A6]], + implicitly[TypeTag[A7]], + implicitly[TypeTag[A8]], + implicitly[TypeTag[A9]]) } /** - * Defines a Scala closure of 10 arguments as user-defined function (UDF). - * The data types are automatically inferred based on the Scala closure's - * signature. By default the returned UDF is deterministic. To change it to - * nondeterministic, call the API `UserDefinedFunction.asNondeterministic()`. + * Defines a Scala closure of 10 arguments as user-defined function (UDF). The data types are + * automatically inferred based on the Scala closure's signature. By default the returned UDF is + * deterministic. To change it to nondeterministic, call the API + * `UserDefinedFunction.asNondeterministic()`. * * @group udf_funcs * @since 1.3.0 */ - def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag](f: Function10[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, RT]): UserDefinedFunction = { - SparkUserDefinedFunction(f, implicitly[TypeTag[RT]], implicitly[TypeTag[A1]], implicitly[TypeTag[A2]], implicitly[TypeTag[A3]], implicitly[TypeTag[A4]], implicitly[TypeTag[A5]], implicitly[TypeTag[A6]], implicitly[TypeTag[A7]], implicitly[TypeTag[A8]], implicitly[TypeTag[A9]], implicitly[TypeTag[A10]]) + def udf[ + RT: TypeTag, + A1: TypeTag, + A2: TypeTag, + A3: TypeTag, + A4: TypeTag, + A5: TypeTag, + A6: TypeTag, + A7: TypeTag, + A8: TypeTag, + A9: TypeTag, + A10: TypeTag]( + f: Function10[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, RT]): UserDefinedFunction = { + SparkUserDefinedFunction( + f, + implicitly[TypeTag[RT]], + implicitly[TypeTag[A1]], + implicitly[TypeTag[A2]], + implicitly[TypeTag[A3]], + implicitly[TypeTag[A4]], + implicitly[TypeTag[A5]], + implicitly[TypeTag[A6]], + implicitly[TypeTag[A7]], + implicitly[TypeTag[A8]], + implicitly[TypeTag[A9]], + implicitly[TypeTag[A10]]) } ////////////////////////////////////////////////////////////////////////////////////////////// @@ -8125,10 +8495,10 @@ object functions { ////////////////////////////////////////////////////////////////////////////////////////////// /** - * Defines a Java UDF0 instance as user-defined function (UDF). - * The caller must specify the output data type, and there is no automatic input type coercion. - * By default the returned UDF is deterministic. To change it to nondeterministic, call the - * API `UserDefinedFunction.asNondeterministic()`. + * Defines a Java UDF0 instance as user-defined function (UDF). The caller must specify the + * output data type, and there is no automatic input type coercion. By default the returned UDF + * is deterministic. To change it to nondeterministic, call the API + * `UserDefinedFunction.asNondeterministic()`. * * @group udf_funcs * @since 2.3.0 @@ -8138,10 +8508,10 @@ object functions { } /** - * Defines a Java UDF1 instance as user-defined function (UDF). - * The caller must specify the output data type, and there is no automatic input type coercion. - * By default the returned UDF is deterministic. To change it to nondeterministic, call the - * API `UserDefinedFunction.asNondeterministic()`. + * Defines a Java UDF1 instance as user-defined function (UDF). The caller must specify the + * output data type, and there is no automatic input type coercion. By default the returned UDF + * is deterministic. To change it to nondeterministic, call the API + * `UserDefinedFunction.asNondeterministic()`. * * @group udf_funcs * @since 2.3.0 @@ -8151,10 +8521,10 @@ object functions { } /** - * Defines a Java UDF2 instance as user-defined function (UDF). - * The caller must specify the output data type, and there is no automatic input type coercion. - * By default the returned UDF is deterministic. To change it to nondeterministic, call the - * API `UserDefinedFunction.asNondeterministic()`. + * Defines a Java UDF2 instance as user-defined function (UDF). The caller must specify the + * output data type, and there is no automatic input type coercion. By default the returned UDF + * is deterministic. To change it to nondeterministic, call the API + * `UserDefinedFunction.asNondeterministic()`. * * @group udf_funcs * @since 2.3.0 @@ -8164,10 +8534,10 @@ object functions { } /** - * Defines a Java UDF3 instance as user-defined function (UDF). - * The caller must specify the output data type, and there is no automatic input type coercion. - * By default the returned UDF is deterministic. To change it to nondeterministic, call the - * API `UserDefinedFunction.asNondeterministic()`. + * Defines a Java UDF3 instance as user-defined function (UDF). The caller must specify the + * output data type, and there is no automatic input type coercion. By default the returned UDF + * is deterministic. To change it to nondeterministic, call the API + * `UserDefinedFunction.asNondeterministic()`. * * @group udf_funcs * @since 2.3.0 @@ -8177,10 +8547,10 @@ object functions { } /** - * Defines a Java UDF4 instance as user-defined function (UDF). - * The caller must specify the output data type, and there is no automatic input type coercion. - * By default the returned UDF is deterministic. To change it to nondeterministic, call the - * API `UserDefinedFunction.asNondeterministic()`. + * Defines a Java UDF4 instance as user-defined function (UDF). The caller must specify the + * output data type, and there is no automatic input type coercion. By default the returned UDF + * is deterministic. To change it to nondeterministic, call the API + * `UserDefinedFunction.asNondeterministic()`. * * @group udf_funcs * @since 2.3.0 @@ -8190,10 +8560,10 @@ object functions { } /** - * Defines a Java UDF5 instance as user-defined function (UDF). - * The caller must specify the output data type, and there is no automatic input type coercion. - * By default the returned UDF is deterministic. To change it to nondeterministic, call the - * API `UserDefinedFunction.asNondeterministic()`. + * Defines a Java UDF5 instance as user-defined function (UDF). The caller must specify the + * output data type, and there is no automatic input type coercion. By default the returned UDF + * is deterministic. To change it to nondeterministic, call the API + * `UserDefinedFunction.asNondeterministic()`. * * @group udf_funcs * @since 2.3.0 @@ -8203,10 +8573,10 @@ object functions { } /** - * Defines a Java UDF6 instance as user-defined function (UDF). - * The caller must specify the output data type, and there is no automatic input type coercion. - * By default the returned UDF is deterministic. To change it to nondeterministic, call the - * API `UserDefinedFunction.asNondeterministic()`. + * Defines a Java UDF6 instance as user-defined function (UDF). The caller must specify the + * output data type, and there is no automatic input type coercion. By default the returned UDF + * is deterministic. To change it to nondeterministic, call the API + * `UserDefinedFunction.asNondeterministic()`. * * @group udf_funcs * @since 2.3.0 @@ -8216,10 +8586,10 @@ object functions { } /** - * Defines a Java UDF7 instance as user-defined function (UDF). - * The caller must specify the output data type, and there is no automatic input type coercion. - * By default the returned UDF is deterministic. To change it to nondeterministic, call the - * API `UserDefinedFunction.asNondeterministic()`. + * Defines a Java UDF7 instance as user-defined function (UDF). The caller must specify the + * output data type, and there is no automatic input type coercion. By default the returned UDF + * is deterministic. To change it to nondeterministic, call the API + * `UserDefinedFunction.asNondeterministic()`. * * @group udf_funcs * @since 2.3.0 @@ -8229,10 +8599,10 @@ object functions { } /** - * Defines a Java UDF8 instance as user-defined function (UDF). - * The caller must specify the output data type, and there is no automatic input type coercion. - * By default the returned UDF is deterministic. To change it to nondeterministic, call the - * API `UserDefinedFunction.asNondeterministic()`. + * Defines a Java UDF8 instance as user-defined function (UDF). The caller must specify the + * output data type, and there is no automatic input type coercion. By default the returned UDF + * is deterministic. To change it to nondeterministic, call the API + * `UserDefinedFunction.asNondeterministic()`. * * @group udf_funcs * @since 2.3.0 @@ -8242,10 +8612,10 @@ object functions { } /** - * Defines a Java UDF9 instance as user-defined function (UDF). - * The caller must specify the output data type, and there is no automatic input type coercion. - * By default the returned UDF is deterministic. To change it to nondeterministic, call the - * API `UserDefinedFunction.asNondeterministic()`. + * Defines a Java UDF9 instance as user-defined function (UDF). The caller must specify the + * output data type, and there is no automatic input type coercion. By default the returned UDF + * is deterministic. To change it to nondeterministic, call the API + * `UserDefinedFunction.asNondeterministic()`. * * @group udf_funcs * @since 2.3.0 @@ -8255,15 +8625,17 @@ object functions { } /** - * Defines a Java UDF10 instance as user-defined function (UDF). - * The caller must specify the output data type, and there is no automatic input type coercion. - * By default the returned UDF is deterministic. To change it to nondeterministic, call the - * API `UserDefinedFunction.asNondeterministic()`. + * Defines a Java UDF10 instance as user-defined function (UDF). The caller must specify the + * output data type, and there is no automatic input type coercion. By default the returned UDF + * is deterministic. To change it to nondeterministic, call the API + * `UserDefinedFunction.asNondeterministic()`. * * @group udf_funcs * @since 2.3.0 */ - def udf(f: UDF10[_, _, _, _, _, _, _, _, _, _, _], returnType: DataType): UserDefinedFunction = { + def udf( + f: UDF10[_, _, _, _, _, _, _, _, _, _, _], + returnType: DataType): UserDefinedFunction = { SparkUserDefinedFunction(ToScalaUDF(f), returnType, 10) } @@ -8273,8 +8645,8 @@ object functions { /** * Defines a deterministic user-defined function (UDF) using a Scala closure. For this variant, * the caller must specify the output data type, and there is no automatic input type coercion. - * By default the returned UDF is deterministic. To change it to nondeterministic, call the - * API `UserDefinedFunction.asNondeterministic()`. + * By default the returned UDF is deterministic. To change it to nondeterministic, call the API + * `UserDefinedFunction.asNondeterministic()`. * * Note that, although the Scala closure can have primitive-type function argument, it doesn't * work well with null values. Because the Scala closure is passed in as Any type, there is no @@ -8283,14 +8655,18 @@ object functions { * default value of the Java type for the null argument, e.g. `udf((x: Int) => x, IntegerType)`, * the result is 0 for null input. * - * @param f A closure in Scala - * @param dataType The output data type of the UDF + * @param f + * A closure in Scala + * @param dataType + * The output data type of the UDF * * @group udf_funcs * @since 2.0.0 */ - @deprecated("Scala `udf` method with return type parameter is deprecated. " + - "Please use Scala `udf` method without return type parameter.", "3.0.0") + @deprecated( + "Scala `udf` method with return type parameter is deprecated. " + + "Please use Scala `udf` method without return type parameter.", + "3.0.0") def udf(f: AnyRef, dataType: DataType): UserDefinedFunction = { if (!SqlApiConf.get.legacyAllowUntypedScalaUDFs) { throw CompilationErrors.usingUntypedScalaUDFError() @@ -8309,8 +8685,7 @@ object functions { def callUDF(udfName: String, cols: Column*): Column = call_function(udfName, cols: _*) /** - * Call an user-defined function. - * Example: + * Call an user-defined function. Example: * {{{ * import org.apache.spark.sql._ * @@ -8329,9 +8704,10 @@ object functions { /** * Call a SQL function. * - * @param funcName function name that follows the SQL identifier syntax - * (can be quoted, can be qualified) - * @param cols the expression parameters of function + * @param funcName + * function name that follows the SQL identifier syntax (can be quoted, can be qualified) + * @param cols + * the expression parameters of function * @group normal_funcs * @since 3.5.0 */ @@ -8352,7 +8728,7 @@ object functions { // API in the same way. Once we land this fix, should deprecate // functions.hours, days, months, years and bucket. object partitioning { - // scalastyle:on + // scalastyle:on /** * (Scala-specific) A transform for timestamps and dates to partition data into years. * diff --git a/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala b/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala index 5762f9f6f5668..555a567053080 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala @@ -25,8 +25,8 @@ import org.apache.spark.util.SparkClassUtils /** * Configuration for all objects that are placed in the `sql/api` project. The normal way of - * accessing this class is through `SqlApiConf.get`. If this code is being used with sql/core - * then its values are bound to the currently set SQLConf. With Spark Connect, it will default to + * accessing this class is through `SqlApiConf.get`. If this code is being used with sql/core then + * its values are bound to the currently set SQLConf. With Spark Connect, it will default to * hardcoded values. */ private[sql] trait SqlApiConf { diff --git a/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConfHelper.scala b/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConfHelper.scala index b7b8e14afb387..13ef13e5894e0 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConfHelper.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConfHelper.scala @@ -21,9 +21,9 @@ import java.util.concurrent.atomic.AtomicReference /** * SqlApiConfHelper is created to avoid a deadlock during a concurrent access to SQLConf and * SqlApiConf, which is because SQLConf and SqlApiConf tries to load each other upon - * initializations. SqlApiConfHelper is private to sql package and is not supposed to be - * accessed by end users. Variables and methods within SqlApiConfHelper are defined to - * be used by SQLConf and SqlApiConf only. + * initializations. SqlApiConfHelper is private to sql package and is not supposed to be accessed + * by end users. Variables and methods within SqlApiConfHelper are defined to be used by SQLConf + * and SqlApiConf only. */ private[sql] object SqlApiConfHelper { // Shared keys. diff --git a/sql/api/src/main/scala/org/apache/spark/sql/internal/ToScalaUDF.scala b/sql/api/src/main/scala/org/apache/spark/sql/internal/ToScalaUDF.scala index 66ea50c57bcf7..4d476108d9ec5 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/internal/ToScalaUDF.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/internal/ToScalaUDF.scala @@ -27,9 +27,10 @@ import org.apache.spark.sql.streaming.GroupState * org.apache.spark.api.java.function.* to scala functions. * * Please note that this class is being used in Spark Connect Scala UDFs. We need to be careful - * with any modifications to this class, otherwise we will break backwards compatibility. Concretely - * this means you can only add methods to this class. You cannot rename the class, move it, change - * its `serialVersionUID`, remove methods, change method signatures, or change method semantics. + * with any modifications to this class, otherwise we will break backwards compatibility. + * Concretely this means you can only add methods to this class. You cannot rename the class, move + * it, change its `serialVersionUID`, remove methods, change method signatures, or change method + * semantics. */ @SerialVersionUID(2019907615267866045L) private[sql] object ToScalaUDF extends Serializable { @@ -42,8 +43,8 @@ private[sql] object ToScalaUDF extends Serializable { def apply[K, V, U](f: MapGroupsFunction[K, V, U]): (K, Iterator[V]) => U = (key, values) => f.call(key, values.asJava) - def apply[K, V, S, U](f: MapGroupsWithStateFunction[K, V, S, U]) - : (K, Iterator[V], GroupState[S]) => U = + def apply[K, V, S, U]( + f: MapGroupsWithStateFunction[K, V, S, U]): (K, Iterator[V], GroupState[S]) => U = (key, values, state) => f.call(key, values.asJava, state) def apply[V, U](f: MapPartitionsFunction[V, U]): Iterator[V] => Iterator[U] = @@ -53,11 +54,11 @@ private[sql] object ToScalaUDF extends Serializable { (key, values) => f.call(key, values.asJava).asScala def apply[K, V, S, U](f: FlatMapGroupsWithStateFunction[K, V, S, U]) - : (K, Iterator[V], GroupState[S]) => Iterator[U] = + : (K, Iterator[V], GroupState[S]) => Iterator[U] = (key, values, state) => f.call(key, values.asJava, state).asScala - def apply[K, V, U, R](f: CoGroupFunction[K, V, U, R]) - : (K, Iterator[V], Iterator[U]) => Iterator[R] = + def apply[K, V, U, R]( + f: CoGroupFunction[K, V, U, R]): (K, Iterator[V], Iterator[U]) => Iterator[R] = (key, left, right) => f.call(key, left.asJava, right.asJava).asScala def apply[V](f: ForeachFunction[V]): V => Unit = f.call @@ -83,13 +84,13 @@ private[sql] object ToScalaUDF extends Serializable { | $funcCall |}""".stripMargin) } - */ + */ /** * Create a scala.Function0 wrapper for a org.apache.spark.sql.api.java.UDF0 instance. */ - def apply(f: UDF0[_]): () => Any = { - () => f.asInstanceOf[UDF0[Any]].call() + def apply(f: UDF0[_]): () => Any = { () => + f.asInstanceOf[UDF0[Any]].call() } /** @@ -124,126 +125,633 @@ private[sql] object ToScalaUDF extends Serializable { * Create a scala.Function5 wrapper for a org.apache.spark.sql.api.java.UDF5 instance. */ def apply(f: UDF5[_, _, _, _, _, _]): (Any, Any, Any, Any, Any) => Any = { - f.asInstanceOf[UDF5[Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any) + f.asInstanceOf[UDF5[Any, Any, Any, Any, Any, Any]] + .call(_: Any, _: Any, _: Any, _: Any, _: Any) } /** * Create a scala.Function6 wrapper for a org.apache.spark.sql.api.java.UDF6 instance. */ def apply(f: UDF6[_, _, _, _, _, _, _]): (Any, Any, Any, Any, Any, Any) => Any = { - f.asInstanceOf[UDF6[Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any) + f.asInstanceOf[UDF6[Any, Any, Any, Any, Any, Any, Any]] + .call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any) } /** * Create a scala.Function7 wrapper for a org.apache.spark.sql.api.java.UDF7 instance. */ def apply(f: UDF7[_, _, _, _, _, _, _, _]): (Any, Any, Any, Any, Any, Any, Any) => Any = { - f.asInstanceOf[UDF7[Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) + f.asInstanceOf[UDF7[Any, Any, Any, Any, Any, Any, Any, Any]] + .call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) } /** * Create a scala.Function8 wrapper for a org.apache.spark.sql.api.java.UDF8 instance. */ - def apply(f: UDF8[_, _, _, _, _, _, _, _, _]): (Any, Any, Any, Any, Any, Any, Any, Any) => Any = { - f.asInstanceOf[UDF8[Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) + def apply( + f: UDF8[_, _, _, _, _, _, _, _, _]): (Any, Any, Any, Any, Any, Any, Any, Any) => Any = { + f.asInstanceOf[UDF8[Any, Any, Any, Any, Any, Any, Any, Any, Any]] + .call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) } /** * Create a scala.Function9 wrapper for a org.apache.spark.sql.api.java.UDF9 instance. */ - def apply(f: UDF9[_, _, _, _, _, _, _, _, _, _]): (Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any = { - f.asInstanceOf[UDF9[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) + def apply(f: UDF9[_, _, _, _, _, _, _, _, _, _]) + : (Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any = { + f.asInstanceOf[UDF9[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]] + .call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) } /** * Create a scala.Function10 wrapper for a org.apache.spark.sql.api.java.UDF10 instance. */ - def apply(f: UDF10[_, _, _, _, _, _, _, _, _, _, _]): Function10[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any] = { - f.asInstanceOf[UDF10[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) + def apply(f: UDF10[_, _, _, _, _, _, _, _, _, _, _]) + : Function10[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any] = { + f.asInstanceOf[UDF10[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]] + .call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) } /** * Create a scala.Function11 wrapper for a org.apache.spark.sql.api.java.UDF11 instance. */ - def apply(f: UDF11[_, _, _, _, _, _, _, _, _, _, _, _]): Function11[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any] = { - f.asInstanceOf[UDF11[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) + def apply(f: UDF11[_, _, _, _, _, _, _, _, _, _, _, _]) + : Function11[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any] = { + f.asInstanceOf[UDF11[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]] + .call( + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any) } /** * Create a scala.Function12 wrapper for a org.apache.spark.sql.api.java.UDF12 instance. */ - def apply(f: UDF12[_, _, _, _, _, _, _, _, _, _, _, _, _]): Function12[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any] = { - f.asInstanceOf[UDF12[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) + def apply(f: UDF12[_, _, _, _, _, _, _, _, _, _, _, _, _]) + : Function12[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any] = { + f.asInstanceOf[UDF12[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]] + .call( + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any) } /** * Create a scala.Function13 wrapper for a org.apache.spark.sql.api.java.UDF13 instance. */ - def apply(f: UDF13[_, _, _, _, _, _, _, _, _, _, _, _, _, _]): Function13[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any] = { - f.asInstanceOf[UDF13[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) + def apply(f: UDF13[_, _, _, _, _, _, _, _, _, _, _, _, _, _]) + : Function13[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any] = { + f.asInstanceOf[UDF13[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]] + .call( + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any) } /** * Create a scala.Function14 wrapper for a org.apache.spark.sql.api.java.UDF14 instance. */ - def apply(f: UDF14[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _]): Function14[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any] = { - f.asInstanceOf[UDF14[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) + def apply(f: UDF14[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _]) + : Function14[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any] = { + f.asInstanceOf[ + UDF14[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]] + .call( + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any) } /** * Create a scala.Function15 wrapper for a org.apache.spark.sql.api.java.UDF15 instance. */ - def apply(f: UDF15[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]): Function15[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any] = { - f.asInstanceOf[UDF15[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) + def apply(f: UDF15[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]): Function15[ + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any] = { + f.asInstanceOf[ + UDF15[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]] + .call( + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any) } /** * Create a scala.Function16 wrapper for a org.apache.spark.sql.api.java.UDF16 instance. */ - def apply(f: UDF16[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]): Function16[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any] = { - f.asInstanceOf[UDF16[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) + def apply(f: UDF16[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]): Function16[ + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any] = { + f.asInstanceOf[ + UDF16[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]] + .call( + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any) } /** * Create a scala.Function17 wrapper for a org.apache.spark.sql.api.java.UDF17 instance. */ - def apply(f: UDF17[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]): Function17[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any] = { - f.asInstanceOf[UDF17[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) + def apply(f: UDF17[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]): Function17[ + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any] = { + f.asInstanceOf[UDF17[ + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any]] + .call( + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any) } /** * Create a scala.Function18 wrapper for a org.apache.spark.sql.api.java.UDF18 instance. */ - def apply(f: UDF18[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]): Function18[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any] = { - f.asInstanceOf[UDF18[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) + def apply(f: UDF18[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]): Function18[ + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any] = { + f.asInstanceOf[UDF18[ + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any]] + .call( + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any) } /** * Create a scala.Function19 wrapper for a org.apache.spark.sql.api.java.UDF19 instance. */ - def apply(f: UDF19[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]): Function19[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any] = { - f.asInstanceOf[UDF19[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) + def apply(f: UDF19[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]): Function19[ + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any] = { + f.asInstanceOf[UDF19[ + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any]] + .call( + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any) } /** * Create a scala.Function20 wrapper for a org.apache.spark.sql.api.java.UDF20 instance. */ - def apply(f: UDF20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]): Function20[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any] = { - f.asInstanceOf[UDF20[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) + def apply(f: UDF20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]): Function20[ + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any] = { + f.asInstanceOf[UDF20[ + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any]] + .call( + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any) } /** * Create a scala.Function21 wrapper for a org.apache.spark.sql.api.java.UDF21 instance. */ - def apply(f: UDF21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]): Function21[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any] = { - f.asInstanceOf[UDF21[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) + def apply( + f: UDF21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]): Function21[ + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any] = { + f.asInstanceOf[UDF21[ + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any]] + .call( + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any) } /** * Create a scala.Function22 wrapper for a org.apache.spark.sql.api.java.UDF22 instance. */ - def apply(f: UDF22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]): Function22[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any] = { - f.asInstanceOf[UDF22[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) + def apply( + f: UDF22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]): Function22[ + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any] = { + f.asInstanceOf[UDF22[ + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any, + Any]] + .call( + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any, + _: Any) } // scalastyle:on line.size.limit } @@ -253,9 +761,10 @@ private[sql] object ToScalaUDF extends Serializable { * foreachPartition. * * Please note that this class is being used in Spark Connect Scala UDFs. We need to be careful - * with any modifications to this class, otherwise we will break backwards compatibility. Concretely - * this means you can only add methods to this class. You cannot rename the class, move it, change - * its `serialVersionUID`, remove methods, change method signatures, or change method semantics. + * with any modifications to this class, otherwise we will break backwards compatibility. + * Concretely this means you can only add methods to this class. You cannot rename the class, move + * it, change its `serialVersionUID`, remove methods, change method signatures, or change method + * semantics. */ @SerialVersionUID(0L) // TODO object UDFAdaptors extends Serializable { @@ -285,7 +794,8 @@ object UDFAdaptors extends Serializable { def iterableOnceToSeq[A, B](f: A => IterableOnce[B]): A => Seq[B] = value => f(value).iterator.toSeq - def mapGroupsToFlatMapGroups[K, V, U](f: (K, Iterator[V]) => U): (K, Iterator[V]) => Iterator[U] = + def mapGroupsToFlatMapGroups[K, V, U]( + f: (K, Iterator[V]) => U): (K, Iterator[V]) => Iterator[U] = (key, values) => Iterator.single(f(key, values)) def mapGroupsWithStateToFlatMapWithState[K, V, S, U]( @@ -309,8 +819,8 @@ object UDFAdaptors extends Serializable { } def flatMapGroupsWithMappedValues[K, IV, V, R]( - f: (K, Iterator[V]) => IterableOnce[R], - valueMapFunc: Option[IV => V]): (K, Iterator[IV]) => IterableOnce[R] = valueMapFunc match { + f: (K, Iterator[V]) => IterableOnce[R], + valueMapFunc: Option[IV => V]): (K, Iterator[IV]) => IterableOnce[R] = valueMapFunc match { case Some(mapValue) => (k, values) => f(k, values.map(mapValue)) case None => f.asInstanceOf[(K, Iterator[IV]) => IterableOnce[R]] } diff --git a/sql/api/src/main/scala/org/apache/spark/sql/internal/columnNodes.scala b/sql/api/src/main/scala/org/apache/spark/sql/internal/columnNodes.scala index 9d77b2e6f3e22..51b26a1fa2435 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/internal/columnNodes.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/internal/columnNodes.scala @@ -32,10 +32,12 @@ import org.apache.spark.util.SparkClassUtils * implementation specific form (e.g. Catalyst expressions, or Connect protobuf messages). * * This API is a mirror image of Connect's expression.proto. There are a couple of extensions to - * make constructing nodes easier (e.g. [[CaseWhenOtherwise]]). We could not use the actual connect - * protobuf messages because of classpath clashes (e.g. Guava & gRPC) and Maven shading issues. + * make constructing nodes easier (e.g. [[CaseWhenOtherwise]]). We could not use the actual + * connect protobuf messages because of classpath clashes (e.g. Guava & gRPC) and Maven shading + * issues. */ private[sql] trait ColumnNode extends ColumnNodeLike { + /** * Origin where the node was created. */ @@ -93,13 +95,17 @@ private[internal] object ColumnNode { /** * A literal column. * - * @param value of the literal. This is the unconverted input value. - * @param dataType of the literal. If none is provided the dataType is inferred. + * @param value + * of the literal. This is the unconverted input value. + * @param dataType + * of the literal. If none is provided the dataType is inferred. */ private[sql] case class Literal( value: Any, dataType: Option[DataType] = None, - override val origin: Origin = CurrentOrigin.get) extends ColumnNode with DataTypeErrorsBase { + override val origin: Origin = CurrentOrigin.get) + extends ColumnNode + with DataTypeErrorsBase { override private[internal] def normalize(): Literal = copy(origin = NO_ORIGIN) override def sql: String = value match { @@ -116,16 +122,19 @@ private[sql] case class Literal( /** * Reference to an attribute produced by one of the underlying DataFrames. * - * @param unparsedIdentifier name of the attribute. - * @param planId id of the plan (Dataframe) that produces the attribute. - * @param isMetadataColumn whether this is a metadata column. + * @param unparsedIdentifier + * name of the attribute. + * @param planId + * id of the plan (Dataframe) that produces the attribute. + * @param isMetadataColumn + * whether this is a metadata column. */ private[sql] case class UnresolvedAttribute( unparsedIdentifier: String, planId: Option[Long] = None, isMetadataColumn: Boolean = false, override val origin: Origin = CurrentOrigin.get) - extends ColumnNode { + extends ColumnNode { override private[internal] def normalize(): UnresolvedAttribute = copy(planId = None, origin = NO_ORIGIN) override def sql: String = unparsedIdentifier @@ -134,14 +143,16 @@ private[sql] case class UnresolvedAttribute( /** * Reference to all columns in a namespace (global, a Dataframe, or a nested struct). * - * @param unparsedTarget name of the namespace. None if the global namespace is supposed to be used. - * @param planId id of the plan (Dataframe) that produces the attribute. + * @param unparsedTarget + * name of the namespace. None if the global namespace is supposed to be used. + * @param planId + * id of the plan (Dataframe) that produces the attribute. */ private[sql] case class UnresolvedStar( unparsedTarget: Option[String], planId: Option[Long] = None, override val origin: Origin = CurrentOrigin.get) - extends ColumnNode { + extends ColumnNode { override private[internal] def normalize(): UnresolvedStar = copy(planId = None, origin = NO_ORIGIN) override def sql: String = unparsedTarget.map(_ + ".*").getOrElse("*") @@ -151,10 +162,12 @@ private[sql] case class UnresolvedStar( * Call a function. This can either be a built-in function, a UDF, or a UDF registered in the * Catalog. * - * @param functionName of the function to invoke. - * @param arguments to pass into the function. - * @param isDistinct (aggregate only) whether the input of the aggregate function should be - * de-duplicated. + * @param functionName + * of the function to invoke. + * @param arguments + * to pass into the function. + * @param isDistinct + * (aggregate only) whether the input of the aggregate function should be de-duplicated. */ private[sql] case class UnresolvedFunction( functionName: String, @@ -163,7 +176,7 @@ private[sql] case class UnresolvedFunction( isUserDefinedFunction: Boolean = false, isInternal: Boolean = false, override val origin: Origin = CurrentOrigin.get) - extends ColumnNode { + extends ColumnNode { override private[internal] def normalize(): UnresolvedFunction = copy(arguments = ColumnNode.normalize(arguments), origin = NO_ORIGIN) @@ -173,11 +186,13 @@ private[sql] case class UnresolvedFunction( /** * Evaluate a SQL expression. * - * @param expression text to execute. + * @param expression + * text to execute. */ private[sql] case class SqlExpression( expression: String, - override val origin: Origin = CurrentOrigin.get) extends ColumnNode { + override val origin: Origin = CurrentOrigin.get) + extends ColumnNode { override private[internal] def normalize(): SqlExpression = copy(origin = NO_ORIGIN) override def sql: String = expression } @@ -185,15 +200,19 @@ private[sql] case class SqlExpression( /** * Name a column, and (optionally) modify its metadata. * - * @param child to name - * @param name to use - * @param metadata (optional) metadata to add. + * @param child + * to name + * @param name + * to use + * @param metadata + * (optional) metadata to add. */ private[sql] case class Alias( child: ColumnNode, name: Seq[String], metadata: Option[Metadata] = None, - override val origin: Origin = CurrentOrigin.get) extends ColumnNode { + override val origin: Origin = CurrentOrigin.get) + extends ColumnNode { override private[internal] def normalize(): Alias = copy(child = child.normalize(), origin = NO_ORIGIN) @@ -210,15 +229,19 @@ private[sql] case class Alias( * Cast the value of a Column to a different [[DataType]]. The behavior of the cast can be * influenced by the `evalMode`. * - * @param child that produces the input value. - * @param dataType to cast to. - * @param evalMode (try/ansi/legacy) to use for the cast. + * @param child + * that produces the input value. + * @param dataType + * to cast to. + * @param evalMode + * (try/ansi/legacy) to use for the cast. */ private[sql] case class Cast( child: ColumnNode, dataType: DataType, evalMode: Option[Cast.EvalMode] = None, - override val origin: Origin = CurrentOrigin.get) extends ColumnNode { + override val origin: Origin = CurrentOrigin.get) + extends ColumnNode { override private[internal] def normalize(): Cast = copy(child = child.normalize(), origin = NO_ORIGIN) @@ -237,13 +260,16 @@ private[sql] object Cast { /** * Reference to all columns in the global namespace in that match a regex. * - * @param regex name of the namespace. None if the global namespace is supposed to be used. - * @param planId id of the plan (Dataframe) that produces the attribute. + * @param regex + * name of the namespace. None if the global namespace is supposed to be used. + * @param planId + * id of the plan (Dataframe) that produces the attribute. */ private[sql] case class UnresolvedRegex( regex: String, planId: Option[Long] = None, - override val origin: Origin = CurrentOrigin.get) extends ColumnNode { + override val origin: Origin = CurrentOrigin.get) + extends ColumnNode { override private[internal] def normalize(): UnresolvedRegex = copy(planId = None, origin = NO_ORIGIN) override def sql: String = regex @@ -252,16 +278,19 @@ private[sql] case class UnresolvedRegex( /** * Sort the input column. * - * @param child to sort. - * @param sortDirection to sort in, either Ascending or Descending. - * @param nullOrdering where to place nulls, either at the begin or the end. + * @param child + * to sort. + * @param sortDirection + * to sort in, either Ascending or Descending. + * @param nullOrdering + * where to place nulls, either at the begin or the end. */ private[sql] case class SortOrder( child: ColumnNode, sortDirection: SortOrder.SortDirection, nullOrdering: SortOrder.NullOrdering, override val origin: Origin = CurrentOrigin.get) - extends ColumnNode { + extends ColumnNode { override private[internal] def normalize(): SortOrder = copy(child = child.normalize(), origin = NO_ORIGIN) @@ -280,14 +309,16 @@ private[sql] object SortOrder { /** * Evaluate a function within a window. * - * @param windowFunction function to execute. - * @param windowSpec of the window. + * @param windowFunction + * function to execute. + * @param windowSpec + * of the window. */ private[sql] case class Window( windowFunction: ColumnNode, windowSpec: WindowSpec, override val origin: Origin = CurrentOrigin.get) - extends ColumnNode { + extends ColumnNode { override private[internal] def normalize(): Window = copy( windowFunction = windowFunction.normalize(), windowSpec = windowSpec.normalize(), @@ -299,7 +330,8 @@ private[sql] case class Window( private[sql] case class WindowSpec( partitionColumns: Seq[ColumnNode], sortColumns: Seq[SortOrder], - frame: Option[WindowFrame] = None) extends ColumnNodeLike { + frame: Option[WindowFrame] = None) + extends ColumnNodeLike { override private[internal] def normalize(): WindowSpec = copy( partitionColumns = ColumnNode.normalize(partitionColumns), sortColumns = ColumnNode.normalize(sortColumns), @@ -317,7 +349,7 @@ private[sql] case class WindowFrame( frameType: WindowFrame.FrameType, lower: WindowFrame.FrameBoundary, upper: WindowFrame.FrameBoundary) - extends ColumnNodeLike { + extends ColumnNodeLike { override private[internal] def normalize(): WindowFrame = copy(lower = lower.normalize(), upper = upper.normalize()) override private[internal] def sql: String = @@ -352,13 +384,16 @@ private[sql] object WindowFrame { /** * Lambda function to execute. This typically passed as an argument to a function. * - * @param function to execute. - * @param arguments the bound lambda variables. + * @param function + * to execute. + * @param arguments + * the bound lambda variables. */ private[sql] case class LambdaFunction( function: ColumnNode, arguments: Seq[UnresolvedNamedLambdaVariable], - override val origin: Origin) extends ColumnNode { + override val origin: Origin) + extends ColumnNode { override private[internal] def normalize(): LambdaFunction = copy( function = function.normalize(), @@ -382,11 +417,13 @@ object LambdaFunction { /** * Variable used in a [[LambdaFunction]]. * - * @param name of the variable. + * @param name + * of the variable. */ private[sql] case class UnresolvedNamedLambdaVariable( name: String, - override val origin: Origin = CurrentOrigin.get) extends ColumnNode { + override val origin: Origin = CurrentOrigin.get) + extends ColumnNode { override private[internal] def normalize(): UnresolvedNamedLambdaVariable = copy(origin = NO_ORIGIN) @@ -413,21 +450,22 @@ object UnresolvedNamedLambdaVariable { } /** - * Extract a value from a complex type. This can be a field from a struct, a value from a map, - * or an element from an array. + * Extract a value from a complex type. This can be a field from a struct, a value from a map, or + * an element from an array. * - * @param child that produces a complex value. - * @param extraction that is used to access the complex type. This needs to be a string type for - * structs and maps, and it needs to be an integer for arrays. + * @param child + * that produces a complex value. + * @param extraction + * that is used to access the complex type. This needs to be a string type for structs and maps, + * and it needs to be an integer for arrays. */ private[sql] case class UnresolvedExtractValue( child: ColumnNode, extraction: ColumnNode, - override val origin: Origin = CurrentOrigin.get) extends ColumnNode { - override private[internal] def normalize(): UnresolvedExtractValue = copy( - child = child.normalize(), - extraction = extraction.normalize(), - origin = NO_ORIGIN) + override val origin: Origin = CurrentOrigin.get) + extends ColumnNode { + override private[internal] def normalize(): UnresolvedExtractValue = + copy(child = child.normalize(), extraction = extraction.normalize(), origin = NO_ORIGIN) override def sql: String = s"${child.sql}[${extraction.sql}]" } @@ -435,15 +473,19 @@ private[sql] case class UnresolvedExtractValue( /** * Update or drop the field of a struct. * - * @param structExpression that will be updated. - * @param fieldName name of the field to update. - * @param valueExpression new value of the field. If this is None the field will be dropped. + * @param structExpression + * that will be updated. + * @param fieldName + * name of the field to update. + * @param valueExpression + * new value of the field. If this is None the field will be dropped. */ private[sql] case class UpdateFields( structExpression: ColumnNode, fieldName: String, valueExpression: Option[ColumnNode] = None, - override val origin: Origin = CurrentOrigin.get) extends ColumnNode { + override val origin: Origin = CurrentOrigin.get) + extends ColumnNode { override private[internal] def normalize(): UpdateFields = copy( structExpression = structExpression.normalize(), valueExpression = ColumnNode.normalize(valueExpression), @@ -455,18 +497,20 @@ private[sql] case class UpdateFields( } /** - * Evaluate one or more conditional branches. The value of the first branch for which the predicate - * evalutes to true is returned. If none of the branches evaluate to true, the value of `otherwise` - * is returned. + * Evaluate one or more conditional branches. The value of the first branch for which the + * predicate evalutes to true is returned. If none of the branches evaluate to true, the value of + * `otherwise` is returned. * - * @param branches to evaluate. Each entry if a pair of condition and value. - * @param otherwise (optional) to evaluate when none of the branches evaluate to true. + * @param branches + * to evaluate. Each entry if a pair of condition and value. + * @param otherwise + * (optional) to evaluate when none of the branches evaluate to true. */ private[sql] case class CaseWhenOtherwise( branches: Seq[(ColumnNode, ColumnNode)], otherwise: Option[ColumnNode] = None, override val origin: Origin = CurrentOrigin.get) - extends ColumnNode { + extends ColumnNode { assert(branches.nonEmpty) override private[internal] def normalize(): CaseWhenOtherwise = copy( branches = branches.map(kv => (kv._1.normalize(), kv._2.normalize())), @@ -483,15 +527,17 @@ private[sql] case class CaseWhenOtherwise( /** * Invoke an inline user defined function. * - * @param function to invoke. - * @param arguments to pass into the user defined function. + * @param function + * to invoke. + * @param arguments + * to pass into the user defined function. */ private[sql] case class InvokeInlineUserDefinedFunction( function: UserDefinedFunctionLike, arguments: Seq[ColumnNode], isDistinct: Boolean = false, override val origin: Origin = CurrentOrigin.get) - extends ColumnNode { + extends ColumnNode { override private[internal] def normalize(): InvokeInlineUserDefinedFunction = copy(arguments = ColumnNode.normalize(arguments), origin = NO_ORIGIN) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/internal/types/AbstractArrayType.scala b/sql/api/src/main/scala/org/apache/spark/sql/internal/types/AbstractArrayType.scala index 406449a337271..5c8c77985bb2c 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/internal/types/AbstractArrayType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/internal/types/AbstractArrayType.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.internal.types import org.apache.spark.sql.types.{AbstractDataType, ArrayType, DataType} - /** * Use AbstractArrayType(AbstractDataType) for defining expected types for expression parameters. */ @@ -30,7 +29,7 @@ case class AbstractArrayType(elementType: AbstractDataType) extends AbstractData override private[sql] def acceptsType(other: DataType): Boolean = { other.isInstanceOf[ArrayType] && - elementType.acceptsType(other.asInstanceOf[ArrayType].elementType) + elementType.acceptsType(other.asInstanceOf[ArrayType].elementType) } override private[spark] def simpleString: String = s"array<${elementType.simpleString}>" diff --git a/sql/api/src/main/scala/org/apache/spark/sql/internal/types/AbstractMapType.scala b/sql/api/src/main/scala/org/apache/spark/sql/internal/types/AbstractMapType.scala index 62f422f6f80a7..32f4341839f01 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/internal/types/AbstractMapType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/internal/types/AbstractMapType.scala @@ -19,23 +19,20 @@ package org.apache.spark.sql.internal.types import org.apache.spark.sql.types.{AbstractDataType, DataType, MapType} - /** - * Use AbstractMapType(AbstractDataType, AbstractDataType) - * for defining expected types for expression parameters. + * Use AbstractMapType(AbstractDataType, AbstractDataType) for defining expected types for + * expression parameters. */ -case class AbstractMapType( - keyType: AbstractDataType, - valueType: AbstractDataType - ) extends AbstractDataType { +case class AbstractMapType(keyType: AbstractDataType, valueType: AbstractDataType) + extends AbstractDataType { override private[sql] def defaultConcreteType: DataType = MapType(keyType.defaultConcreteType, valueType.defaultConcreteType, valueContainsNull = true) override private[sql] def acceptsType(other: DataType): Boolean = { other.isInstanceOf[MapType] && - keyType.acceptsType(other.asInstanceOf[MapType].keyType) && - valueType.acceptsType(other.asInstanceOf[MapType].valueType) + keyType.acceptsType(other.asInstanceOf[MapType].keyType) && + valueType.acceptsType(other.asInstanceOf[MapType].valueType) } override private[spark] def simpleString: String = diff --git a/sql/api/src/main/scala/org/apache/spark/sql/streaming/ExpiredTimerInfo.scala b/sql/api/src/main/scala/org/apache/spark/sql/streaming/ExpiredTimerInfo.scala index 49dc393f8481b..a0958aceb3b3a 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/streaming/ExpiredTimerInfo.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/streaming/ExpiredTimerInfo.scala @@ -22,12 +22,13 @@ import java.io.Serializable import org.apache.spark.annotation.{Evolving, Experimental} /** - * Class used to provide access to expired timer's expiry time. These values - * are only relevant if the ExpiredTimerInfo is valid. + * Class used to provide access to expired timer's expiry time. These values are only relevant if + * the ExpiredTimerInfo is valid. */ @Experimental @Evolving private[sql] trait ExpiredTimerInfo extends Serializable { + /** * Check if provided ExpiredTimerInfo is valid. */ diff --git a/sql/api/src/main/scala/org/apache/spark/sql/streaming/GroupState.scala b/sql/api/src/main/scala/org/apache/spark/sql/streaming/GroupState.scala index f08a2fd3cc55c..146990917a3fc 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/streaming/GroupState.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/streaming/GroupState.scala @@ -27,89 +27,89 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalGroupState * `flatMapGroupsWithState` operations on `KeyValueGroupedDataset`. * * Detail description on `[map/flatMap]GroupsWithState` operation - * -------------------------------------------------------------- - * Both, `mapGroupsWithState` and `flatMapGroupsWithState` in `KeyValueGroupedDataset` - * will invoke the user-given function on each group (defined by the grouping function in - * `Dataset.groupByKey()`) while maintaining a user-defined per-group state between invocations. - * For a static batch Dataset, the function will be invoked once per group. For a streaming - * Dataset, the function will be invoked for each group repeatedly in every trigger. - * That is, in every batch of the `StreamingQuery`, - * the function will be invoked once for each group that has data in the trigger. Furthermore, - * if timeout is set, then the function will be invoked on timed-out groups (more detail below). + * -------------------------------------------------------------- Both, `mapGroupsWithState` and + * `flatMapGroupsWithState` in `KeyValueGroupedDataset` will invoke the user-given function on + * each group (defined by the grouping function in `Dataset.groupByKey()`) while maintaining a + * user-defined per-group state between invocations. For a static batch Dataset, the function will + * be invoked once per group. For a streaming Dataset, the function will be invoked for each group + * repeatedly in every trigger. That is, in every batch of the `StreamingQuery`, the function will + * be invoked once for each group that has data in the trigger. Furthermore, if timeout is set, + * then the function will be invoked on timed-out groups (more detail below). * * The function is invoked with the following parameters. - * - The key of the group. - * - An iterator containing all the values for this group. - * - A user-defined state object set by previous invocations of the given function. + * - The key of the group. + * - An iterator containing all the values for this group. + * - A user-defined state object set by previous invocations of the given function. * * In case of a batch Dataset, there is only one invocation and the state object will be empty as - * there is no prior state. Essentially, for batch Datasets, `[map/flatMap]GroupsWithState` - * is equivalent to `[map/flatMap]Groups` and any updates to the state and/or timeouts have - * no effect. + * there is no prior state. Essentially, for batch Datasets, `[map/flatMap]GroupsWithState` is + * equivalent to `[map/flatMap]Groups` and any updates to the state and/or timeouts have no + * effect. * * The major difference between `mapGroupsWithState` and `flatMapGroupsWithState` is that the - * former allows the function to return one and only one record, whereas the latter - * allows the function to return any number of records (including no records). Furthermore, the + * former allows the function to return one and only one record, whereas the latter allows the + * function to return any number of records (including no records). Furthermore, the * `flatMapGroupsWithState` is associated with an operation output mode, which can be either - * `Append` or `Update`. Semantically, this defines whether the output records of one trigger - * is effectively replacing the previously output records (from previous triggers) or is appending - * to the list of previously output records. Essentially, this defines how the Result Table (refer - * to the semantics in the programming guide) is updated, and allows us to reason about the - * semantics of later operations. + * `Append` or `Update`. Semantically, this defines whether the output records of one trigger is + * effectively replacing the previously output records (from previous triggers) or is appending to + * the list of previously output records. Essentially, this defines how the Result Table (refer to + * the semantics in the programming guide) is updated, and allows us to reason about the semantics + * of later operations. * - * Important points to note about the function (both mapGroupsWithState and flatMapGroupsWithState). - * - In a trigger, the function will be called only the groups present in the batch. So do not - * assume that the function will be called in every trigger for every group that has state. - * - There is no guaranteed ordering of values in the iterator in the function, neither with - * batch, nor with streaming Datasets. - * - All the data will be shuffled before applying the function. - * - If timeout is set, then the function will also be called with no values. - * See more details on `GroupStateTimeout` below. + * Important points to note about the function (both mapGroupsWithState and + * flatMapGroupsWithState). + * - In a trigger, the function will be called only the groups present in the batch. So do not + * assume that the function will be called in every trigger for every group that has state. + * - There is no guaranteed ordering of values in the iterator in the function, neither with + * batch, nor with streaming Datasets. + * - All the data will be shuffled before applying the function. + * - If timeout is set, then the function will also be called with no values. See more details + * on `GroupStateTimeout` below. * * Important points to note about using `GroupState`. - * - The value of the state cannot be null. So updating state with null will throw - * `IllegalArgumentException`. - * - Operations on `GroupState` are not thread-safe. This is to avoid memory barriers. - * - If `remove()` is called, then `exists()` will return `false`, - * `get()` will throw `NoSuchElementException` and `getOption()` will return `None` - * - After that, if `update(newState)` is called, then `exists()` will again return `true`, - * `get()` and `getOption()`will return the updated value. + * - The value of the state cannot be null. So updating state with null will throw + * `IllegalArgumentException`. + * - Operations on `GroupState` are not thread-safe. This is to avoid memory barriers. + * - If `remove()` is called, then `exists()` will return `false`, `get()` will throw + * `NoSuchElementException` and `getOption()` will return `None` + * - After that, if `update(newState)` is called, then `exists()` will again return `true`, + * `get()` and `getOption()`will return the updated value. * * Important points to note about using `GroupStateTimeout`. - * - The timeout type is a global param across all the groups (set as `timeout` param in - * `[map|flatMap]GroupsWithState`, but the exact timeout duration/timestamp is configurable per - * group by calling `setTimeout...()` in `GroupState`. - * - Timeouts can be either based on processing time (i.e. - * `GroupStateTimeout.ProcessingTimeTimeout`) or event time (i.e. - * `GroupStateTimeout.EventTimeTimeout`). - * - With `ProcessingTimeTimeout`, the timeout duration can be set by calling - * `GroupState.setTimeoutDuration`. The timeout will occur when the clock has advanced by the set - * duration. Guarantees provided by this timeout with a duration of D ms are as follows: - * - Timeout will never occur before the clock time has advanced by D ms - * - Timeout will occur eventually when there is a trigger in the query - * (i.e. after D ms). So there is no strict upper bound on when the timeout would occur. - * For example, the trigger interval of the query will affect when the timeout actually occurs. - * If there is no data in the stream (for any group) for a while, then there will not be - * any trigger and timeout function call will not occur until there is data. - * - Since the processing time timeout is based on the clock time, it is affected by the - * variations in the system clock (i.e. time zone changes, clock skew, etc.). - * - With `EventTimeTimeout`, the user also has to specify the event time watermark in - * the query using `Dataset.withWatermark()`. With this setting, data that is older than the - * watermark is filtered out. The timeout can be set for a group by setting a timeout timestamp - * using`GroupState.setTimeoutTimestamp()`, and the timeout would occur when the watermark - * advances beyond the set timestamp. You can control the timeout delay by two parameters - - * (i) watermark delay and an additional duration beyond the timestamp in the event (which - * is guaranteed to be newer than watermark due to the filtering). Guarantees provided by this - * timeout are as follows: - * - Timeout will never occur before the watermark has exceeded the set timeout. - * - Similar to processing time timeouts, there is no strict upper bound on the delay when - * the timeout actually occurs. The watermark can advance only when there is data in the - * stream and the event time of the data has actually advanced. - * - When the timeout occurs for a group, the function is called for that group with no values, and - * `GroupState.hasTimedOut()` set to true. - * - The timeout is reset every time the function is called on a group, that is, - * when the group has new data, or the group has timed out. So the user has to set the timeout - * duration every time the function is called, otherwise, there will not be any timeout set. + * - The timeout type is a global param across all the groups (set as `timeout` param in + * `[map|flatMap]GroupsWithState`, but the exact timeout duration/timestamp is configurable + * per group by calling `setTimeout...()` in `GroupState`. + * - Timeouts can be either based on processing time (i.e. + * `GroupStateTimeout.ProcessingTimeTimeout`) or event time (i.e. + * `GroupStateTimeout.EventTimeTimeout`). + * - With `ProcessingTimeTimeout`, the timeout duration can be set by calling + * `GroupState.setTimeoutDuration`. The timeout will occur when the clock has advanced by the + * set duration. Guarantees provided by this timeout with a duration of D ms are as follows: + * - Timeout will never occur before the clock time has advanced by D ms + * - Timeout will occur eventually when there is a trigger in the query (i.e. after D ms). So + * there is no strict upper bound on when the timeout would occur. For example, the trigger + * interval of the query will affect when the timeout actually occurs. If there is no data + * in the stream (for any group) for a while, then there will not be any trigger and timeout + * function call will not occur until there is data. + * - Since the processing time timeout is based on the clock time, it is affected by the + * variations in the system clock (i.e. time zone changes, clock skew, etc.). + * - With `EventTimeTimeout`, the user also has to specify the event time watermark in the query + * using `Dataset.withWatermark()`. With this setting, data that is older than the watermark + * is filtered out. The timeout can be set for a group by setting a timeout timestamp + * using`GroupState.setTimeoutTimestamp()`, and the timeout would occur when the watermark + * advances beyond the set timestamp. You can control the timeout delay by two parameters - + * (i) watermark delay and an additional duration beyond the timestamp in the event (which is + * guaranteed to be newer than watermark due to the filtering). Guarantees provided by this + * timeout are as follows: + * - Timeout will never occur before the watermark has exceeded the set timeout. + * - Similar to processing time timeouts, there is no strict upper bound on the delay when the + * timeout actually occurs. The watermark can advance only when there is data in the stream + * and the event time of the data has actually advanced. + * - When the timeout occurs for a group, the function is called for that group with no values, + * and `GroupState.hasTimedOut()` set to true. + * - The timeout is reset every time the function is called on a group, that is, when the group + * has new data, or the group has timed out. So the user has to set the timeout duration every + * time the function is called, otherwise, there will not be any timeout set. * * `[map/flatMap]GroupsWithState` can take a user defined initial state as an additional argument. * This state will be applied when the first batch of the streaming query is processed. If there @@ -181,7 +181,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalGroupState * state.setTimeoutDuration("1 hour"); // Set the timeout * } * ... -* // return something + * // return something * } * }; * @@ -191,8 +191,9 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalGroupState * mappingFunction, Encoders.INT, Encoders.STRING, GroupStateTimeout.ProcessingTimeTimeout); * }}} * - * @tparam S User-defined type of the state to be stored for each group. Must be encodable into - * Spark SQL types (see `Encoder` for more details). + * @tparam S + * User-defined type of the state to be stored for each group. Must be encodable into Spark SQL + * types (see `Encoder` for more details). * @since 2.2.0 */ @Experimental @@ -217,44 +218,48 @@ trait GroupState[S] extends LogicalGroupState[S] { /** * Whether the function has been called because the key has timed out. - * @note This can return true only when timeouts are enabled in `[map/flatMap]GroupsWithState`. + * @note + * This can return true only when timeouts are enabled in `[map/flatMap]GroupsWithState`. */ def hasTimedOut: Boolean - /** * Set the timeout duration in ms for this key. * - * @note [[GroupStateTimeout Processing time timeout]] must be enabled in - * `[map/flatMap]GroupsWithState` for calling this method. - * @note This method has no effect when used in a batch query. + * @note + * [[GroupStateTimeout Processing time timeout]] must be enabled in + * `[map/flatMap]GroupsWithState` for calling this method. + * @note + * This method has no effect when used in a batch query. */ @throws[IllegalArgumentException]("if 'durationMs' is not positive") @throws[UnsupportedOperationException]( "if processing time timeout has not been enabled in [map|flatMap]GroupsWithState") def setTimeoutDuration(durationMs: Long): Unit - /** * Set the timeout duration for this key as a string. For example, "1 hour", "2 days", etc. * - * @note [[GroupStateTimeout Processing time timeout]] must be enabled in - * `[map/flatMap]GroupsWithState` for calling this method. - * @note This method has no effect when used in a batch query. + * @note + * [[GroupStateTimeout Processing time timeout]] must be enabled in + * `[map/flatMap]GroupsWithState` for calling this method. + * @note + * This method has no effect when used in a batch query. */ @throws[IllegalArgumentException]("if 'duration' is not a valid duration") @throws[UnsupportedOperationException]( "if processing time timeout has not been enabled in [map|flatMap]GroupsWithState") def setTimeoutDuration(duration: String): Unit - /** - * Set the timeout timestamp for this key as milliseconds in epoch time. - * This timestamp cannot be older than the current watermark. + * Set the timeout timestamp for this key as milliseconds in epoch time. This timestamp cannot + * be older than the current watermark. * - * @note [[GroupStateTimeout Event time timeout]] must be enabled in - * `[map/flatMap]GroupsWithState` for calling this method. - * @note This method has no effect when used in a batch query. + * @note + * [[GroupStateTimeout Event time timeout]] must be enabled in `[map/flatMap]GroupsWithState` + * for calling this method. + * @note + * This method has no effect when used in a batch query. */ @throws[IllegalArgumentException]( "if 'timestampMs' is not positive or less than the current watermark in a streaming query") @@ -262,16 +267,16 @@ trait GroupState[S] extends LogicalGroupState[S] { "if event time timeout has not been enabled in [map|flatMap]GroupsWithState") def setTimeoutTimestamp(timestampMs: Long): Unit - /** * Set the timeout timestamp for this key as milliseconds in epoch time and an additional - * duration as a string (e.g. "1 hour", "2 days", etc.). - * The final timestamp (including the additional duration) cannot be older than the - * current watermark. + * duration as a string (e.g. "1 hour", "2 days", etc.). The final timestamp (including the + * additional duration) cannot be older than the current watermark. * - * @note [[GroupStateTimeout Event time timeout]] must be enabled in - * `[map/flatMap]GroupsWithState` for calling this method. - * @note This method has no side effect when used in a batch query. + * @note + * [[GroupStateTimeout Event time timeout]] must be enabled in `[map/flatMap]GroupsWithState` + * for calling this method. + * @note + * This method has no side effect when used in a batch query. */ @throws[IllegalArgumentException]( "if 'additionalDuration' is invalid or the final timeout timestamp is less than " + @@ -280,56 +285,57 @@ trait GroupState[S] extends LogicalGroupState[S] { "if event time timeout has not been enabled in [map|flatMap]GroupsWithState") def setTimeoutTimestamp(timestampMs: Long, additionalDuration: String): Unit - /** - * Set the timeout timestamp for this key as a java.sql.Date. - * This timestamp cannot be older than the current watermark. + * Set the timeout timestamp for this key as a java.sql.Date. This timestamp cannot be older + * than the current watermark. * - * @note [[GroupStateTimeout Event time timeout]] must be enabled in - * `[map/flatMap]GroupsWithState` for calling this method. - * @note This method has no side effect when used in a batch query. + * @note + * [[GroupStateTimeout Event time timeout]] must be enabled in `[map/flatMap]GroupsWithState` + * for calling this method. + * @note + * This method has no side effect when used in a batch query. */ @throws[UnsupportedOperationException]( "if event time timeout has not been enabled in [map|flatMap]GroupsWithState") def setTimeoutTimestamp(timestamp: java.sql.Date): Unit - /** - * Set the timeout timestamp for this key as a java.sql.Date and an additional - * duration as a string (e.g. "1 hour", "2 days", etc.). - * The final timestamp (including the additional duration) cannot be older than the - * current watermark. + * Set the timeout timestamp for this key as a java.sql.Date and an additional duration as a + * string (e.g. "1 hour", "2 days", etc.). The final timestamp (including the additional + * duration) cannot be older than the current watermark. * - * @note [[GroupStateTimeout Event time timeout]] must be enabled in - * `[map/flatMap]GroupsWithState` for calling this method. - * @note This method has no side effect when used in a batch query. + * @note + * [[GroupStateTimeout Event time timeout]] must be enabled in `[map/flatMap]GroupsWithState` + * for calling this method. + * @note + * This method has no side effect when used in a batch query. */ @throws[IllegalArgumentException]("if 'additionalDuration' is invalid") @throws[UnsupportedOperationException]( "if event time timeout has not been enabled in [map|flatMap]GroupsWithState") def setTimeoutTimestamp(timestamp: java.sql.Date, additionalDuration: String): Unit - /** * Get the current event time watermark as milliseconds in epoch time. * - * @note In a streaming query, this can be called only when watermark is set before calling - * `[map/flatMap]GroupsWithState`. In a batch query, this method always returns -1. - * @note The watermark gets propagated in the end of each query. As a result, this method will - * return 0 (1970-01-01T00:00:00) for the first micro-batch. If you use this value - * as a part of the timestamp set in the `setTimeoutTimestamp`, it may lead to the - * state expiring immediately in the next micro-batch, once the watermark gets the - * real value from your data. + * @note + * In a streaming query, this can be called only when watermark is set before calling + * `[map/flatMap]GroupsWithState`. In a batch query, this method always returns -1. + * @note + * The watermark gets propagated in the end of each query. As a result, this method will + * return 0 (1970-01-01T00:00:00) for the first micro-batch. If you use this value as a part + * of the timestamp set in the `setTimeoutTimestamp`, it may lead to the state expiring + * immediately in the next micro-batch, once the watermark gets the real value from your data. */ @throws[UnsupportedOperationException]( "if watermark has not been set before in [map|flatMap]GroupsWithState") def getCurrentWatermarkMs(): Long - /** * Get the current processing time as milliseconds in epoch time. - * @note In a streaming query, this will return a constant value throughout the duration of a - * trigger, even if the trigger is re-executed. + * @note + * In a streaming query, this will return a constant value throughout the duration of a + * trigger, even if the trigger is re-executed. */ def getCurrentProcessingTimeMs(): Long } diff --git a/sql/api/src/main/scala/org/apache/spark/sql/streaming/ListState.scala b/sql/api/src/main/scala/org/apache/spark/sql/streaming/ListState.scala index 0e2d6cc3778c6..568578d1f4ff6 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/streaming/ListState.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/streaming/ListState.scala @@ -21,8 +21,7 @@ import org.apache.spark.annotation.{Evolving, Experimental} @Experimental @Evolving /** - * Interface used for arbitrary stateful operations with the v2 API to capture - * list value state. + * Interface used for arbitrary stateful operations with the v2 API to capture list value state. */ private[sql] trait ListState[S] extends Serializable { diff --git a/sql/api/src/main/scala/org/apache/spark/sql/streaming/MapState.scala b/sql/api/src/main/scala/org/apache/spark/sql/streaming/MapState.scala index 030c3ee989c6f..7b01888bbac49 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/streaming/MapState.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/streaming/MapState.scala @@ -21,10 +21,10 @@ import org.apache.spark.annotation.{Evolving, Experimental} @Experimental @Evolving /** - * Interface used for arbitrary stateful operations with the v2 API to capture - * map value state. + * Interface used for arbitrary stateful operations with the v2 API to capture map value state. */ trait MapState[K, V] extends Serializable { + /** Whether state exists or not. */ def exists(): Boolean @@ -35,7 +35,7 @@ trait MapState[K, V] extends Serializable { def containsKey(key: K): Boolean /** Update value for given user key */ - def updateValue(key: K, value: V) : Unit + def updateValue(key: K, value: V): Unit /** Get the map associated with grouping key */ def iterator(): Iterator[(K, V)] diff --git a/sql/api/src/main/scala/org/apache/spark/sql/streaming/QueryInfo.scala b/sql/api/src/main/scala/org/apache/spark/sql/streaming/QueryInfo.scala index 7754a514fdd6b..f239bcff49fea 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/streaming/QueryInfo.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/streaming/QueryInfo.scala @@ -22,8 +22,8 @@ import java.util.UUID import org.apache.spark.annotation.{Evolving, Experimental} /** - * Represents the query info provided to the stateful processor used in the - * arbitrary state API v2 to easily identify task retries on the same partition. + * Represents the query info provided to the stateful processor used in the arbitrary state API v2 + * to easily identify task retries on the same partition. */ @Experimental @Evolving diff --git a/sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessor.scala b/sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessor.scala index 54e6a9a4ab678..d2c6010454c55 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessor.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessor.scala @@ -31,31 +31,35 @@ import org.apache.spark.sql.errors.ExecutionErrors private[sql] abstract class StatefulProcessor[K, I, O] extends Serializable { /** - * Handle to the stateful processor that provides access to the state store and other - * stateful processing related APIs. + * Handle to the stateful processor that provides access to the state store and other stateful + * processing related APIs. */ private var statefulProcessorHandle: StatefulProcessorHandle = null /** - * Function that will be invoked as the first method that allows for users to - * initialize all their state variables and perform other init actions before handling data. - * @param outputMode - output mode for the stateful processor - * @param timeMode - time mode for the stateful processor. + * Function that will be invoked as the first method that allows for users to initialize all + * their state variables and perform other init actions before handling data. + * @param outputMode + * \- output mode for the stateful processor + * @param timeMode + * \- time mode for the stateful processor. */ - def init( - outputMode: OutputMode, - timeMode: TimeMode): Unit + def init(outputMode: OutputMode, timeMode: TimeMode): Unit /** * Function that will allow users to interact with input data rows along with the grouping key * and current timer values and optionally provide output rows. - * @param key - grouping key - * @param inputRows - iterator of input rows associated with grouping key - * @param timerValues - instance of TimerValues that provides access to current processing/event - * time if available - * @param expiredTimerInfo - instance of ExpiredTimerInfo that provides access to expired timer - * if applicable - * @return - Zero or more output rows + * @param key + * \- grouping key + * @param inputRows + * \- iterator of input rows associated with grouping key + * @param timerValues + * \- instance of TimerValues that provides access to current processing/event time if + * available + * @param expiredTimerInfo + * \- instance of ExpiredTimerInfo that provides access to expired timer if applicable + * @return + * \- Zero or more output rows */ def handleInputRows( key: K, @@ -64,16 +68,17 @@ private[sql] abstract class StatefulProcessor[K, I, O] extends Serializable { expiredTimerInfo: ExpiredTimerInfo): Iterator[O] /** - * Function called as the last method that allows for users to perform - * any cleanup or teardown operations. + * Function called as the last method that allows for users to perform any cleanup or teardown + * operations. */ - def close (): Unit = {} + def close(): Unit = {} /** * Function to set the stateful processor handle that will be used to interact with the state * store and other stateful processor related operations. * - * @param handle - instance of StatefulProcessorHandle + * @param handle + * \- instance of StatefulProcessorHandle */ final def setHandle(handle: StatefulProcessorHandle): Unit = { statefulProcessorHandle = handle @@ -82,7 +87,8 @@ private[sql] abstract class StatefulProcessor[K, I, O] extends Serializable { /** * Function to get the stateful processor handle that will be used to interact with the state * - * @return handle - instance of StatefulProcessorHandle + * @return + * handle - instance of StatefulProcessorHandle */ final def getHandle: StatefulProcessorHandle = { if (statefulProcessorHandle == null) { @@ -93,23 +99,25 @@ private[sql] abstract class StatefulProcessor[K, I, O] extends Serializable { } /** - * Stateful processor with support for specifying initial state. - * Accepts a user-defined type as initial state to be initialized in the first batch. - * This can be used for starting a new streaming query with existing state from a - * previous streaming query. + * Stateful processor with support for specifying initial state. Accepts a user-defined type as + * initial state to be initialized in the first batch. This can be used for starting a new + * streaming query with existing state from a previous streaming query. */ @Experimental @Evolving private[sql] abstract class StatefulProcessorWithInitialState[K, I, O, S] - extends StatefulProcessor[K, I, O] { + extends StatefulProcessor[K, I, O] { /** * Function that will be invoked only in the first batch for users to process initial states. * - * @param key - grouping key - * @param initialState - A row in the initial state to be processed - * @param timerValues - instance of TimerValues that provides access to current processing/event - * time if available + * @param key + * \- grouping key + * @param initialState + * \- A row in the initial state to be processed + * @param timerValues + * \- instance of TimerValues that provides access to current processing/event time if + * available */ def handleInitialState(key: K, initialState: S, timerValues: TimerValues): Unit } diff --git a/sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessorHandle.scala b/sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessorHandle.scala index 4dc2ca875ef0e..d1eca0f3967d9 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessorHandle.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessorHandle.scala @@ -22,39 +22,46 @@ import org.apache.spark.annotation.{Evolving, Experimental} import org.apache.spark.sql.Encoder /** - * Represents the operation handle provided to the stateful processor used in the - * arbitrary state API v2. + * Represents the operation handle provided to the stateful processor used in the arbitrary state + * API v2. */ @Experimental @Evolving private[sql] trait StatefulProcessorHandle extends Serializable { /** - * Function to create new or return existing single value state variable of given type. - * The user must ensure to call this function only within the `init()` method of the - * StatefulProcessor. + * Function to create new or return existing single value state variable of given type. The user + * must ensure to call this function only within the `init()` method of the StatefulProcessor. * - * @param stateName - name of the state variable - * @param valEncoder - SQL encoder for state variable - * @tparam T - type of state variable - * @return - instance of ValueState of type T that can be used to store state persistently + * @param stateName + * \- name of the state variable + * @param valEncoder + * \- SQL encoder for state variable + * @tparam T + * \- type of state variable + * @return + * \- instance of ValueState of type T that can be used to store state persistently */ def getValueState[T](stateName: String, valEncoder: Encoder[T]): ValueState[T] /** - * Function to create new or return existing single value state variable of given type - * with ttl. State values will not be returned past ttlDuration, and will be eventually removed - * from the state store. Any state update resets the ttl to current processing time plus - * ttlDuration. + * Function to create new or return existing single value state variable of given type with ttl. + * State values will not be returned past ttlDuration, and will be eventually removed from the + * state store. Any state update resets the ttl to current processing time plus ttlDuration. * * The user must ensure to call this function only within the `init()` method of the * StatefulProcessor. * - * @param stateName - name of the state variable - * @param valEncoder - SQL encoder for state variable - * @param ttlConfig - the ttl configuration (time to live duration etc.) - * @tparam T - type of state variable - * @return - instance of ValueState of type T that can be used to store state persistently + * @param stateName + * \- name of the state variable + * @param valEncoder + * \- SQL encoder for state variable + * @param ttlConfig + * \- the ttl configuration (time to live duration etc.) + * @tparam T + * \- type of state variable + * @return + * \- instance of ValueState of type T that can be used to store state persistently */ def getValueState[T]( stateName: String, @@ -62,30 +69,39 @@ private[sql] trait StatefulProcessorHandle extends Serializable { ttlConfig: TTLConfig): ValueState[T] /** - * Creates new or returns existing list state associated with stateName. - * The ListState persists values of type T. + * Creates new or returns existing list state associated with stateName. The ListState persists + * values of type T. * - * @param stateName - name of the state variable - * @param valEncoder - SQL encoder for state variable - * @tparam T - type of state variable - * @return - instance of ListState of type T that can be used to store state persistently + * @param stateName + * \- name of the state variable + * @param valEncoder + * \- SQL encoder for state variable + * @tparam T + * \- type of state variable + * @return + * \- instance of ListState of type T that can be used to store state persistently */ def getListState[T](stateName: String, valEncoder: Encoder[T]): ListState[T] /** - * Function to create new or return existing list state variable of given type - * with ttl. State values will not be returned past ttlDuration, and will be eventually removed - * from the state store. Any values in listState which have expired after ttlDuration will not - * be returned on get() and will be eventually removed from the state. + * Function to create new or return existing list state variable of given type with ttl. State + * values will not be returned past ttlDuration, and will be eventually removed from the state + * store. Any values in listState which have expired after ttlDuration will not be returned on + * get() and will be eventually removed from the state. * * The user must ensure to call this function only within the `init()` method of the * StatefulProcessor. * - * @param stateName - name of the state variable - * @param valEncoder - SQL encoder for state variable - * @param ttlConfig - the ttl configuration (time to live duration etc.) - * @tparam T - type of state variable - * @return - instance of ListState of type T that can be used to store state persistently + * @param stateName + * \- name of the state variable + * @param valEncoder + * \- SQL encoder for state variable + * @param ttlConfig + * \- the ttl configuration (time to live duration etc.) + * @tparam T + * \- type of state variable + * @return + * \- instance of ListState of type T that can be used to store state persistently */ def getListState[T]( stateName: String, @@ -93,15 +109,21 @@ private[sql] trait StatefulProcessorHandle extends Serializable { ttlConfig: TTLConfig): ListState[T] /** - * Creates new or returns existing map state associated with stateName. - * The MapState persists Key-Value pairs of type [K, V]. + * Creates new or returns existing map state associated with stateName. The MapState persists + * Key-Value pairs of type [K, V]. * - * @param stateName - name of the state variable - * @param userKeyEnc - spark sql encoder for the map key - * @param valEncoder - spark sql encoder for the map value - * @tparam K - type of key for map state variable - * @tparam V - type of value for map state variable - * @return - instance of MapState of type [K,V] that can be used to store state persistently + * @param stateName + * \- name of the state variable + * @param userKeyEnc + * \- spark sql encoder for the map key + * @param valEncoder + * \- spark sql encoder for the map value + * @tparam K + * \- type of key for map state variable + * @tparam V + * \- type of value for map state variable + * @return + * \- instance of MapState of type [K,V] that can be used to store state persistently */ def getMapState[K, V]( stateName: String, @@ -109,57 +131,68 @@ private[sql] trait StatefulProcessorHandle extends Serializable { valEncoder: Encoder[V]): MapState[K, V] /** - * Function to create new or return existing map state variable of given type - * with ttl. State values will not be returned past ttlDuration, and will be eventually removed - * from the state store. Any values in mapState which have expired after ttlDuration will not - * returned on get() and will be eventually removed from the state. + * Function to create new or return existing map state variable of given type with ttl. State + * values will not be returned past ttlDuration, and will be eventually removed from the state + * store. Any values in mapState which have expired after ttlDuration will not returned on get() + * and will be eventually removed from the state. * * The user must ensure to call this function only within the `init()` method of the * StatefulProcessor. * - * @param stateName - name of the state variable - * @param userKeyEnc - spark sql encoder for the map key - * @param valEncoder - SQL encoder for state variable - * @param ttlConfig - the ttl configuration (time to live duration etc.) - * @tparam K - type of key for map state variable - * @tparam V - type of value for map state variable - * @return - instance of MapState of type [K,V] that can be used to store state persistently + * @param stateName + * \- name of the state variable + * @param userKeyEnc + * \- spark sql encoder for the map key + * @param valEncoder + * \- SQL encoder for state variable + * @param ttlConfig + * \- the ttl configuration (time to live duration etc.) + * @tparam K + * \- type of key for map state variable + * @tparam V + * \- type of value for map state variable + * @return + * \- instance of MapState of type [K,V] that can be used to store state persistently */ def getMapState[K, V]( - stateName: String, - userKeyEnc: Encoder[K], - valEncoder: Encoder[V], - ttlConfig: TTLConfig): MapState[K, V] + stateName: String, + userKeyEnc: Encoder[K], + valEncoder: Encoder[V], + ttlConfig: TTLConfig): MapState[K, V] /** Function to return queryInfo for currently running task */ def getQueryInfo(): QueryInfo /** - * Function to register a processing/event time based timer for given implicit grouping key - * and provided timestamp - * @param expiryTimestampMs - timer expiry timestamp in milliseconds + * Function to register a processing/event time based timer for given implicit grouping key and + * provided timestamp + * @param expiryTimestampMs + * \- timer expiry timestamp in milliseconds */ def registerTimer(expiryTimestampMs: Long): Unit /** - * Function to delete a processing/event time based timer for given implicit grouping key - * and provided timestamp - * @param expiryTimestampMs - timer expiry timestamp in milliseconds + * Function to delete a processing/event time based timer for given implicit grouping key and + * provided timestamp + * @param expiryTimestampMs + * \- timer expiry timestamp in milliseconds */ def deleteTimer(expiryTimestampMs: Long): Unit /** - * Function to list all the timers registered for given implicit grouping key - * Note: calling listTimers() within the `handleInputRows` method of the StatefulProcessor - * will return all the unprocessed registered timers, including the one being fired within the - * invocation of `handleInputRows`. - * @return - list of all the registered timers for given implicit grouping key + * Function to list all the timers registered for given implicit grouping key Note: calling + * listTimers() within the `handleInputRows` method of the StatefulProcessor will return all the + * unprocessed registered timers, including the one being fired within the invocation of + * `handleInputRows`. + * @return + * \- list of all the registered timers for given implicit grouping key */ def listTimers(): Iterator[Long] /** * Function to delete and purge state variable if defined previously - * @param stateName - name of the state variable + * @param stateName + * \- name of the state variable */ def deleteIfExists(stateName: String): Unit } diff --git a/sql/api/src/main/scala/org/apache/spark/sql/streaming/TTLConfig.scala b/sql/api/src/main/scala/org/apache/spark/sql/streaming/TTLConfig.scala index 576e09d5d7fe2..ce786aa943d89 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/streaming/TTLConfig.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/streaming/TTLConfig.scala @@ -20,11 +20,11 @@ package org.apache.spark.sql.streaming import java.time.Duration /** - * TTL Configuration for state variable. State values will not be returned past ttlDuration, - * and will be eventually removed from the state store. Any state update resets the ttl to - * current processing time plus ttlDuration. + * TTL Configuration for state variable. State values will not be returned past ttlDuration, and + * will be eventually removed from the state store. Any state update resets the ttl to current + * processing time plus ttlDuration. * - * @param ttlDuration time to live duration for state - * stored in the state variable. + * @param ttlDuration + * time to live duration for state stored in the state variable. */ case class TTLConfig(ttlDuration: Duration) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/streaming/TimerValues.scala b/sql/api/src/main/scala/org/apache/spark/sql/streaming/TimerValues.scala index f0aef58228188..04c5f59428f7f 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/streaming/TimerValues.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/streaming/TimerValues.scala @@ -22,25 +22,29 @@ import java.io.Serializable import org.apache.spark.annotation.{Evolving, Experimental} /** - * Class used to provide access to timer values for processing and event time populated - * before method invocations using the arbitrary state API v2. + * Class used to provide access to timer values for processing and event time populated before + * method invocations using the arbitrary state API v2. */ @Experimental @Evolving private[sql] trait TimerValues extends Serializable { + /** * Get the current processing time as milliseconds in epoch time. - * @note This will return a constant value throughout the duration of a streaming query trigger, - * even if the trigger is re-executed. + * @note + * This will return a constant value throughout the duration of a streaming query trigger, + * even if the trigger is re-executed. */ def getCurrentProcessingTimeInMs(): Long /** * Get the current event time watermark as milliseconds in epoch time. * - * @note This can be called only when watermark is set before calling `transformWithState`. - * @note The watermark gets propagated at the end of each query. As a result, this method will - * return 0 (1970-01-01T00:00:00) for the first micro-batch. + * @note + * This can be called only when watermark is set before calling `transformWithState`. + * @note + * The watermark gets propagated at the end of each query. As a result, this method will + * return 0 (1970-01-01T00:00:00) for the first micro-batch. */ def getCurrentWatermarkInMs(): Long } diff --git a/sql/api/src/main/scala/org/apache/spark/sql/streaming/ValueState.scala b/sql/api/src/main/scala/org/apache/spark/sql/streaming/ValueState.scala index 8a2661e1a55b1..edb5f65365ab8 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/streaming/ValueState.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/streaming/ValueState.scala @@ -24,8 +24,7 @@ import org.apache.spark.annotation.{Evolving, Experimental} @Experimental @Evolving /** - * Interface used for arbitrary stateful operations with the v2 API to capture - * single value state. + * Interface used for arbitrary stateful operations with the v2 API to capture single value state. */ private[sql] trait ValueState[S] extends Serializable { @@ -34,7 +33,8 @@ private[sql] trait ValueState[S] extends Serializable { /** * Get the state value if it exists - * @throws java.util.NoSuchElementException if the state does not exist + * @throws java.util.NoSuchElementException + * if the state does not exist */ @throws[NoSuchElementException] def get(): S @@ -45,7 +45,8 @@ private[sql] trait ValueState[S] extends Serializable { /** * Update the value of the state. * - * @param newState the new value + * @param newState + * the new value */ def update(newState: S): Unit diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala index 67f634f8379cd..9590fb23e16b1 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.errors.DataTypeErrors * A non-concrete data type, reserved for internal uses. */ private[sql] abstract class AbstractDataType { + /** * The default concrete type to use if we want to cast a null literal into this type. */ @@ -47,7 +48,6 @@ private[sql] abstract class AbstractDataType { private[sql] def simpleString: String } - /** * A collection of types that can be used to specify type constraints. The sequence also specifies * precedence: an earlier type takes precedence over a latter type. @@ -59,7 +59,7 @@ private[sql] abstract class AbstractDataType { * This means that we prefer StringType over BinaryType if it is possible to cast to StringType. */ private[sql] class TypeCollection(private val types: Seq[AbstractDataType]) - extends AbstractDataType { + extends AbstractDataType { require(types.nonEmpty, s"TypeCollection ($types) cannot be empty") @@ -73,22 +73,20 @@ private[sql] class TypeCollection(private val types: Seq[AbstractDataType]) } } - private[sql] object TypeCollection { /** * Types that include numeric types and ANSI interval types. */ - val NumericAndAnsiInterval = TypeCollection( - NumericType, - DayTimeIntervalType, - YearMonthIntervalType) + val NumericAndAnsiInterval = + TypeCollection(NumericType, DayTimeIntervalType, YearMonthIntervalType) /** - * Types that include numeric and ANSI interval types, and additionally the legacy interval type. - * They are only used in unary_minus, unary_positive, add and subtract operations. + * Types that include numeric and ANSI interval types, and additionally the legacy interval + * type. They are only used in unary_minus, unary_positive, add and subtract operations. */ - val NumericAndInterval = new TypeCollection(NumericAndAnsiInterval.types :+ CalendarIntervalType) + val NumericAndInterval = new TypeCollection( + NumericAndAnsiInterval.types :+ CalendarIntervalType) def apply(types: AbstractDataType*): TypeCollection = new TypeCollection(types) @@ -98,7 +96,6 @@ private[sql] object TypeCollection { } } - /** * An `AbstractDataType` that matches any concrete data types. */ @@ -114,15 +111,14 @@ protected[sql] object AnyDataType extends AbstractDataType with Serializable { override private[sql] def acceptsType(other: DataType): Boolean = true } - /** - * An internal type used to represent everything that is not null, UDTs, arrays, structs, and maps. + * An internal type used to represent everything that is not null, UDTs, arrays, structs, and + * maps. */ protected[sql] abstract class AtomicType extends DataType object AtomicType - /** * Numeric data types. * @@ -131,7 +127,6 @@ object AtomicType @Stable abstract class NumericType extends AtomicType - private[spark] object NumericType extends AbstractDataType { override private[spark] def defaultConcreteType: DataType = DoubleType @@ -141,22 +136,19 @@ private[spark] object NumericType extends AbstractDataType { other.isInstanceOf[NumericType] } - private[sql] object IntegralType extends AbstractDataType { override private[sql] def defaultConcreteType: DataType = IntegerType override private[sql] def simpleString: String = "integral" - override private[sql] def acceptsType(other: DataType): Boolean = other.isInstanceOf[IntegralType] + override private[sql] def acceptsType(other: DataType): Boolean = + other.isInstanceOf[IntegralType] } - private[sql] abstract class IntegralType extends NumericType - private[sql] object FractionalType - private[sql] abstract class FractionalType extends NumericType private[sql] object AnyTimestampType extends AbstractDataType with Serializable { diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/ArrayType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/ArrayType.scala index e5af472d90e25..fc32248b4baf3 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/ArrayType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/ArrayType.scala @@ -29,12 +29,14 @@ import org.apache.spark.sql.catalyst.util.StringConcat */ @Stable object ArrayType extends AbstractDataType { + /** * Construct a [[ArrayType]] object with the given element type. The `containsNull` is true. */ def apply(elementType: DataType): ArrayType = ArrayType(elementType, containsNull = true) - override private[sql] def defaultConcreteType: DataType = ArrayType(NullType, containsNull = true) + override private[sql] def defaultConcreteType: DataType = + ArrayType(NullType, containsNull = true) override private[sql] def acceptsType(other: DataType): Boolean = { other.isInstanceOf[ArrayType] @@ -44,18 +46,19 @@ object ArrayType extends AbstractDataType { } /** - * The data type for collections of multiple values. - * Internally these are represented as columns that contain a ``scala.collection.Seq``. + * The data type for collections of multiple values. Internally these are represented as columns + * that contain a ``scala.collection.Seq``. * * Please use `DataTypes.createArrayType()` to create a specific instance. * - * An [[ArrayType]] object comprises two fields, `elementType: [[DataType]]` and - * `containsNull: Boolean`. - * The field of `elementType` is used to specify the type of array elements. - * The field of `containsNull` is used to specify if the array can have `null` values. + * An [[ArrayType]] object comprises two fields, `elementType: [[DataType]]` and `containsNull: + * Boolean`. The field of `elementType` is used to specify the type of array elements. The field + * of `containsNull` is used to specify if the array can have `null` values. * - * @param elementType The data type of values. - * @param containsNull Indicates if the array can have `null` values + * @param elementType + * The data type of values. + * @param containsNull + * Indicates if the array can have `null` values * * @since 1.3.0 */ @@ -82,8 +85,8 @@ case class ArrayType(elementType: DataType, containsNull: Boolean) extends DataT ("containsNull" -> containsNull) /** - * The default size of a value of the ArrayType is the default size of the element type. - * We assume that there is only 1 element on average in an array. See SPARK-18853. + * The default size of a value of the ArrayType is the default size of the element type. We + * assume that there is only 1 element on average in an array. See SPARK-18853. */ override def defaultSize: Int = 1 * elementType.defaultSize @@ -97,8 +100,8 @@ case class ArrayType(elementType: DataType, containsNull: Boolean) extends DataT ArrayType(elementType.asNullable, containsNull = true) /** - * Returns the same data type but set all nullability fields are true - * (`StructField.nullable`, `ArrayType.containsNull`, and `MapType.valueContainsNull`). + * Returns the same data type but set all nullability fields are true (`StructField.nullable`, + * `ArrayType.containsNull`, and `MapType.valueContainsNull`). * * @since 4.0.0 */ diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/BinaryType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/BinaryType.scala index c280f66f943aa..20bfd9bf5684f 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/BinaryType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/BinaryType.scala @@ -20,11 +20,12 @@ package org.apache.spark.sql.types import org.apache.spark.annotation.Stable /** - * The data type representing `Array[Byte]` values. - * Please use the singleton `DataTypes.BinaryType`. + * The data type representing `Array[Byte]` values. Please use the singleton + * `DataTypes.BinaryType`. */ @Stable -class BinaryType private() extends AtomicType { +class BinaryType private () extends AtomicType { + /** * The default size of a value of the BinaryType is 100 bytes. */ diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/BooleanType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/BooleanType.scala index 836c41a996ac4..090c56eaf9af7 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/BooleanType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/BooleanType.scala @@ -25,7 +25,8 @@ import org.apache.spark.annotation.Stable * @since 1.3.0 */ @Stable -class BooleanType private() extends AtomicType { +class BooleanType private () extends AtomicType { + /** * The default size of a value of the BooleanType is 1 byte. */ diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/ByteType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/ByteType.scala index 546ac02f2639a..4a27a00dacb8a 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/ByteType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/ByteType.scala @@ -25,7 +25,8 @@ import org.apache.spark.annotation.Stable * @since 1.3.0 */ @Stable -class ByteType private() extends IntegralType { +class ByteType private () extends IntegralType { + /** * The default size of a value of the ByteType is 1 byte. */ @@ -36,7 +37,6 @@ class ByteType private() extends IntegralType { private[spark] override def asNullable: ByteType = this } - /** * @since 1.3.0 */ diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/CalendarIntervalType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/CalendarIntervalType.scala index d506a1521e183..f6b6256db0417 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/CalendarIntervalType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/CalendarIntervalType.scala @@ -21,19 +21,19 @@ import org.apache.spark.annotation.Stable /** * The data type representing calendar intervals. The calendar interval is stored internally in - * three components: - * an integer value representing the number of `months` in this interval, - * an integer value representing the number of `days` in this interval, - * a long value representing the number of `microseconds` in this interval. + * three components: an integer value representing the number of `months` in this interval, an + * integer value representing the number of `days` in this interval, a long value representing the + * number of `microseconds` in this interval. * * Please use the singleton `DataTypes.CalendarIntervalType` to refer the type. * - * @note Calendar intervals are not comparable. + * @note + * Calendar intervals are not comparable. * * @since 1.5.0 */ @Stable -class CalendarIntervalType private() extends DataType { +class CalendarIntervalType private () extends DataType { override def defaultSize: Int = 16 diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/DataType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/DataType.scala index 277d5c9458d6f..008c9cd07076c 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/DataType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/DataType.scala @@ -49,6 +49,7 @@ import org.apache.spark.util.SparkClassUtils @JsonSerialize(using = classOf[DataTypeJsonSerializer]) @JsonDeserialize(using = classOf[DataTypeJsonDeserializer]) abstract class DataType extends AbstractDataType { + /** * The default size of a value of this data type, used internally for size estimation. */ @@ -57,7 +58,9 @@ abstract class DataType extends AbstractDataType { /** Name of the type used in JSON serialization. */ def typeName: String = { this.getClass.getSimpleName - .stripSuffix("$").stripSuffix("Type").stripSuffix("UDT") + .stripSuffix("$") + .stripSuffix("Type") + .stripSuffix("UDT") .toLowerCase(Locale.ROOT) } @@ -92,8 +95,8 @@ abstract class DataType extends AbstractDataType { } /** - * Returns the same data type but set all nullability fields are true - * (`StructField.nullable`, `ArrayType.containsNull`, and `MapType.valueContainsNull`). + * Returns the same data type but set all nullability fields are true (`StructField.nullable`, + * `ArrayType.containsNull`, and `MapType.valueContainsNull`). */ private[spark] def asNullable: DataType @@ -107,7 +110,6 @@ abstract class DataType extends AbstractDataType { override private[sql] def acceptsType(other: DataType): Boolean = sameType(other) } - /** * @since 1.3.0 */ @@ -128,14 +130,18 @@ object DataType { } /** - * Parses data type from a string with schema. It calls `parser` for `schema`. - * If it fails, calls `fallbackParser`. If the fallback function fails too, combines error message - * from `parser` and `fallbackParser`. + * Parses data type from a string with schema. It calls `parser` for `schema`. If it fails, + * calls `fallbackParser`. If the fallback function fails too, combines error message from + * `parser` and `fallbackParser`. * - * @param schema The schema string to parse by `parser` or `fallbackParser`. - * @param parser The function that should be invoke firstly. - * @param fallbackParser The function that is called when `parser` fails. - * @return The data type parsed from the `schema` schema. + * @param schema + * The schema string to parse by `parser` or `fallbackParser`. + * @param parser + * The function that should be invoke firstly. + * @param fallbackParser + * The function that is called when `parser` fails. + * @return + * The data type parsed from the `schema` schema. */ def parseTypeWithFallback( schema: String, @@ -161,8 +167,20 @@ object DataType { def fromJson(json: String): DataType = parseDataType(parse(json)) private val otherTypes = { - Seq(NullType, DateType, TimestampType, BinaryType, IntegerType, BooleanType, LongType, - DoubleType, FloatType, ShortType, ByteType, StringType, CalendarIntervalType, + Seq( + NullType, + DateType, + TimestampType, + BinaryType, + IntegerType, + BooleanType, + LongType, + DoubleType, + FloatType, + ShortType, + ByteType, + StringType, + CalendarIntervalType, DayTimeIntervalType(DAY), DayTimeIntervalType(DAY, HOUR), DayTimeIntervalType(DAY, MINUTE), @@ -178,7 +196,8 @@ object DataType { YearMonthIntervalType(YEAR, MONTH), TimestampNTZType, VariantType) - .map(t => t.typeName -> t).toMap + .map(t => t.typeName -> t) + .toMap } /** Given the string representation of a type, return its DataType */ @@ -191,11 +210,12 @@ object DataType { // For backwards compatibility, previously the type name of NullType is "null" case "null" => NullType case "timestamp_ltz" => TimestampType - case other => otherTypes.getOrElse( - other, - throw new SparkIllegalArgumentException( - errorClass = "INVALID_JSON_DATA_TYPE", - messageParameters = Map("invalidType" -> name))) + case other => + otherTypes.getOrElse( + other, + throw new SparkIllegalArgumentException( + errorClass = "INVALID_JSON_DATA_TYPE", + messageParameters = Map("invalidType" -> name))) } } @@ -220,56 +240,55 @@ object DataType { } case JSortedObject( - ("containsNull", JBool(n)), - ("elementType", t: JValue), - ("type", JString("array"))) => + ("containsNull", JBool(n)), + ("elementType", t: JValue), + ("type", JString("array"))) => assertValidTypeForCollations(fieldPath, "array", collationsMap) val elementType = parseDataType(t, appendFieldToPath(fieldPath, "element"), collationsMap) ArrayType(elementType, n) case JSortedObject( - ("keyType", k: JValue), - ("type", JString("map")), - ("valueContainsNull", JBool(n)), - ("valueType", v: JValue)) => + ("keyType", k: JValue), + ("type", JString("map")), + ("valueContainsNull", JBool(n)), + ("valueType", v: JValue)) => assertValidTypeForCollations(fieldPath, "map", collationsMap) val keyType = parseDataType(k, appendFieldToPath(fieldPath, "key"), collationsMap) val valueType = parseDataType(v, appendFieldToPath(fieldPath, "value"), collationsMap) MapType(keyType, valueType, n) - case JSortedObject( - ("fields", JArray(fields)), - ("type", JString("struct"))) => + case JSortedObject(("fields", JArray(fields)), ("type", JString("struct"))) => assertValidTypeForCollations(fieldPath, "struct", collationsMap) StructType(fields.map(parseStructField)) // Scala/Java UDT case JSortedObject( - ("class", JString(udtClass)), - ("pyClass", _), - ("sqlType", _), - ("type", JString("udt"))) => + ("class", JString(udtClass)), + ("pyClass", _), + ("sqlType", _), + ("type", JString("udt"))) => SparkClassUtils.classForName[UserDefinedType[_]](udtClass).getConstructor().newInstance() // Python UDT case JSortedObject( - ("pyClass", JString(pyClass)), - ("serializedClass", JString(serialized)), - ("sqlType", v: JValue), - ("type", JString("udt"))) => - new PythonUserDefinedType(parseDataType(v), pyClass, serialized) - - case other => throw new SparkIllegalArgumentException( - errorClass = "INVALID_JSON_DATA_TYPE", - messageParameters = Map("invalidType" -> compact(render(other)))) + ("pyClass", JString(pyClass)), + ("serializedClass", JString(serialized)), + ("sqlType", v: JValue), + ("type", JString("udt"))) => + new PythonUserDefinedType(parseDataType(v), pyClass, serialized) + + case other => + throw new SparkIllegalArgumentException( + errorClass = "INVALID_JSON_DATA_TYPE", + messageParameters = Map("invalidType" -> compact(render(other)))) } private def parseStructField(json: JValue): StructField = json match { case JSortedObject( - ("metadata", JObject(metadataFields)), - ("name", JString(name)), - ("nullable", JBool(nullable)), - ("type", dataType: JValue)) => + ("metadata", JObject(metadataFields)), + ("name", JString(name)), + ("nullable", JBool(nullable)), + ("type", dataType: JValue)) => val collationsMap = getCollationsMap(metadataFields) val metadataWithoutCollations = JObject(metadataFields.filterNot(_._1 == COLLATIONS_METADATA_KEY)) @@ -280,18 +299,17 @@ object DataType { Metadata.fromJObject(metadataWithoutCollations)) // Support reading schema when 'metadata' is missing. case JSortedObject( - ("name", JString(name)), - ("nullable", JBool(nullable)), - ("type", dataType: JValue)) => + ("name", JString(name)), + ("nullable", JBool(nullable)), + ("type", dataType: JValue)) => StructField(name, parseDataType(dataType), nullable) // Support reading schema when 'nullable' is missing. - case JSortedObject( - ("name", JString(name)), - ("type", dataType: JValue)) => + case JSortedObject(("name", JString(name)), ("type", dataType: JValue)) => StructField(name, parseDataType(dataType)) - case other => throw new SparkIllegalArgumentException( - errorClass = "INVALID_JSON_DATA_TYPE", - messageParameters = Map("invalidType" -> compact(render(other)))) + case other => + throw new SparkIllegalArgumentException( + errorClass = "INVALID_JSON_DATA_TYPE", + messageParameters = Map("invalidType" -> compact(render(other)))) } private def assertValidTypeForCollations( @@ -319,13 +337,12 @@ object DataType { val collationsJsonOpt = metadataFields.find(_._1 == COLLATIONS_METADATA_KEY).map(_._2) collationsJsonOpt match { case Some(JObject(fields)) => - fields.collect { - case (fieldPath, JString(collation)) => - collation.split("\\.", 2) match { - case Array(provider: String, collationName: String) => - CollationFactory.assertValidProvider(provider) - fieldPath -> collationName - } + fields.collect { case (fieldPath, JString(collation)) => + collation.split("\\.", 2) match { + case Array(provider: String, collationName: String) => + CollationFactory.assertValidProvider(provider) + fieldPath -> collationName + } }.toMap case _ => Map.empty @@ -356,15 +373,15 @@ object DataType { * Compares two types, ignoring compatible nullability of ArrayType, MapType, StructType. * * Compatible nullability is defined as follows: - * - If `from` and `to` are ArrayTypes, `from` has a compatible nullability with `to` - * if and only if `to.containsNull` is true, or both of `from.containsNull` and - * `to.containsNull` are false. - * - If `from` and `to` are MapTypes, `from` has a compatible nullability with `to` - * if and only if `to.valueContainsNull` is true, or both of `from.valueContainsNull` and - * `to.valueContainsNull` are false. - * - If `from` and `to` are StructTypes, `from` has a compatible nullability with `to` - * if and only if for all every pair of fields, `to.nullable` is true, or both - * of `fromField.nullable` and `toField.nullable` are false. + * - If `from` and `to` are ArrayTypes, `from` has a compatible nullability with `to` if and + * only if `to.containsNull` is true, or both of `from.containsNull` and `to.containsNull` + * are false. + * - If `from` and `to` are MapTypes, `from` has a compatible nullability with `to` if and + * only if `to.valueContainsNull` is true, or both of `from.valueContainsNull` and + * `to.valueContainsNull` are false. + * - If `from` and `to` are StructTypes, `from` has a compatible nullability with `to` if and + * only if for all every pair of fields, `to.nullable` is true, or both of + * `fromField.nullable` and `toField.nullable` are false. */ private[sql] def equalsIgnoreCompatibleNullability(from: DataType, to: DataType): Boolean = { equalsIgnoreCompatibleNullability(from, to, ignoreName = false) @@ -375,15 +392,15 @@ object DataType { * also the field name. It compares based on the position. * * Compatible nullability is defined as follows: - * - If `from` and `to` are ArrayTypes, `from` has a compatible nullability with `to` - * if and only if `to.containsNull` is true, or both of `from.containsNull` and - * `to.containsNull` are false. - * - If `from` and `to` are MapTypes, `from` has a compatible nullability with `to` - * if and only if `to.valueContainsNull` is true, or both of `from.valueContainsNull` and - * `to.valueContainsNull` are false. - * - If `from` and `to` are StructTypes, `from` has a compatible nullability with `to` - * if and only if for all every pair of fields, `to.nullable` is true, or both - * of `fromField.nullable` and `toField.nullable` are false. + * - If `from` and `to` are ArrayTypes, `from` has a compatible nullability with `to` if and + * only if `to.containsNull` is true, or both of `from.containsNull` and `to.containsNull` + * are false. + * - If `from` and `to` are MapTypes, `from` has a compatible nullability with `to` if and + * only if `to.valueContainsNull` is true, or both of `from.valueContainsNull` and + * `to.valueContainsNull` are false. + * - If `from` and `to` are StructTypes, `from` has a compatible nullability with `to` if and + * only if for all every pair of fields, `to.nullable` is true, or both of + * `fromField.nullable` and `toField.nullable` are false. */ private[sql] def equalsIgnoreNameAndCompatibleNullability( from: DataType, @@ -401,16 +418,16 @@ object DataType { case (MapType(fromKey, fromValue, fn), MapType(toKey, toValue, tn)) => (tn || !fn) && - equalsIgnoreCompatibleNullability(fromKey, toKey, ignoreName) && - equalsIgnoreCompatibleNullability(fromValue, toValue, ignoreName) + equalsIgnoreCompatibleNullability(fromKey, toKey, ignoreName) && + equalsIgnoreCompatibleNullability(fromValue, toValue, ignoreName) case (StructType(fromFields), StructType(toFields)) => fromFields.length == toFields.length && - fromFields.zip(toFields).forall { case (fromField, toField) => - (ignoreName || fromField.name == toField.name) && - (toField.nullable || !fromField.nullable) && - equalsIgnoreCompatibleNullability(fromField.dataType, toField.dataType, ignoreName) - } + fromFields.zip(toFields).forall { case (fromField, toField) => + (ignoreName || fromField.name == toField.name) && + (toField.nullable || !fromField.nullable) && + equalsIgnoreCompatibleNullability(fromField.dataType, toField.dataType, ignoreName) + } case (fromDataType, toDataType) => fromDataType == toDataType } @@ -420,42 +437,42 @@ object DataType { * Check if `from` is equal to `to` type except for collations, which are checked to be * compatible so that data of type `from` can be interpreted as of type `to`. */ - private[sql] def equalsIgnoreCompatibleCollation( - from: DataType, - to: DataType): Boolean = { + private[sql] def equalsIgnoreCompatibleCollation(from: DataType, to: DataType): Boolean = { (from, to) match { // String types with possibly different collations are compatible. case (_: StringType, _: StringType) => true case (ArrayType(fromElement, fromContainsNull), ArrayType(toElement, toContainsNull)) => (fromContainsNull == toContainsNull) && - equalsIgnoreCompatibleCollation(fromElement, toElement) + equalsIgnoreCompatibleCollation(fromElement, toElement) - case (MapType(fromKey, fromValue, fromContainsNull), - MapType(toKey, toValue, toContainsNull)) => + case ( + MapType(fromKey, fromValue, fromContainsNull), + MapType(toKey, toValue, toContainsNull)) => fromContainsNull == toContainsNull && - // Map keys cannot change collation. - fromKey == toKey && - equalsIgnoreCompatibleCollation(fromValue, toValue) + // Map keys cannot change collation. + fromKey == toKey && + equalsIgnoreCompatibleCollation(fromValue, toValue) case (StructType(fromFields), StructType(toFields)) => fromFields.length == toFields.length && - fromFields.zip(toFields).forall { case (fromField, toField) => - fromField.name == toField.name && - fromField.nullable == toField.nullable && - fromField.metadata == toField.metadata && - equalsIgnoreCompatibleCollation(fromField.dataType, toField.dataType) - } + fromFields.zip(toFields).forall { case (fromField, toField) => + fromField.name == toField.name && + fromField.nullable == toField.nullable && + fromField.metadata == toField.metadata && + equalsIgnoreCompatibleCollation(fromField.dataType, toField.dataType) + } case (fromDataType, toDataType) => fromDataType == toDataType } } /** - * Returns true if the two data types share the same "shape", i.e. the types - * are the same, but the field names don't need to be the same. + * Returns true if the two data types share the same "shape", i.e. the types are the same, but + * the field names don't need to be the same. * - * @param ignoreNullability whether to ignore nullability when comparing the types + * @param ignoreNullability + * whether to ignore nullability when comparing the types */ def equalsStructurally( from: DataType, @@ -464,20 +481,21 @@ object DataType { (from, to) match { case (left: ArrayType, right: ArrayType) => equalsStructurally(left.elementType, right.elementType, ignoreNullability) && - (ignoreNullability || left.containsNull == right.containsNull) + (ignoreNullability || left.containsNull == right.containsNull) case (left: MapType, right: MapType) => equalsStructurally(left.keyType, right.keyType, ignoreNullability) && - equalsStructurally(left.valueType, right.valueType, ignoreNullability) && - (ignoreNullability || left.valueContainsNull == right.valueContainsNull) + equalsStructurally(left.valueType, right.valueType, ignoreNullability) && + (ignoreNullability || left.valueContainsNull == right.valueContainsNull) case (StructType(fromFields), StructType(toFields)) => fromFields.length == toFields.length && - fromFields.zip(toFields) - .forall { case (l, r) => - equalsStructurally(l.dataType, r.dataType, ignoreNullability) && - (ignoreNullability || l.nullable == r.nullable) - } + fromFields + .zip(toFields) + .forall { case (l, r) => + equalsStructurally(l.dataType, r.dataType, ignoreNullability) && + (ignoreNullability || l.nullable == r.nullable) + } case (fromDataType, toDataType) => fromDataType == toDataType } @@ -496,14 +514,15 @@ object DataType { case (left: MapType, right: MapType) => equalsStructurallyByName(left.keyType, right.keyType, resolver) && - equalsStructurallyByName(left.valueType, right.valueType, resolver) + equalsStructurallyByName(left.valueType, right.valueType, resolver) case (StructType(fromFields), StructType(toFields)) => fromFields.length == toFields.length && - fromFields.zip(toFields) - .forall { case (l, r) => - resolver(l.name, r.name) && equalsStructurallyByName(l.dataType, r.dataType, resolver) - } + fromFields + .zip(toFields) + .forall { case (l, r) => + resolver(l.name, r.name) && equalsStructurallyByName(l.dataType, r.dataType, resolver) + } case _ => true } @@ -518,12 +537,12 @@ object DataType { equalsIgnoreNullability(leftElementType, rightElementType) case (MapType(leftKeyType, leftValueType, _), MapType(rightKeyType, rightValueType, _)) => equalsIgnoreNullability(leftKeyType, rightKeyType) && - equalsIgnoreNullability(leftValueType, rightValueType) + equalsIgnoreNullability(leftValueType, rightValueType) case (StructType(leftFields), StructType(rightFields)) => leftFields.length == rightFields.length && - leftFields.zip(rightFields).forall { case (l, r) => - l.name == r.name && equalsIgnoreNullability(l.dataType, r.dataType) - } + leftFields.zip(rightFields).forall { case (l, r) => + l.name == r.name && equalsIgnoreNullability(l.dataType, r.dataType) + } case (l, r) => l == r } } @@ -539,14 +558,14 @@ object DataType { case (MapType(fromKey, fromValue, _), MapType(toKey, toValue, _)) => equalsIgnoreCaseAndNullability(fromKey, toKey) && - equalsIgnoreCaseAndNullability(fromValue, toValue) + equalsIgnoreCaseAndNullability(fromValue, toValue) case (StructType(fromFields), StructType(toFields)) => fromFields.length == toFields.length && - fromFields.zip(toFields).forall { case (l, r) => - l.name.equalsIgnoreCase(r.name) && - equalsIgnoreCaseAndNullability(l.dataType, r.dataType) - } + fromFields.zip(toFields).forall { case (l, r) => + l.name.equalsIgnoreCase(r.name) && + equalsIgnoreCaseAndNullability(l.dataType, r.dataType) + } case (fromDataType, toDataType) => fromDataType == toDataType } diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/DateType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/DateType.scala index d37ebbcdad727..402c4c0d95298 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/DateType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/DateType.scala @@ -20,14 +20,15 @@ package org.apache.spark.sql.types import org.apache.spark.annotation.Stable /** - * The date type represents a valid date in the proleptic Gregorian calendar. - * Valid range is [0001-01-01, 9999-12-31]. + * The date type represents a valid date in the proleptic Gregorian calendar. Valid range is + * [0001-01-01, 9999-12-31]. * * Please use the singleton `DataTypes.DateType` to refer the type. * @since 1.3.0 */ @Stable -class DateType private() extends DatetimeType { +class DateType private () extends DatetimeType { + /** * The default size of a value of the DateType is 4 bytes. */ @@ -37,10 +38,10 @@ class DateType private() extends DatetimeType { } /** - * The companion case object and the DateType class is separated so the companion object - * also subclasses the class. Otherwise, the companion object would be of type "DateType$" - * in byte code. The DateType class is defined with a private constructor so its companion - * object is the only possible instantiation. + * The companion case object and the DateType class is separated so the companion object also + * subclasses the class. Otherwise, the companion object would be of type "DateType$" in byte + * code. The DateType class is defined with a private constructor so its companion object is the + * only possible instantiation. * * @since 1.3.0 */ diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/DayTimeIntervalType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/DayTimeIntervalType.scala index a1d014fa51f1c..90d6d7c29a6ba 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/DayTimeIntervalType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/DayTimeIntervalType.scala @@ -22,8 +22,8 @@ import org.apache.spark.sql.errors.DataTypeErrors import org.apache.spark.sql.types.DayTimeIntervalType.fieldToString /** - * The type represents day-time intervals of the SQL standard. A day-time interval is made up - * of a contiguous subset of the following fields: + * The type represents day-time intervals of the SQL standard. A day-time interval is made up of a + * contiguous subset of the following fields: * - SECOND, seconds within minutes and possibly fractions of a second [0..59.999999], * - MINUTE, minutes within hours [0..59], * - HOUR, hours within days [0..23], @@ -31,18 +31,21 @@ import org.apache.spark.sql.types.DayTimeIntervalType.fieldToString * * `DayTimeIntervalType` represents positive as well as negative day-time intervals. * - * @param startField The leftmost field which the type comprises of. Valid values: - * 0 (DAY), 1 (HOUR), 2 (MINUTE), 3 (SECOND). - * @param endField The rightmost field which the type comprises of. Valid values: - * 0 (DAY), 1 (HOUR), 2 (MINUTE), 3 (SECOND). + * @param startField + * The leftmost field which the type comprises of. Valid values: 0 (DAY), 1 (HOUR), 2 (MINUTE), + * 3 (SECOND). + * @param endField + * The rightmost field which the type comprises of. Valid values: 0 (DAY), 1 (HOUR), 2 (MINUTE), + * 3 (SECOND). * * @since 3.2.0 */ @Unstable case class DayTimeIntervalType(startField: Byte, endField: Byte) extends AnsiIntervalType { + /** - * The day-time interval type has constant precision. A value of the type always occupies 8 bytes. - * The DAY field is constrained by the upper bound 106751991 to fit to `Long`. + * The day-time interval type has constant precision. A value of the type always occupies 8 + * bytes. The DAY field is constrained by the upper bound 106751991 to fit to `Long`. */ override def defaultSize: Int = 8 diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/Decimal.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/Decimal.scala index 6de8570b1422f..bd94c386ab533 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/Decimal.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/Decimal.scala @@ -31,9 +31,9 @@ import org.apache.spark.unsafe.types.UTF8String * A mutable implementation of BigDecimal that can hold a Long if values are small enough. * * The semantics of the fields are as follows: - * - _precision and _scale represent the SQL precision and scale we are looking for - * - If decimalVal is set, it represents the whole decimal value - * - Otherwise, the decimal value is longVal / (10 ** _scale) + * - _precision and _scale represent the SQL precision and scale we are looking for + * - If decimalVal is set, it represents the whole decimal value + * - Otherwise, the decimal value is longVal / (10 ** _scale) * * Note, for values between -1.0 and 1.0, precision digits are only counted after dot. */ @@ -88,22 +88,22 @@ final class Decimal extends Ordered[Decimal] with Serializable { } /** - * Set this Decimal to the given unscaled Long, with a given precision and scale, - * and return it, or return null if it cannot be set due to overflow. + * Set this Decimal to the given unscaled Long, with a given precision and scale, and return it, + * or return null if it cannot be set due to overflow. */ def setOrNull(unscaled: Long, precision: Int, scale: Int): Decimal = { DecimalType.checkNegativeScale(scale) if (unscaled <= -POW_10(MAX_LONG_DIGITS) || unscaled >= POW_10(MAX_LONG_DIGITS)) { // We can't represent this compactly as a long without risking overflow if (precision < 19) { - return null // Requested precision is too low to represent this value + return null // Requested precision is too low to represent this value } this.decimalVal = BigDecimal(unscaled, scale) this.longVal = 0L } else { val p = POW_10(math.min(precision, MAX_LONG_DIGITS)) if (unscaled <= -p || unscaled >= p) { - return null // Requested precision is too low to represent this value + return null // Requested precision is too low to represent this value } this.decimalVal = null this.longVal = unscaled @@ -126,7 +126,8 @@ final class Decimal extends Ordered[Decimal] with Serializable { "roundedValue" -> decimalVal.toString, "originalValue" -> decimal.toString, "precision" -> precision.toString, - "scale" -> scale.toString), Array.empty) + "scale" -> scale.toString), + Array.empty) } this.longVal = 0L this._precision = precision @@ -160,8 +161,8 @@ final class Decimal extends Ordered[Decimal] with Serializable { } /** - * If the value is not in the range of long, convert it to BigDecimal and - * the precision and scale are based on the converted value. + * If the value is not in the range of long, convert it to BigDecimal and the precision and + * scale are based on the converted value. * * This code avoids BigDecimal object allocation as possible to improve runtime efficiency */ @@ -262,37 +263,47 @@ final class Decimal extends Ordered[Decimal] with Serializable { def toByte: Byte = toLong.toByte /** - * @return the Byte value that is equal to the rounded decimal. - * @throws ArithmeticException if the decimal is too big to fit in Byte type. + * @return + * the Byte value that is equal to the rounded decimal. + * @throws ArithmeticException + * if the decimal is too big to fit in Byte type. */ private[sql] def roundToByte(): Byte = - roundToNumeric[Byte](ByteType, Byte.MaxValue, Byte.MinValue) (_.toByte) (_.toByte) + roundToNumeric[Byte](ByteType, Byte.MaxValue, Byte.MinValue)(_.toByte)(_.toByte) /** - * @return the Short value that is equal to the rounded decimal. - * @throws ArithmeticException if the decimal is too big to fit in Short type. + * @return + * the Short value that is equal to the rounded decimal. + * @throws ArithmeticException + * if the decimal is too big to fit in Short type. */ private[sql] def roundToShort(): Short = - roundToNumeric[Short](ShortType, Short.MaxValue, Short.MinValue) (_.toShort) (_.toShort) + roundToNumeric[Short](ShortType, Short.MaxValue, Short.MinValue)(_.toShort)(_.toShort) /** - * @return the Int value that is equal to the rounded decimal. - * @throws ArithmeticException if the decimal too big to fit in Int type. + * @return + * the Int value that is equal to the rounded decimal. + * @throws ArithmeticException + * if the decimal too big to fit in Int type. */ private[sql] def roundToInt(): Int = - roundToNumeric[Int](IntegerType, Int.MaxValue, Int.MinValue) (_.toInt) (_.toInt) + roundToNumeric[Int](IntegerType, Int.MaxValue, Int.MinValue)(_.toInt)(_.toInt) private def toSqlValue: String = this.toString + "BD" - private def roundToNumeric[T <: AnyVal](integralType: IntegralType, maxValue: Int, minValue: Int) - (f1: Long => T) (f2: Double => T): T = { + private def roundToNumeric[T <: AnyVal]( + integralType: IntegralType, + maxValue: Int, + minValue: Int)(f1: Long => T)(f2: Double => T): T = { if (decimalVal.eq(null)) { val numericVal = f1(actualLongVal) if (actualLongVal == numericVal) { numericVal } else { throw DataTypeErrors.castingCauseOverflowError( - toSqlValue, DecimalType(this.precision, this.scale), integralType) + toSqlValue, + DecimalType(this.precision, this.scale), + integralType) } } else { val doubleVal = decimalVal.toDouble @@ -300,14 +311,18 @@ final class Decimal extends Ordered[Decimal] with Serializable { f2(doubleVal) } else { throw DataTypeErrors.castingCauseOverflowError( - toSqlValue, DecimalType(this.precision, this.scale), integralType) + toSqlValue, + DecimalType(this.precision, this.scale), + integralType) } } } /** - * @return the Long value that is equal to the rounded decimal. - * @throws ArithmeticException if the decimal too big to fit in Long type. + * @return + * the Long value that is equal to the rounded decimal. + * @throws ArithmeticException + * if the decimal too big to fit in Long type. */ private[sql] def roundToLong(): Long = { if (decimalVal.eq(null)) { @@ -321,7 +336,9 @@ final class Decimal extends Ordered[Decimal] with Serializable { } catch { case _: ArithmeticException => throw DataTypeErrors.castingCauseOverflowError( - toSqlValue, DecimalType(this.precision, this.scale), LongType) + toSqlValue, + DecimalType(this.precision, this.scale), + LongType) } } } @@ -329,7 +346,8 @@ final class Decimal extends Ordered[Decimal] with Serializable { /** * Update precision and scale while keeping our value the same, and return true if successful. * - * @return true if successful, false if overflow would occur + * @return + * true if successful, false if overflow would occur */ def changePrecision(precision: Int, scale: Int): Boolean = { changePrecision(precision, scale, ROUND_HALF_UP) @@ -338,8 +356,9 @@ final class Decimal extends Ordered[Decimal] with Serializable { /** * Create new `Decimal` with given precision and scale. * - * @return a non-null `Decimal` value if successful. Otherwise, if `nullOnOverflow` is true, null - * is returned; if `nullOnOverflow` is false, an `ArithmeticException` is thrown. + * @return + * a non-null `Decimal` value if successful. Otherwise, if `nullOnOverflow` is true, null is + * returned; if `nullOnOverflow` is false, an `ArithmeticException` is thrown. */ private[sql] def toPrecision( precision: Int, @@ -354,8 +373,7 @@ final class Decimal extends Ordered[Decimal] with Serializable { if (nullOnOverflow) { null } else { - throw DataTypeErrors.cannotChangeDecimalPrecisionError( - this, precision, scale, context) + throw DataTypeErrors.cannotChangeDecimalPrecisionError(this, precision, scale, context) } } } @@ -363,7 +381,8 @@ final class Decimal extends Ordered[Decimal] with Serializable { /** * Update precision and scale while keeping our value the same, and return true if successful. * - * @return true if successful, false if overflow would occur + * @return + * true if successful, false if overflow would occur */ private[sql] def changePrecision( precision: Int, @@ -482,7 +501,7 @@ final class Decimal extends Ordered[Decimal] with Serializable { // ------------------------------------------------------------------------ // e1 + e2 max(s1, s2) + max(p1-s1, p2-s2) + 1 max(s1, s2) // e1 - e2 max(s1, s2) + max(p1-s1, p2-s2) + 1 max(s1, s2) - def + (that: Decimal): Decimal = { + def +(that: Decimal): Decimal = { if (decimalVal.eq(null) && that.decimalVal.eq(null) && scale == that.scale) { Decimal(longVal + that.longVal, Math.max(precision, that.precision) + 1, scale) } else { @@ -490,7 +509,7 @@ final class Decimal extends Ordered[Decimal] with Serializable { } } - def - (that: Decimal): Decimal = { + def -(that: Decimal): Decimal = { if (decimalVal.eq(null) && that.decimalVal.eq(null) && scale == that.scale) { Decimal(longVal - that.longVal, Math.max(precision, that.precision) + 1, scale) } else { @@ -499,14 +518,19 @@ final class Decimal extends Ordered[Decimal] with Serializable { } // TypeCoercion will take care of the precision, scale of result - def * (that: Decimal): Decimal = + def *(that: Decimal): Decimal = Decimal(toJavaBigDecimal.multiply(that.toJavaBigDecimal, MATH_CONTEXT)) - def / (that: Decimal): Decimal = - if (that.isZero) null else Decimal(toJavaBigDecimal.divide(that.toJavaBigDecimal, - DecimalType.MAX_SCALE + 1, MATH_CONTEXT.getRoundingMode)) + def /(that: Decimal): Decimal = + if (that.isZero) { + null + } else { + Decimal( + toJavaBigDecimal + .divide(that.toJavaBigDecimal, DecimalType.MAX_SCALE + 1, MATH_CONTEXT.getRoundingMode)) + } - def % (that: Decimal): Decimal = + def %(that: Decimal): Decimal = if (that.isZero) null else Decimal(toJavaBigDecimal.remainder(that.toJavaBigDecimal, MATH_CONTEXT)) @@ -526,12 +550,14 @@ final class Decimal extends Ordered[Decimal] with Serializable { def abs: Decimal = if (this < Decimal.ZERO) this.unary_- else this - def floor: Decimal = if (scale == 0) this else { + def floor: Decimal = if (scale == 0) this + else { val newPrecision = DecimalType.bounded(precision - scale + 1, 0).precision toPrecision(newPrecision, 0, ROUND_FLOOR, nullOnOverflow = false) } - def ceil: Decimal = if (scale == 0) this else { + def ceil: Decimal = if (scale == 0) this + else { val newPrecision = DecimalType.bounded(precision - scale + 1, 0).precision toPrecision(newPrecision, 0, ROUND_CEILING, nullOnOverflow = false) } @@ -612,7 +638,7 @@ object Decimal { // We fast fail because constructing a very large JavaBigDecimal to Decimal is very slow. // For example: Decimal("6.0790316E+25569151") if (numDigitsInIntegralPart(bigDecimal) > DecimalType.MAX_PRECISION && - !SqlApiConf.get.allowNegativeScaleOfDecimalEnabled) { + !SqlApiConf.get.allowNegativeScaleOfDecimalEnabled) { null } else { Decimal(bigDecimal) @@ -632,7 +658,7 @@ object Decimal { // We fast fail because constructing a very large JavaBigDecimal to Decimal is very slow. // For example: Decimal("6.0790316E+25569151") if (numDigitsInIntegralPart(bigDecimal) > DecimalType.MAX_PRECISION && - !SqlApiConf.get.allowNegativeScaleOfDecimalEnabled) { + !SqlApiConf.get.allowNegativeScaleOfDecimalEnabled) { throw DataTypeErrors.outOfDecimalTypeRangeError(str) } else { Decimal(bigDecimal) @@ -657,16 +683,18 @@ object Decimal { // Max precision of a decimal value stored in `numBytes` bytes def maxPrecisionForBytes(numBytes: Int): Int = { - Math.round( // convert double to long - Math.floor(Math.log10( // number of base-10 digits - Math.pow(2, 8 * numBytes - 1) - 1))) // max value stored in numBytes + Math + .round( // convert double to long + Math.floor(Math.log10( // number of base-10 digits + Math.pow(2, 8 * numBytes - 1) - 1)) + ) // max value stored in numBytes .asInstanceOf[Int] } // Returns the minimum number of bytes needed to store a decimal with a given `precision`. lazy val minBytesForPrecision = Array.tabulate[Int](39)(computeMinBytesForPrecision) - private def computeMinBytesForPrecision(precision : Int) : Int = { + private def computeMinBytesForPrecision(precision: Int): Int = { var numBytes = 1 while (math.pow(2.0, 8 * numBytes - 1) < math.pow(10.0, precision)) { numBytes += 1 diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/DecimalType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/DecimalType.scala index 9de34d0b3bc16..bff483cefda91 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/DecimalType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/DecimalType.scala @@ -26,9 +26,8 @@ import org.apache.spark.sql.errors.DataTypeErrors import org.apache.spark.sql.internal.SqlApiConf /** - * The data type representing `java.math.BigDecimal` values. - * A Decimal that must have fixed precision (the maximum number of digits) and scale (the number - * of digits on right side of dot). + * The data type representing `java.math.BigDecimal` values. A Decimal that must have fixed + * precision (the maximum number of digits) and scale (the number of digits on right side of dot). * * The precision can be up to 38, scale can also be up to 38 (less or equal to precision). * @@ -49,7 +48,8 @@ case class DecimalType(precision: Int, scale: Int) extends FractionalType { if (precision > DecimalType.MAX_PRECISION) { throw DataTypeErrors.decimalPrecisionExceedsMaxPrecisionError( - precision, DecimalType.MAX_PRECISION) + precision, + DecimalType.MAX_PRECISION) } // default constructor for Java @@ -63,8 +63,8 @@ case class DecimalType(precision: Int, scale: Int) extends FractionalType { override def sql: String = typeName.toUpperCase(Locale.ROOT) /** - * Returns whether this DecimalType is wider than `other`. If yes, it means `other` - * can be casted into `this` safely without losing any precision or range. + * Returns whether this DecimalType is wider than `other`. If yes, it means `other` can be + * casted into `this` safely without losing any precision or range. */ private[sql] def isWiderThan(other: DataType): Boolean = isWiderThanInternal(other) @@ -78,8 +78,8 @@ case class DecimalType(precision: Int, scale: Int) extends FractionalType { } /** - * Returns whether this DecimalType is tighter than `other`. If yes, it means `this` - * can be casted into `other` safely without losing any precision or range. + * Returns whether this DecimalType is tighter than `other`. If yes, it means `this` can be + * casted into `other` safely without losing any precision or range. */ private[sql] def isTighterThan(other: DataType): Boolean = other match { case dt: DecimalType => @@ -94,8 +94,8 @@ case class DecimalType(precision: Int, scale: Int) extends FractionalType { } /** - * The default size of a value of the DecimalType is 8 bytes when precision is at most 18, - * and 16 bytes otherwise. + * The default size of a value of the DecimalType is 8 bytes when precision is at most 18, and + * 16 bytes otherwise. */ override def defaultSize: Int = if (precision <= Decimal.MAX_LONG_DIGITS) 8 else 16 @@ -104,7 +104,6 @@ case class DecimalType(precision: Int, scale: Int) extends FractionalType { private[spark] override def asNullable: DecimalType = this } - /** * Extra factory methods and pattern matchers for Decimals. * @@ -167,10 +166,11 @@ object DecimalType extends AbstractDataType { /** * Scale adjustment implementation is based on Hive's one, which is itself inspired to * SQLServer's one. In particular, when a result precision is greater than - * {@link #MAX_PRECISION}, the corresponding scale is reduced to prevent the integral part of a + * {@link #MAX_PRECISION} , the corresponding scale is reduced to prevent the integral part of a * result from being truncated. * - * This method is used only when `spark.sql.decimalOperations.allowPrecisionLoss` is set to true. + * This method is used only when `spark.sql.decimalOperations.allowPrecisionLoss` is set to + * true. */ private[sql] def adjustPrecisionScale(precision: Int, scale: Int): DecimalType = { // Assumptions: diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/DoubleType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/DoubleType.scala index bc0ed725cf266..873f0c237c6c4 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/DoubleType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/DoubleType.scala @@ -27,7 +27,8 @@ import org.apache.spark.annotation.Stable * @since 1.3.0 */ @Stable -class DoubleType private() extends FractionalType { +class DoubleType private () extends FractionalType { + /** * The default size of a value of the DoubleType is 8 bytes. */ diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/FloatType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/FloatType.scala index 8b54f830d48a3..df4b03cd42bd4 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/FloatType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/FloatType.scala @@ -27,7 +27,8 @@ import org.apache.spark.annotation.Stable * @since 1.3.0 */ @Stable -class FloatType private() extends FractionalType { +class FloatType private () extends FractionalType { + /** * The default size of a value of the FloatType is 4 bytes. */ @@ -36,7 +37,6 @@ class FloatType private() extends FractionalType { private[spark] override def asNullable: FloatType = this } - /** * @since 1.3.0 */ diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/IntegerType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/IntegerType.scala index b26a555c9b572..dc4727cb1215b 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/IntegerType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/IntegerType.scala @@ -25,7 +25,8 @@ import org.apache.spark.annotation.Stable * @since 1.3.0 */ @Stable -class IntegerType private() extends IntegralType { +class IntegerType private () extends IntegralType { + /** * The default size of a value of the IntegerType is 4 bytes. */ diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/LongType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/LongType.scala index 87ebacfe9ce88..f65c4c70acd27 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/LongType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/LongType.scala @@ -25,7 +25,8 @@ import org.apache.spark.annotation.Stable * @since 1.3.0 */ @Stable -class LongType private() extends IntegralType { +class LongType private () extends IntegralType { + /** * The default size of a value of the LongType is 8 bytes. */ diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/MapType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/MapType.scala index dba870466fc1c..1dfb9aaf9e29b 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/MapType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/MapType.scala @@ -28,15 +28,16 @@ import org.apache.spark.sql.catalyst.util.StringConcat * * Please use `DataTypes.createMapType()` to create a specific instance. * - * @param keyType The data type of map keys. - * @param valueType The data type of map values. - * @param valueContainsNull Indicates if map values have `null` values. + * @param keyType + * The data type of map keys. + * @param valueType + * The data type of map values. + * @param valueContainsNull + * Indicates if map values have `null` values. */ @Stable -case class MapType( - keyType: DataType, - valueType: DataType, - valueContainsNull: Boolean) extends DataType { +case class MapType(keyType: DataType, valueType: DataType, valueContainsNull: Boolean) + extends DataType { /** No-arg constructor for kryo. */ def this() = this(null, null, false) @@ -48,8 +49,9 @@ case class MapType( if (maxDepth > 0) { stringConcat.append(s"$prefix-- key: ${keyType.typeName}\n") DataType.buildFormattedString(keyType, s"$prefix |", stringConcat, maxDepth) - stringConcat.append(s"$prefix-- value: ${valueType.typeName} " + - s"(valueContainsNull = $valueContainsNull)\n") + stringConcat.append( + s"$prefix-- value: ${valueType.typeName} " + + s"(valueContainsNull = $valueContainsNull)\n") DataType.buildFormattedString(valueType, s"$prefix |", stringConcat, maxDepth) } } @@ -61,9 +63,9 @@ case class MapType( ("valueContainsNull" -> valueContainsNull) /** - * The default size of a value of the MapType is - * (the default size of the key type + the default size of the value type). - * We assume that there is only 1 element on average in a map. See SPARK-18853. + * The default size of a value of the MapType is (the default size of the key type + the default + * size of the value type). We assume that there is only 1 element on average in a map. See + * SPARK-18853. */ override def defaultSize: Int = 1 * (keyType.defaultSize + valueType.defaultSize) @@ -77,8 +79,8 @@ case class MapType( MapType(keyType.asNullable, valueType.asNullable, valueContainsNull = true) /** - * Returns the same data type but set all nullability fields are true - * (`StructField.nullable`, `ArrayType.containsNull`, and `MapType.valueContainsNull`). + * Returns the same data type but set all nullability fields are true (`StructField.nullable`, + * `ArrayType.containsNull`, and `MapType.valueContainsNull`). * * @since 4.0.0 */ @@ -104,8 +106,8 @@ object MapType extends AbstractDataType { override private[sql] def simpleString: String = "map" /** - * Construct a [[MapType]] object with the given key type and value type. - * The `valueContainsNull` is true. + * Construct a [[MapType]] object with the given key type and value type. The + * `valueContainsNull` is true. */ def apply(keyType: DataType, valueType: DataType): MapType = MapType(keyType: DataType, valueType: DataType, valueContainsNull = true) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/Metadata.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/Metadata.scala index 70e03905d4b05..05f91b5ba2313 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/Metadata.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/Metadata.scala @@ -26,7 +26,6 @@ import org.apache.spark.annotation.Stable import org.apache.spark.sql.errors.DataTypeErrors import org.apache.spark.util.ArrayImplicits._ - /** * Metadata is a wrapper over Map[String, Any] that limits the value type to simple ones: Boolean, * Long, Double, String, Metadata, Array[Boolean], Array[Long], Array[Double], Array[String], and @@ -35,13 +34,14 @@ import org.apache.spark.util.ArrayImplicits._ * The default constructor is private. User should use either [[MetadataBuilder]] or * `Metadata.fromJson()` to create Metadata instances. * - * @param map an immutable map that stores the data + * @param map + * an immutable map that stores the data * * @since 1.3.0 */ @Stable sealed class Metadata private[types] (private[types] val map: Map[String, Any]) - extends Serializable { + extends Serializable { /** No-arg constructor for kryo. */ protected def this() = this(null) @@ -173,7 +173,8 @@ object Metadata { builder.putStringArray(key, value.asInstanceOf[List[JString]].map(_.s).toArray) case _: JObject => builder.putMetadataArray( - key, value.asInstanceOf[List[JObject]].map(fromJObject).toArray) + key, + value.asInstanceOf[List[JObject]].map(fromJObject).toArray) case other => throw DataTypeErrors.unsupportedArrayTypeError(other.getClass) } diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/NullType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/NullType.scala index d211fac70c641..4e7fd3a00a8af 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/NullType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/NullType.scala @@ -25,7 +25,7 @@ import org.apache.spark.annotation.Stable * @since 1.3.0 */ @Stable -class NullType private() extends DataType { +class NullType private () extends DataType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "NullType$" in byte code. // Defined with a private constructor so the companion object is the only possible instantiation. diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/ShortType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/ShortType.scala index 66696793e6279..c3b6bc75facd3 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/ShortType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/ShortType.scala @@ -25,7 +25,8 @@ import org.apache.spark.annotation.Stable * @since 1.3.0 */ @Stable -class ShortType private() extends IntegralType { +class ShortType private () extends IntegralType { + /** * The default size of a value of the ShortType is 2 bytes. */ diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala index df77f091f41f4..eba12c4ff4875 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala @@ -26,15 +26,17 @@ import org.apache.spark.sql.catalyst.util.CollationFactory * The data type representing `String` values. Please use the singleton `DataTypes.StringType`. * * @since 1.3.0 - * @param collationId The id of collation for this StringType. + * @param collationId + * The id of collation for this StringType. */ @Stable -class StringType private(val collationId: Int) extends AtomicType with Serializable { +class StringType private (val collationId: Int) extends AtomicType with Serializable { + /** - * Support for Binary Equality implies that strings are considered equal only if - * they are byte for byte equal. E.g. all accent or case-insensitive collations are considered - * non-binary. If this field is true, byte level operations can be used against this datatype - * (e.g. for equality and hashing). + * Support for Binary Equality implies that strings are considered equal only if they are byte + * for byte equal. E.g. all accent or case-insensitive collations are considered non-binary. If + * this field is true, byte level operations can be used against this datatype (e.g. for + * equality and hashing). */ private[sql] def supportsBinaryEquality: Boolean = CollationFactory.fetchCollation(collationId).supportsBinaryEquality @@ -49,18 +51,18 @@ class StringType private(val collationId: Int) extends AtomicType with Serializa collationId == CollationFactory.UTF8_LCASE_COLLATION_ID /** - * Support for Binary Ordering implies that strings are considered equal only - * if they are byte for byte equal. E.g. all accent or case-insensitive collations are - * considered non-binary. Also their ordering does not require calls to ICU library, as - * it follows spark internal implementation. If this field is true, byte level operations - * can be used against this datatype (e.g. for equality, hashing and ordering). + * Support for Binary Ordering implies that strings are considered equal only if they are byte + * for byte equal. E.g. all accent or case-insensitive collations are considered non-binary. + * Also their ordering does not require calls to ICU library, as it follows spark internal + * implementation. If this field is true, byte level operations can be used against this + * datatype (e.g. for equality, hashing and ordering). */ private[sql] def supportsBinaryOrdering: Boolean = CollationFactory.fetchCollation(collationId).supportsBinaryOrdering /** - * Type name that is shown to the customer. - * If this is an UTF8_BINARY collation output is `string` due to backwards compatibility. + * Type name that is shown to the customer. If this is an UTF8_BINARY collation output is + * `string` due to backwards compatibility. */ override def typeName: String = if (isUTF8BinaryCollation) "string" diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/StructField.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/StructField.scala index 3ff96fea9ee04..d4e590629921c 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/StructField.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/StructField.scala @@ -31,11 +31,15 @@ import org.apache.spark.util.SparkSchemaUtils /** * A field inside a StructType. - * @param name The name of this field. - * @param dataType The data type of this field. - * @param nullable Indicates if values of this field can be `null` values. - * @param metadata The metadata of this field. The metadata should be preserved during - * transformation if the content of the column is not modified, e.g, in selection. + * @param name + * The name of this field. + * @param dataType + * The data type of this field. + * @param nullable + * Indicates if values of this field can be `null` values. + * @param metadata + * The metadata of this field. The metadata should be preserved during transformation if the + * content of the column is not modified, e.g, in selection. * * @since 1.3.0 */ @@ -54,8 +58,9 @@ case class StructField( stringConcat: StringConcat, maxDepth: Int): Unit = { if (maxDepth > 0) { - stringConcat.append(s"$prefix-- ${SparkSchemaUtils.escapeMetaCharacters(name)}: " + - s"${dataType.typeName} (nullable = $nullable)\n") + stringConcat.append( + s"$prefix-- ${SparkSchemaUtils.escapeMetaCharacters(name)}: " + + s"${dataType.typeName} (nullable = $nullable)\n") DataType.buildFormattedString(dataType, s"$prefix |", stringConcat, maxDepth) } } diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/StructType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/StructType.scala index 3e637b5110122..4ef1cf400b80e 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/StructType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/StructType.scala @@ -41,10 +41,10 @@ import org.apache.spark.util.SparkCollectionUtils * {{{ * StructType(fields: Seq[StructField]) * }}} - * For a [[StructType]] object, one or multiple [[StructField]]s can be extracted by names. - * If multiple [[StructField]]s are extracted, a [[StructType]] object will be returned. - * If a provided name does not have a matching field, it will be ignored. For the case - * of extracting a single [[StructField]], a `null` will be returned. + * For a [[StructType]] object, one or multiple [[StructField]]s can be extracted by names. If + * multiple [[StructField]]s are extracted, a [[StructType]] object will be returned. If a + * provided name does not have a matching field, it will be ignored. For the case of extracting a + * single [[StructField]], a `null` will be returned. * * Scala Example: * {{{ @@ -126,8 +126,8 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru override def equals(that: Any): Boolean = { that match { case StructType(otherFields) => - java.util.Arrays.equals( - fields.asInstanceOf[Array[AnyRef]], otherFields.asInstanceOf[Array[AnyRef]]) + java.util.Arrays + .equals(fields.asInstanceOf[Array[AnyRef]], otherFields.asInstanceOf[Array[AnyRef]]) case _ => false } } @@ -146,7 +146,7 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru * .add(StructField("a", IntegerType, true)) * .add(StructField("b", LongType, false)) * .add(StructField("c", StringType, true)) - *}}} + * }}} */ def add(field: StructField): StructType = { StructType(fields :+ field) @@ -155,10 +155,8 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru /** * Creates a new [[StructType]] by adding a new nullable field with no metadata. * - * val struct = (new StructType) - * .add("a", IntegerType) - * .add("b", LongType) - * .add("c", StringType) + * val struct = (new StructType) .add("a", IntegerType) .add("b", LongType) .add("c", + * StringType) */ def add(name: String, dataType: DataType): StructType = { StructType(fields :+ StructField(name, dataType, nullable = true, Metadata.empty)) @@ -167,10 +165,8 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru /** * Creates a new [[StructType]] by adding a new field with no metadata. * - * val struct = (new StructType) - * .add("a", IntegerType, true) - * .add("b", LongType, false) - * .add("c", StringType, true) + * val struct = (new StructType) .add("a", IntegerType, true) .add("b", LongType, false) + * .add("c", StringType, true) */ def add(name: String, dataType: DataType, nullable: Boolean): StructType = { StructType(fields :+ StructField(name, dataType, nullable, Metadata.empty)) @@ -185,11 +181,7 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru * .add("c", StringType, true, Metadata.empty) * }}} */ - def add( - name: String, - dataType: DataType, - nullable: Boolean, - metadata: Metadata): StructType = { + def add(name: String, dataType: DataType, nullable: Boolean, metadata: Metadata): StructType = { StructType(fields :+ StructField(name, dataType, nullable, metadata)) } @@ -202,11 +194,7 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru * .add("c", StringType, true, "comment3") * }}} */ - def add( - name: String, - dataType: DataType, - nullable: Boolean, - comment: String): StructType = { + def add(name: String, dataType: DataType, nullable: Boolean, comment: String): StructType = { StructType(fields :+ StructField(name, dataType, nullable).withComment(comment)) } @@ -226,8 +214,8 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru } /** - * Creates a new [[StructType]] by adding a new field with no metadata where the - * dataType is specified as a String. + * Creates a new [[StructType]] by adding a new field with no metadata where the dataType is + * specified as a String. * * {{{ * val struct = (new StructType) @@ -241,8 +229,8 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru } /** - * Creates a new [[StructType]] by adding a new field and specifying metadata where the - * dataType is specified as a String. + * Creates a new [[StructType]] by adding a new field and specifying metadata where the dataType + * is specified as a String. * {{{ * val struct = (new StructType) * .add("a", "int", true, Metadata.empty) @@ -250,17 +238,13 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru * .add("c", "string", true, Metadata.empty) * }}} */ - def add( - name: String, - dataType: String, - nullable: Boolean, - metadata: Metadata): StructType = { + def add(name: String, dataType: String, nullable: Boolean, metadata: Metadata): StructType = { add(name, DataTypeParser.parseDataType(dataType), nullable, metadata) } /** - * Creates a new [[StructType]] by adding a new field and specifying metadata where the - * dataType is specified as a String. + * Creates a new [[StructType]] by adding a new field and specifying metadata where the dataType + * is specified as a String. * {{{ * val struct = (new StructType) * .add("a", "int", true, "comment1") @@ -268,21 +252,19 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru * .add("c", "string", true, "comment3") * }}} */ - def add( - name: String, - dataType: String, - nullable: Boolean, - comment: String): StructType = { + def add(name: String, dataType: String, nullable: Boolean, comment: String): StructType = { add(name, DataTypeParser.parseDataType(dataType), nullable, comment) } /** * Extracts the [[StructField]] with the given name. * - * @throws IllegalArgumentException if a field with the given name does not exist + * @throws IllegalArgumentException + * if a field with the given name does not exist */ def apply(name: String): StructField = { - nameToField.getOrElse(name, + nameToField.getOrElse( + name, throw new SparkIllegalArgumentException( errorClass = "FIELD_NOT_FOUND", messageParameters = immutable.Map( @@ -294,7 +276,8 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru * Returns a [[StructType]] containing [[StructField]]s of the given names, preserving the * original order of fields. * - * @throws IllegalArgumentException if at least one given field name does not exist + * @throws IllegalArgumentException + * if at least one given field name does not exist */ def apply(names: Set[String]): StructType = { val nonExistFields = names -- fieldNamesSet @@ -312,10 +295,12 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru /** * Returns the index of a given field. * - * @throws IllegalArgumentException if a field with the given name does not exist + * @throws IllegalArgumentException + * if a field with the given name does not exist */ def fieldIndex(name: String): Int = { - nameToIndex.getOrElse(name, + nameToIndex.getOrElse( + name, throw new SparkIllegalArgumentException( errorClass = "FIELD_NOT_FOUND", messageParameters = immutable.Map( @@ -354,10 +339,7 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru } else if (found.isEmpty) { None } else { - findField( - parent = found.head, - searchPath = searchPath.tail, - normalizedPath) + findField(parent = found.head, searchPath = searchPath.tail, normalizedPath) } } @@ -433,11 +415,14 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru override def defaultSize: Int = fields.map(_.dataType.defaultSize).sum override def simpleString: String = { - val fieldTypes = fields.to(LazyList) + val fieldTypes = fields + .to(LazyList) .map(field => s"${field.name}:${field.dataType.simpleString}") SparkStringUtils.truncatedString( fieldTypes, - "struct<", ",", ">", + "struct<", + ",", + ">", SqlApiConf.get.maxToStringFields) } @@ -460,9 +445,9 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru /** * Returns a string containing a schema in DDL format. For example, the following value: - * `StructType(Seq(StructField("eventId", IntegerType), StructField("s", StringType)))` - * will be converted to `eventId` INT, `s` STRING. - * The returned DDL schema can be used in a table creation. + * `StructType(Seq(StructField("eventId", IntegerType), StructField("s", StringType)))` will be + * converted to `eventId` INT, `s` STRING. The returned DDL schema can be used in a table + * creation. * * @since 2.4.0 */ @@ -470,8 +455,8 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru private[sql] override def simpleString(maxNumberFields: Int): String = { val builder = new StringBuilder - val fieldTypes = fields.take(maxNumberFields).map { - f => s"${f.name}: ${f.dataType.simpleString(maxNumberFields)}" + val fieldTypes = fields.take(maxNumberFields).map { f => + s"${f.name}: ${f.dataType.simpleString(maxNumberFields)}" } builder.append("struct<") builder.append(fieldTypes.mkString(", ")) @@ -486,31 +471,29 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru } /** - * Merges with another schema (`StructType`). For a struct field A from `this` and a struct field - * B from `that`, + * Merges with another schema (`StructType`). For a struct field A from `this` and a struct + * field B from `that`, * - * 1. If A and B have the same name and data type, they are merged to a field C with the same name - * and data type. C is nullable if and only if either A or B is nullable. - * 2. If A doesn't exist in `that`, it's included in the result schema. - * 3. If B doesn't exist in `this`, it's also included in the result schema. - * 4. Otherwise, `this` and `that` are considered as conflicting schemas and an exception would be - * thrown. + * 1. If A and B have the same name and data type, they are merged to a field C with the same + * name and data type. C is nullable if and only if either A or B is nullable. 2. If A + * doesn't exist in `that`, it's included in the result schema. 3. If B doesn't exist in + * `this`, it's also included in the result schema. 4. Otherwise, `this` and `that` are + * considered as conflicting schemas and an exception would be thrown. */ private[sql] def merge(that: StructType, caseSensitive: Boolean = true): StructType = StructType.merge(this, that, caseSensitive).asInstanceOf[StructType] override private[spark] def asNullable: StructType = { - val newFields = fields.map { - case StructField(name, dataType, nullable, metadata) => - StructField(name, dataType.asNullable, nullable = true, metadata) + val newFields = fields.map { case StructField(name, dataType, nullable, metadata) => + StructField(name, dataType.asNullable, nullable = true, metadata) } StructType(newFields) } /** - * Returns the same data type but set all nullability fields are true - * (`StructField.nullable`, `ArrayType.containsNull`, and `MapType.valueContainsNull`). + * Returns the same data type but set all nullability fields are true (`StructField.nullable`, + * `ArrayType.containsNull`, and `MapType.valueContainsNull`). * * @since 4.0.0 */ @@ -562,7 +545,8 @@ object StructType extends AbstractDataType { case StructType(fields) => val newFields = fields.map { f => val mb = new MetadataBuilder() - f.copy(dataType = removeMetadata(key, f.dataType), + f.copy( + dataType = removeMetadata(key, f.dataType), metadata = mb.withMetadata(f.metadata).remove(key).build()) } StructType(newFields) @@ -570,39 +554,49 @@ object StructType extends AbstractDataType { } /** - * This leverages `merge` to merge data types for UNION operator by specializing - * the handling of struct types to follow UNION semantics. + * This leverages `merge` to merge data types for UNION operator by specializing the handling of + * struct types to follow UNION semantics. */ private[sql] def unionLikeMerge(left: DataType, right: DataType): DataType = - mergeInternal(left, right, (s1: StructType, s2: StructType) => { - val leftFields = s1.fields - val rightFields = s2.fields - require(leftFields.length == rightFields.length, "To merge nullability, " + - "two structs must have same number of fields.") - - val newFields = leftFields.zip(rightFields).map { - case (leftField, rightField) => + mergeInternal( + left, + right, + (s1: StructType, s2: StructType) => { + val leftFields = s1.fields + val rightFields = s2.fields + require( + leftFields.length == rightFields.length, + "To merge nullability, " + + "two structs must have same number of fields.") + + val newFields = leftFields.zip(rightFields).map { case (leftField, rightField) => leftField.copy( dataType = unionLikeMerge(leftField.dataType, rightField.dataType), nullable = leftField.nullable || rightField.nullable) - } - StructType(newFields) - }) - - private[sql] def merge(left: DataType, right: DataType, caseSensitive: Boolean = true): DataType = - mergeInternal(left, right, (s1: StructType, s2: StructType) => { - val leftFields = s1.fields - val rightFields = s2.fields - val newFields = mutable.ArrayBuffer.empty[StructField] + } + StructType(newFields) + }) - def normalize(name: String): String = { - if (caseSensitive) name else name.toLowerCase(Locale.ROOT) - } + private[sql] def merge( + left: DataType, + right: DataType, + caseSensitive: Boolean = true): DataType = + mergeInternal( + left, + right, + (s1: StructType, s2: StructType) => { + val leftFields = s1.fields + val rightFields = s2.fields + val newFields = mutable.ArrayBuffer.empty[StructField] + + def normalize(name: String): String = { + if (caseSensitive) name else name.toLowerCase(Locale.ROOT) + } - val rightMapped = fieldsMap(rightFields, caseSensitive) - leftFields.foreach { - case leftField @ StructField(leftName, leftType, leftNullable, _) => - rightMapped.get(normalize(leftName)) + val rightMapped = fieldsMap(rightFields, caseSensitive) + leftFields.foreach { case leftField @ StructField(leftName, leftType, leftNullable, _) => + rightMapped + .get(normalize(leftName)) .map { case rightField @ StructField(rightName, rightType, rightNullable, _) => try { leftField.copy( @@ -610,39 +604,40 @@ object StructType extends AbstractDataType { nullable = leftNullable || rightNullable) } catch { case NonFatal(e) => - throw DataTypeErrors.cannotMergeIncompatibleDataTypesError( - leftType, rightType) + throw DataTypeErrors.cannotMergeIncompatibleDataTypesError(leftType, rightType) } } .orElse { Some(leftField) } .foreach(newFields += _) - } - - val leftMapped = fieldsMap(leftFields, caseSensitive) - rightFields - .filterNot(f => leftMapped.contains(normalize(f.name))) - .foreach { f => - newFields += f } - StructType(newFields.toArray) - }) + val leftMapped = fieldsMap(leftFields, caseSensitive) + rightFields + .filterNot(f => leftMapped.contains(normalize(f.name))) + .foreach { f => + newFields += f + } + + StructType(newFields.toArray) + }) private def mergeInternal( left: DataType, right: DataType, mergeStruct: (StructType, StructType) => StructType): DataType = (left, right) match { - case (ArrayType(leftElementType, leftContainsNull), - ArrayType(rightElementType, rightContainsNull)) => + case ( + ArrayType(leftElementType, leftContainsNull), + ArrayType(rightElementType, rightContainsNull)) => ArrayType( mergeInternal(leftElementType, rightElementType, mergeStruct), leftContainsNull || rightContainsNull) - case (MapType(leftKeyType, leftValueType, leftContainsNull), - MapType(rightKeyType, rightValueType, rightContainsNull)) => + case ( + MapType(leftKeyType, leftValueType, leftContainsNull), + MapType(rightKeyType, rightValueType, rightContainsNull)) => MapType( mergeInternal(leftKeyType, rightKeyType, mergeStruct), mergeInternal(leftValueType, rightValueType, mergeStruct), @@ -650,17 +645,20 @@ object StructType extends AbstractDataType { case (s1: StructType, s2: StructType) => mergeStruct(s1, s2) - case (DecimalType.Fixed(leftPrecision, leftScale), - DecimalType.Fixed(rightPrecision, rightScale)) => + case ( + DecimalType.Fixed(leftPrecision, leftScale), + DecimalType.Fixed(rightPrecision, rightScale)) => if (leftScale == rightScale) { DecimalType(leftPrecision.max(rightPrecision), leftScale) } else { throw DataTypeErrors.cannotMergeDecimalTypesWithIncompatibleScaleError( - leftScale, rightScale) + leftScale, + rightScale) } case (leftUdt: UserDefinedType[_], rightUdt: UserDefinedType[_]) - if leftUdt.userClass == rightUdt.userClass => leftUdt + if leftUdt.userClass == rightUdt.userClass => + leftUdt case (YearMonthIntervalType(lstart, lend), YearMonthIntervalType(rstart, rend)) => YearMonthIntervalType(Math.min(lstart, rstart).toByte, Math.max(lend, rend).toByte) @@ -706,10 +704,12 @@ object StructType extends AbstractDataType { // Found a missing field in `source`. newFields += field } else if (bothStructType(found.get.dataType, field.dataType) && - !found.get.dataType.sameType(field.dataType)) { + !found.get.dataType.sameType(field.dataType)) { // Found a field with same name, but different data type. - findMissingFields(found.get.dataType.asInstanceOf[StructType], - field.dataType.asInstanceOf[StructType], resolver).map { missingType => + findMissingFields( + found.get.dataType.asInstanceOf[StructType], + field.dataType.asInstanceOf[StructType], + resolver).map { missingType => newFields += found.get.copy(dataType = missingType) } } diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/TimestampNTZType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/TimestampNTZType.scala index 9968d75dd2577..b08d16f0e2c97 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/TimestampNTZType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/TimestampNTZType.scala @@ -20,16 +20,17 @@ package org.apache.spark.sql.types import org.apache.spark.annotation.Unstable /** - * The timestamp without time zone type represents a local time in microsecond precision, - * which is independent of time zone. - * Its valid range is [0001-01-01T00:00:00.000000, 9999-12-31T23:59:59.999999]. - * To represent an absolute point in time, use `TimestampType` instead. + * The timestamp without time zone type represents a local time in microsecond precision, which is + * independent of time zone. Its valid range is [0001-01-01T00:00:00.000000, + * 9999-12-31T23:59:59.999999]. To represent an absolute point in time, use `TimestampType` + * instead. * * Please use the singleton `DataTypes.TimestampNTZType` to refer the type. * @since 3.4.0 */ @Unstable -class TimestampNTZType private() extends DatetimeType { +class TimestampNTZType private () extends DatetimeType { + /** * The default size of a value of the TimestampNTZType is 8 bytes. */ @@ -42,9 +43,9 @@ class TimestampNTZType private() extends DatetimeType { /** * The companion case object and its class is separated so the companion object also subclasses - * the TimestampNTZType class. Otherwise, the companion object would be of type - * "TimestampNTZType" in byte code. Defined with a private constructor so the companion - * object is the only possible instantiation. + * the TimestampNTZType class. Otherwise, the companion object would be of type "TimestampNTZType" + * in byte code. Defined with a private constructor so the companion object is the only possible + * instantiation. * * @since 3.4.0 */ diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/TimestampType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/TimestampType.scala index 1185e4a9e32ca..bf869d1f38c57 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/TimestampType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/TimestampType.scala @@ -20,16 +20,16 @@ package org.apache.spark.sql.types import org.apache.spark.annotation.Stable /** - * The timestamp type represents a time instant in microsecond precision. - * Valid range is [0001-01-01T00:00:00.000000Z, 9999-12-31T23:59:59.999999Z] where - * the left/right-bound is a date and time of the proleptic Gregorian - * calendar in UTC+00:00. + * The timestamp type represents a time instant in microsecond precision. Valid range is + * [0001-01-01T00:00:00.000000Z, 9999-12-31T23:59:59.999999Z] where the left/right-bound is a date + * and time of the proleptic Gregorian calendar in UTC+00:00. * * Please use the singleton `DataTypes.TimestampType` to refer the type. * @since 1.3.0 */ @Stable -class TimestampType private() extends DatetimeType { +class TimestampType private () extends DatetimeType { + /** * The default size of a value of the TimestampType is 8 bytes. */ @@ -40,8 +40,8 @@ class TimestampType private() extends DatetimeType { /** * The companion case object and its class is separated so the companion object also subclasses - * the TimestampType class. Otherwise, the companion object would be of type "TimestampType$" - * in byte code. Defined with a private constructor so the companion object is the only possible + * the TimestampType class. Otherwise, the companion object would be of type "TimestampType$" in + * byte code. Defined with a private constructor so the companion object is the only possible * instantiation. * * @since 1.3.0 diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/UDTRegistration.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/UDTRegistration.scala index 9219c1d139b99..85d421a07577b 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/UDTRegistration.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/UDTRegistration.scala @@ -45,21 +45,26 @@ object UDTRegistration extends Serializable with Logging { /** * Queries if a given user class is already registered or not. - * @param userClassName the name of user class - * @return boolean value indicates if the given user class is registered or not + * @param userClassName + * the name of user class + * @return + * boolean value indicates if the given user class is registered or not */ def exists(userClassName: String): Boolean = udtMap.contains(userClassName) /** - * Registers an UserDefinedType to an user class. If the user class is already registered - * with another UserDefinedType, warning log message will be shown. - * @param userClass the name of user class - * @param udtClass the name of UserDefinedType class for the given userClass + * Registers an UserDefinedType to an user class. If the user class is already registered with + * another UserDefinedType, warning log message will be shown. + * @param userClass + * the name of user class + * @param udtClass + * the name of UserDefinedType class for the given userClass */ def register(userClass: String, udtClass: String): Unit = { if (udtMap.contains(userClass)) { - logWarning(log"Cannot register UDT for ${MDC(LogKeys.CLASS_NAME, userClass)}, " + - log"which is already registered.") + logWarning( + log"Cannot register UDT for ${MDC(LogKeys.CLASS_NAME, userClass)}, " + + log"which is already registered.") } else { // When register UDT with class name, we can't check if the UDT class is an UserDefinedType, // or not. The check is deferred. @@ -69,8 +74,10 @@ object UDTRegistration extends Serializable with Logging { /** * Returns the Class of UserDefinedType for the name of a given user class. - * @param userClass class name of user class - * @return Option value of the Class object of UserDefinedType + * @param userClass + * class name of user class + * @return + * Option value of the Class object of UserDefinedType */ def getUDTFor(userClass: String): Option[Class[_]] = { udtMap.get(userClass).map { udtClassName => diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/UpCastRule.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/UpCastRule.scala index 7ec00bde0b25f..4993e249b3059 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/UpCastRule.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/UpCastRule.scala @@ -22,13 +22,8 @@ package org.apache.spark.sql.types private[sql] object UpCastRule { // See https://cwiki.apache.org/confluence/display/Hive/LanguageManual+Types. // The conversion for integral and floating point types have a linear widening hierarchy: - val numericPrecedence: IndexedSeq[NumericType] = IndexedSeq( - ByteType, - ShortType, - IntegerType, - LongType, - FloatType, - DoubleType) + val numericPrecedence: IndexedSeq[NumericType] = + IndexedSeq(ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType) /** * Returns true iff we can safely up-cast the `from` type to `to` type without any truncating or @@ -62,10 +57,9 @@ private[sql] object UpCastRule { case (StructType(fromFields), StructType(toFields)) => fromFields.length == toFields.length && - fromFields.zip(toFields).forall { - case (f1, f2) => - resolvableNullability(f1.nullable, f2.nullable) && canUpCast(f1.dataType, f2.dataType) - } + fromFields.zip(toFields).forall { case (f1, f2) => + resolvableNullability(f1.nullable, f2.nullable) && canUpCast(f1.dataType, f2.dataType) + } case (_: DayTimeIntervalType, _: DayTimeIntervalType) => true case (_: YearMonthIntervalType, _: YearMonthIntervalType) => true diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/UserDefinedType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/UserDefinedType.scala index 5cbd876b31e68..dd8ca26c52462 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/UserDefinedType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/UserDefinedType.scala @@ -27,15 +27,14 @@ import org.apache.spark.annotation.{DeveloperApi, Since} /** * The data type for User Defined Types (UDTs). * - * This interface allows a user to make their own classes more interoperable with SparkSQL; - * e.g., by creating a [[UserDefinedType]] for a class X, it becomes possible to create - * a `DataFrame` which has class X in the schema. + * This interface allows a user to make their own classes more interoperable with SparkSQL; e.g., + * by creating a [[UserDefinedType]] for a class X, it becomes possible to create a `DataFrame` + * which has class X in the schema. * - * For SparkSQL to recognize UDTs, the UDT must be annotated with - * [[SQLUserDefinedType]]. + * For SparkSQL to recognize UDTs, the UDT must be annotated with [[SQLUserDefinedType]]. * - * The conversion via `serialize` occurs when instantiating a `DataFrame` from another RDD. - * The conversion via `deserialize` occurs when reading from a `DataFrame`. + * The conversion via `serialize` occurs when instantiating a `DataFrame` from another RDD. The + * conversion via `deserialize` occurs when reading from a `DataFrame`. */ @DeveloperApi @Since("3.2.0") @@ -81,7 +80,7 @@ abstract class UserDefinedType[UserType >: Null] extends DataType with Serializa override private[sql] def acceptsType(dataType: DataType): Boolean = dataType match { case other: UserDefinedType[_] if this.userClass != null && other.userClass != null => this.getClass == other.getClass || - this.userClass.isAssignableFrom(other.userClass) + this.userClass.isAssignableFrom(other.userClass) case _ => false } @@ -98,6 +97,7 @@ abstract class UserDefinedType[UserType >: Null] extends DataType with Serializa } private[spark] object UserDefinedType { + /** * Get the sqlType of a (potential) [[UserDefinedType]]. */ @@ -115,7 +115,8 @@ private[spark] object UserDefinedType { private[sql] class PythonUserDefinedType( val sqlType: DataType, override val pyUDT: String, - override val serializedPyClass: String) extends UserDefinedType[Any] { + override val serializedPyClass: String) + extends UserDefinedType[Any] { /* The serialization is handled by UDT class in Python */ override def serialize(obj: Any): Any = obj diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/VariantType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/VariantType.scala index 103fe7a59fc83..4d775c3e1e390 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/VariantType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/VariantType.scala @@ -20,9 +20,9 @@ package org.apache.spark.sql.types import org.apache.spark.annotation.Unstable /** - * The data type representing semi-structured values with arbitrary hierarchical data structures. It - * is intended to store parsed JSON values and most other data types in the system (e.g., it cannot - * store a map with a non-string key type). + * The data type representing semi-structured values with arbitrary hierarchical data structures. + * It is intended to store parsed JSON values and most other data types in the system (e.g., it + * cannot store a map with a non-string key type). * * @since 4.0.0 */ diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/YearMonthIntervalType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/YearMonthIntervalType.scala index 6532a3b220c5b..f69054f2c1fbc 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/YearMonthIntervalType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/YearMonthIntervalType.scala @@ -29,18 +29,19 @@ import org.apache.spark.sql.types.YearMonthIntervalType.fieldToString * * `YearMonthIntervalType` represents positive as well as negative year-month intervals. * - * @param startField The leftmost field which the type comprises of. Valid values: - * 0 (YEAR), 1 (MONTH). - * @param endField The rightmost field which the type comprises of. Valid values: - * 0 (YEAR), 1 (MONTH). + * @param startField + * The leftmost field which the type comprises of. Valid values: 0 (YEAR), 1 (MONTH). + * @param endField + * The rightmost field which the type comprises of. Valid values: 0 (YEAR), 1 (MONTH). * * @since 3.2.0 */ @Unstable case class YearMonthIntervalType(startField: Byte, endField: Byte) extends AnsiIntervalType { + /** - * Year-month interval values always occupy 4 bytes. - * The YEAR field is constrained by the upper bound 178956970 to fit to `Int`. + * Year-month interval values always occupy 4 bytes. The YEAR field is constrained by the upper + * bound 178956970 to fit to `Int`. */ override def defaultSize: Int = 4 diff --git a/sql/api/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala b/sql/api/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala index 6852fe09ef96b..1740cbe2957b8 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala @@ -38,33 +38,33 @@ private[sql] object ArrowUtils { // todo: support more types. /** Maps data type from Spark to Arrow. NOTE: timeZoneId required for TimestampTypes */ - def toArrowType( - dt: DataType, timeZoneId: String, largeVarTypes: Boolean = false): ArrowType = dt match { - case BooleanType => ArrowType.Bool.INSTANCE - case ByteType => new ArrowType.Int(8, true) - case ShortType => new ArrowType.Int(8 * 2, true) - case IntegerType => new ArrowType.Int(8 * 4, true) - case LongType => new ArrowType.Int(8 * 8, true) - case FloatType => new ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE) - case DoubleType => new ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE) - case _: StringType if !largeVarTypes => ArrowType.Utf8.INSTANCE - case BinaryType if !largeVarTypes => ArrowType.Binary.INSTANCE - case _: StringType if largeVarTypes => ArrowType.LargeUtf8.INSTANCE - case BinaryType if largeVarTypes => ArrowType.LargeBinary.INSTANCE - case DecimalType.Fixed(precision, scale) => new ArrowType.Decimal(precision, scale, 8 * 16) - case DateType => new ArrowType.Date(DateUnit.DAY) - case TimestampType if timeZoneId == null => - throw SparkException.internalError("Missing timezoneId where it is mandatory.") - case TimestampType => new ArrowType.Timestamp(TimeUnit.MICROSECOND, timeZoneId) - case TimestampNTZType => - new ArrowType.Timestamp(TimeUnit.MICROSECOND, null) - case NullType => ArrowType.Null.INSTANCE - case _: YearMonthIntervalType => new ArrowType.Interval(IntervalUnit.YEAR_MONTH) - case _: DayTimeIntervalType => new ArrowType.Duration(TimeUnit.MICROSECOND) - case CalendarIntervalType => new ArrowType.Interval(IntervalUnit.MONTH_DAY_NANO) - case _ => - throw ExecutionErrors.unsupportedDataTypeError(dt) - } + def toArrowType(dt: DataType, timeZoneId: String, largeVarTypes: Boolean = false): ArrowType = + dt match { + case BooleanType => ArrowType.Bool.INSTANCE + case ByteType => new ArrowType.Int(8, true) + case ShortType => new ArrowType.Int(8 * 2, true) + case IntegerType => new ArrowType.Int(8 * 4, true) + case LongType => new ArrowType.Int(8 * 8, true) + case FloatType => new ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE) + case DoubleType => new ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE) + case _: StringType if !largeVarTypes => ArrowType.Utf8.INSTANCE + case BinaryType if !largeVarTypes => ArrowType.Binary.INSTANCE + case _: StringType if largeVarTypes => ArrowType.LargeUtf8.INSTANCE + case BinaryType if largeVarTypes => ArrowType.LargeBinary.INSTANCE + case DecimalType.Fixed(precision, scale) => new ArrowType.Decimal(precision, scale, 8 * 16) + case DateType => new ArrowType.Date(DateUnit.DAY) + case TimestampType if timeZoneId == null => + throw SparkException.internalError("Missing timezoneId where it is mandatory.") + case TimestampType => new ArrowType.Timestamp(TimeUnit.MICROSECOND, timeZoneId) + case TimestampNTZType => + new ArrowType.Timestamp(TimeUnit.MICROSECOND, null) + case NullType => ArrowType.Null.INSTANCE + case _: YearMonthIntervalType => new ArrowType.Interval(IntervalUnit.YEAR_MONTH) + case _: DayTimeIntervalType => new ArrowType.Duration(TimeUnit.MICROSECOND) + case CalendarIntervalType => new ArrowType.Interval(IntervalUnit.MONTH_DAY_NANO) + case _ => + throw ExecutionErrors.unsupportedDataTypeError(dt) + } def fromArrowType(dt: ArrowType): DataType = dt match { case ArrowType.Bool.INSTANCE => BooleanType @@ -73,9 +73,11 @@ private[sql] object ArrowUtils { case int: ArrowType.Int if int.getIsSigned && int.getBitWidth == 8 * 4 => IntegerType case int: ArrowType.Int if int.getIsSigned && int.getBitWidth == 8 * 8 => LongType case float: ArrowType.FloatingPoint - if float.getPrecision() == FloatingPointPrecision.SINGLE => FloatType + if float.getPrecision() == FloatingPointPrecision.SINGLE => + FloatType case float: ArrowType.FloatingPoint - if float.getPrecision() == FloatingPointPrecision.DOUBLE => DoubleType + if float.getPrecision() == FloatingPointPrecision.DOUBLE => + DoubleType case ArrowType.Utf8.INSTANCE => StringType case ArrowType.Binary.INSTANCE => BinaryType case ArrowType.LargeUtf8.INSTANCE => StringType @@ -83,13 +85,15 @@ private[sql] object ArrowUtils { case d: ArrowType.Decimal => DecimalType(d.getPrecision, d.getScale) case date: ArrowType.Date if date.getUnit == DateUnit.DAY => DateType case ts: ArrowType.Timestamp - if ts.getUnit == TimeUnit.MICROSECOND && ts.getTimezone == null => TimestampNTZType + if ts.getUnit == TimeUnit.MICROSECOND && ts.getTimezone == null => + TimestampNTZType case ts: ArrowType.Timestamp if ts.getUnit == TimeUnit.MICROSECOND => TimestampType case ArrowType.Null.INSTANCE => NullType - case yi: ArrowType.Interval if yi.getUnit == IntervalUnit.YEAR_MONTH => YearMonthIntervalType() + case yi: ArrowType.Interval if yi.getUnit == IntervalUnit.YEAR_MONTH => + YearMonthIntervalType() case di: ArrowType.Duration if di.getUnit == TimeUnit.MICROSECOND => DayTimeIntervalType() - case ci: ArrowType.Interval - if ci.getUnit == IntervalUnit.MONTH_DAY_NANO => CalendarIntervalType + case ci: ArrowType.Interval if ci.getUnit == IntervalUnit.MONTH_DAY_NANO => + CalendarIntervalType case _ => throw ExecutionErrors.unsupportedArrowTypeError(dt) } @@ -103,37 +107,54 @@ private[sql] object ArrowUtils { dt match { case ArrayType(elementType, containsNull) => val fieldType = new FieldType(nullable, ArrowType.List.INSTANCE, null) - new Field(name, fieldType, - Seq(toArrowField("element", elementType, containsNull, timeZoneId, - largeVarTypes)).asJava) + new Field( + name, + fieldType, + Seq( + toArrowField("element", elementType, containsNull, timeZoneId, largeVarTypes)).asJava) case StructType(fields) => val fieldType = new FieldType(nullable, ArrowType.Struct.INSTANCE, null) - new Field(name, fieldType, - fields.map { field => - toArrowField(field.name, field.dataType, field.nullable, timeZoneId, largeVarTypes) - }.toImmutableArraySeq.asJava) + new Field( + name, + fieldType, + fields + .map { field => + toArrowField(field.name, field.dataType, field.nullable, timeZoneId, largeVarTypes) + } + .toImmutableArraySeq + .asJava) case MapType(keyType, valueType, valueContainsNull) => val mapType = new FieldType(nullable, new ArrowType.Map(false), null) // Note: Map Type struct can not be null, Struct Type key field can not be null - new Field(name, mapType, - Seq(toArrowField(MapVector.DATA_VECTOR_NAME, - new StructType() - .add(MapVector.KEY_NAME, keyType, nullable = false) - .add(MapVector.VALUE_NAME, valueType, nullable = valueContainsNull), - nullable = false, - timeZoneId, - largeVarTypes)).asJava) + new Field( + name, + mapType, + Seq( + toArrowField( + MapVector.DATA_VECTOR_NAME, + new StructType() + .add(MapVector.KEY_NAME, keyType, nullable = false) + .add(MapVector.VALUE_NAME, valueType, nullable = valueContainsNull), + nullable = false, + timeZoneId, + largeVarTypes)).asJava) case udt: UserDefinedType[_] => toArrowField(name, udt.sqlType, nullable, timeZoneId, largeVarTypes) case _: VariantType => - val fieldType = new FieldType(nullable, ArrowType.Struct.INSTANCE, null, + val fieldType = new FieldType( + nullable, + ArrowType.Struct.INSTANCE, + null, Map("variant" -> "true").asJava) - new Field(name, fieldType, - Seq(toArrowField("value", BinaryType, false, timeZoneId, largeVarTypes), + new Field( + name, + fieldType, + Seq( + toArrowField("value", BinaryType, false, timeZoneId, largeVarTypes), toArrowField("metadata", BinaryType, false, timeZoneId, largeVarTypes)).asJava) case dataType => - val fieldType = new FieldType(nullable, toArrowType(dataType, timeZoneId, - largeVarTypes), null) + val fieldType = + new FieldType(nullable, toArrowType(dataType, timeZoneId, largeVarTypes), null) new Field(name, fieldType, Seq.empty[Field].asJava) } } @@ -149,9 +170,12 @@ private[sql] object ArrowUtils { val elementField = field.getChildren().get(0) val elementType = fromArrowField(elementField) ArrayType(elementType, containsNull = elementField.isNullable) - case ArrowType.Struct.INSTANCE if field.getMetadata.getOrDefault("variant", "") == "true" - && field.getChildren.asScala.map(_.getName).asJava - .containsAll(Seq("value", "metadata").asJava) => + case ArrowType.Struct.INSTANCE + if field.getMetadata.getOrDefault("variant", "") == "true" + && field.getChildren.asScala + .map(_.getName) + .asJava + .containsAll(Seq("value", "metadata").asJava) => VariantType case ArrowType.Struct.INSTANCE => val fields = field.getChildren().asScala.map { child => @@ -163,7 +187,9 @@ private[sql] object ArrowUtils { } } - /** Maps schema from Spark to Arrow. NOTE: timeZoneId required for TimestampType in StructType */ + /** + * Maps schema from Spark to Arrow. NOTE: timeZoneId required for TimestampType in StructType + */ def toArrowSchema( schema: StructType, timeZoneId: String, @@ -187,14 +213,17 @@ private[sql] object ArrowUtils { } private def deduplicateFieldNames( - dt: DataType, errorOnDuplicatedFieldNames: Boolean): DataType = dt match { - case udt: UserDefinedType[_] => deduplicateFieldNames(udt.sqlType, errorOnDuplicatedFieldNames) + dt: DataType, + errorOnDuplicatedFieldNames: Boolean): DataType = dt match { + case udt: UserDefinedType[_] => + deduplicateFieldNames(udt.sqlType, errorOnDuplicatedFieldNames) case st @ StructType(fields) => val newNames = if (st.names.toSet.size == st.names.length) { st.names } else { if (errorOnDuplicatedFieldNames) { - throw ExecutionErrors.duplicatedFieldNameInArrowStructError(st.names.toImmutableArraySeq) + throw ExecutionErrors.duplicatedFieldNameInArrowStructError( + st.names.toImmutableArraySeq) } val genNawName = st.names.groupBy(identity).map { case (name, names) if names.length > 1 => @@ -207,7 +236,10 @@ private[sql] object ArrowUtils { val newFields = fields.zip(newNames).map { case (StructField(_, dataType, nullable, metadata), name) => StructField( - name, deduplicateFieldNames(dataType, errorOnDuplicatedFieldNames), nullable, metadata) + name, + deduplicateFieldNames(dataType, errorOnDuplicatedFieldNames), + nullable, + metadata) } StructType(newFields) case ArrayType(elementType, containsNull) =>