ds = spark.createDataset(data, Encoders.STRING());
+ * }}}
+ *
+ * @since 2.0.0
+ */
+ 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.
+ *
+ * @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.
+ *
+ * @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.
+ *
+ * @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.
+ *
+ * @since 2.0.0
+ */
+ def range(start: Long, end: Long, step: Long, numPartitions: Int): DS[lang.Long]
+
+ /* ------------------------- *
+ | Catalog-related methods |
+ * ------------------------- */
+
+ /**
+ * 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.
+ * @since 2.0.0
+ */
+ def table(tableName: String): DS[Row]
+
+ /* ----------------- *
+ | Everything else |
+ * ----------------- */
+
+ /**
+ * 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.
+ * @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.
+ * @since 3.4.0
+ */
+ @Experimental
+ def sql(sqlText: String, args: util.Map[String, Any]): DS[Row] = {
+ sql(sqlText, args.asScala.toMap)
+ }
+
+ /**
+ * 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
+ */
+ def sql(sqlText: String): DS[Row] = sql(sqlText, Map.empty[String, Any])
+
+ /**
+ * Executes some code block and prints to stdout the time taken to execute the block. This is
+ * available in Scala only and is used primarily for interactive testing and debugging.
+ *
+ * @since 2.1.0
+ */
+ def time[T](f: => T): T = {
+ val start = System.nanoTime()
+ val ret = f
+ val end = System.nanoTime()
+ // scalastyle:off println
+ println(s"Time taken: ${NANOSECONDS.toMillis(end - start)} ms")
+ // scalastyle:on println
+ ret
+ }
+
+ /**
+ * Synonym for `close()`.
+ *
+ * @since 2.0.0
+ */
+ def stop(): Unit = close()
+}
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 ffc198fb4cae3..e46d6c95b31ae 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
@@ -1702,7 +1702,7 @@ object functions {
* @group normal_funcs
* @since 1.5.0
*/
- def broadcast[DS[_] <: api.Dataset[_, DS]](df: DS[_]): df.type = {
+ def broadcast[DS[U] <: api.Dataset[U, DS]](df: DS[_]): df.type = {
df.hint("broadcast").asInstanceOf[df.type]
}
@@ -7834,6 +7834,14 @@ object functions {
*/
def nullif(col1: Column, col2: Column): Column = Column.fn("nullif", col1, col2)
+ /**
+ * Returns null if `col` is equal to zero, or `col` otherwise.
+ *
+ * @group conditional_funcs
+ * @since 4.0.0
+ */
+ def nullifzero(col: Column): Column = Column.fn("nullifzero", col)
+
/**
* Returns `col2` if `col1` is null, or `col1` otherwise.
*
@@ -7850,6 +7858,14 @@ object functions {
*/
def nvl2(col1: Column, col2: Column, col3: Column): Column = Column.fn("nvl2", col1, col2, col3)
+ /**
+ * Returns zero if `col` is null, or `col` otherwise.
+ *
+ * @group conditional_funcs
+ * @since 4.0.0
+ */
+ def zeroifnull(col: Column): Column = Column.fn("zeroifnull", col)
+
// scalastyle:off line.size.limit
// scalastyle:off parameter.number
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/ProcedureCatalog.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/ProcedureCatalog.java
new file mode 100644
index 0000000000000..6eaacf340cb80
--- /dev/null
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/ProcedureCatalog.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.connector.catalog;
+
+import org.apache.spark.annotation.Evolving;
+import org.apache.spark.sql.connector.catalog.procedures.UnboundProcedure;
+
+/**
+ * A catalog API for working with procedures.
+ *
+ * @since 4.0.0
+ */
+@Evolving
+public interface ProcedureCatalog extends CatalogPlugin {
+ /**
+ * Load a procedure by {@link Identifier identifier} from the catalog.
+ *
+ * @param ident a procedure identifier
+ * @return the loaded unbound procedure
+ */
+ UnboundProcedure loadProcedure(Identifier ident);
+}
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/procedures/BoundProcedure.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/procedures/BoundProcedure.java
new file mode 100644
index 0000000000000..99f0836576f80
--- /dev/null
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/procedures/BoundProcedure.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.connector.catalog.procedures;
+
+import java.util.Iterator;
+
+import org.apache.spark.annotation.Evolving;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.read.LocalScan;
+import org.apache.spark.sql.connector.read.Scan;
+
+/**
+ * A procedure that is bound to input types.
+ *
+ * @since 4.0.0
+ */
+@Evolving
+public interface BoundProcedure extends Procedure {
+ /**
+ * Returns parameters of this procedure.
+ */
+ ProcedureParameter[] parameters();
+
+ /**
+ * Indicates whether this procedure is deterministic.
+ */
+ boolean isDeterministic();
+
+ /**
+ * Executes this procedure with the given input.
+ *
+ * Spark validates and rearranges arguments provided in the CALL statement to ensure that
+ * the order and data types of the fields in {@code input} matches the expected order and
+ * types defined by {@link #parameters() parameters}.
+ *
+ * Each procedure can return any number of result sets. Each result set is represented by
+ * a {@link Scan scan} that reports the type of records it produces and can be used to
+ * collect the output, if needed. If a result set is local and does not a distributed job,
+ * implementations should use {@link LocalScan}.
+ */
+ Iterator call(InternalRow input);
+}
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/procedures/Procedure.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/procedures/Procedure.java
new file mode 100644
index 0000000000000..4f88d215d3197
--- /dev/null
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/procedures/Procedure.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.connector.catalog.procedures;
+
+import org.apache.spark.annotation.Evolving;
+
+/**
+ * A base interface for all procedures.
+ *
+ * @since 4.0.0
+ */
+@Evolving
+public interface Procedure {
+ /**
+ * Returns the name of this procedure.
+ */
+ String name();
+
+ /**
+ * Returns the description of this procedure.
+ */
+ default String description() {
+ return getClass().toString();
+ }
+}
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/procedures/ProcedureParameter.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/procedures/ProcedureParameter.java
new file mode 100644
index 0000000000000..90d531ae21892
--- /dev/null
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/procedures/ProcedureParameter.java
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.connector.catalog.procedures;
+
+import javax.annotation.Nullable;
+
+import org.apache.spark.annotation.Evolving;
+import org.apache.spark.sql.internal.connector.ProcedureParameterImpl;
+import org.apache.spark.sql.types.DataType;
+
+import static org.apache.spark.sql.connector.catalog.procedures.ProcedureParameter.Mode.IN;
+
+/**
+ * A {@link Procedure procedure} parameter.
+ *
+ * @since 4.0.0
+ */
+@Evolving
+public interface ProcedureParameter {
+ /**
+ * Creates a builder for an IN procedure parameter.
+ *
+ * @param name the name of the parameter
+ * @param dataType the type of the parameter
+ * @return the constructed stored procedure parameter
+ */
+ static Builder in(String name, DataType dataType) {
+ return new Builder(IN, name, dataType);
+ }
+
+ /**
+ * Returns the mode of this parameter.
+ */
+ Mode mode();
+
+ /**
+ * Returns the name of this parameter.
+ */
+ String name();
+
+ /**
+ * Returns the data type of this parameter.
+ */
+ DataType dataType();
+
+ /**
+ * Returns the SQL string (Spark SQL dialect) of the default value expression of this parameter or
+ * null if not provided.
+ */
+ @Nullable
+ String defaultValueExpression();
+
+ /**
+ * Returns the comment of this parameter or null if not provided.
+ */
+ @Nullable
+ String comment();
+
+ /**
+ * An enum representing procedure parameter modes.
+ */
+ enum Mode {
+ IN,
+ INOUT,
+ OUT
+ }
+
+ class Builder {
+ private final Mode mode;
+ private final String name;
+ private final DataType dataType;
+ private String defaultValueExpression;
+ private String comment;
+
+ private Builder(Mode mode, String name, DataType dataType) {
+ this.mode = mode;
+ this.name = name;
+ this.dataType = dataType;
+ }
+
+ /**
+ * Sets the default value expression of the parameter.
+ */
+ public Builder defaultValue(String defaultValueExpression) {
+ this.defaultValueExpression = defaultValueExpression;
+ return this;
+ }
+
+ /**
+ * Sets the comment of the parameter.
+ */
+ public Builder comment(String comment) {
+ this.comment = comment;
+ return this;
+ }
+
+ /**
+ * Builds the stored procedure parameter.
+ */
+ public ProcedureParameter build() {
+ return new ProcedureParameterImpl(mode, name, dataType, defaultValueExpression, comment);
+ }
+ }
+}
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/procedures/UnboundProcedure.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/procedures/UnboundProcedure.java
new file mode 100644
index 0000000000000..ee9a09055243b
--- /dev/null
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/procedures/UnboundProcedure.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.connector.catalog.procedures;
+
+import org.apache.spark.annotation.Evolving;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * A procedure that is not bound to input types.
+ *
+ * @since 4.0.0
+ */
+@Evolving
+public interface UnboundProcedure extends Procedure {
+ /**
+ * Binds this procedure to input types.
+ *
+ * If the catalog supports procedure overloading, the implementation is expected to pick the best
+ * matching version of the procedure. If overloading is not supported, the implementation can
+ * validate if the input types are compatible while binding or delegate that to Spark. Regardless,
+ * Spark will always perform the final validation of the arguments and rearrange them as needed
+ * based on {@link BoundProcedure#parameters() reported parameters}.
+ *
+ * @param inputType the input types to bind to
+ * @return the bound procedure that is most suitable for the given input types
+ */
+ BoundProcedure bind(StructType inputType);
+}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/connector/ProcedureParameterImpl.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/connector/ProcedureParameterImpl.scala
new file mode 100644
index 0000000000000..01ea48af1537c
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/connector/ProcedureParameterImpl.scala
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.internal.connector
+
+import org.apache.spark.sql.connector.catalog.procedures.ProcedureParameter
+import org.apache.spark.sql.connector.catalog.procedures.ProcedureParameter.Mode
+import org.apache.spark.sql.types.DataType
+
+case class ProcedureParameterImpl(
+ mode: Mode,
+ name: String,
+ dataType: DataType,
+ defaultValueExpression: String,
+ comment: String) extends ProcedureParameter
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala
index a236577cba438..358541c942f1f 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala
@@ -17,9 +17,7 @@
package org.apache.spark.sql
-import java.io.Closeable
import java.util.{ServiceLoader, UUID}
-import java.util.concurrent.TimeUnit._
import java.util.concurrent.atomic.{AtomicBoolean, AtomicReference}
import scala.jdk.CollectionConverters._
@@ -93,7 +91,7 @@ class SparkSession private(
@transient private val parentSessionState: Option[SessionState],
@transient private[sql] val extensions: SparkSessionExtensions,
@transient private[sql] val initialSessionOptions: Map[String, String])
- extends Serializable with Closeable with Logging { self =>
+ extends api.SparkSession[Dataset] with Logging { self =>
// The call site where this SparkSession was constructed.
private val creationSite: CallSite = Utils.getCallSite()
@@ -122,11 +120,7 @@ class SparkSession private(
.getOrElse(SQLConf.getFallbackConf)
})
- /**
- * The version of Spark on which this application is running.
- *
- * @since 2.0.0
- */
+ /** @inheritdoc */
def version: String = SPARK_VERSION
/* ----------------------- *
@@ -207,27 +201,7 @@ class SparkSession private(
@Unstable
def experimental: ExperimentalMethods = sessionState.experimentalMethods
- /**
- * A collection of methods for registering user-defined functions (UDF).
- *
- * The following example registers a Scala closure as UDF:
- * {{{
- * sparkSession.udf.register("myUDF", (arg1: Int, arg2: String) => arg2 + arg1)
- * }}}
- *
- * The following example registers a UDF in Java:
- * {{{
- * sparkSession.udf().register("myUDF",
- * (Integer arg1, String arg2) -> arg2 + arg1,
- * 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.
- *
- * @since 2.0.0
- */
+ /** @inheritdoc */
def udf: UDFRegistration = sessionState.udfRegistration
private[sql] def udtf: UDTFRegistration = sessionState.udtfRegistration
@@ -260,17 +234,7 @@ class SparkSession private(
@Unstable
private[sql] def artifactManager: ArtifactManager = sessionState.artifactManager
- /**
- * 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
- */
+ /** @inheritdoc */
def newSession(): SparkSession = {
new SparkSession(
sparkContext,
@@ -308,19 +272,11 @@ class SparkSession private(
| Methods for creating DataFrames |
* --------------------------------- */
- /**
- * Returns a `DataFrame` with no rows or columns.
- *
- * @since 2.0.0
- */
+ /** @inheritdoc */
@transient
lazy val emptyDataFrame: DataFrame = Dataset.ofRows(self, LocalRelation())
- /**
- * Creates a new [[Dataset]] of type T containing zero elements.
- *
- * @since 2.0.0
- */
+ /** @inheritdoc */
def emptyDataset[T: Encoder]: Dataset[T] = {
val encoder = implicitly[Encoder[T]]
new Dataset(self, LocalRelation(encoder.schema), encoder)
@@ -336,11 +292,7 @@ class SparkSession private(
Dataset.ofRows(self, ExternalRDD(rdd, self)(encoder))
}
- /**
- * Creates a `DataFrame` from a local Seq of Product.
- *
- * @since 2.0.0
- */
+ /** @inheritdoc */
def createDataFrame[A <: Product : TypeTag](data: Seq[A]): DataFrame = withActive {
val schema = ScalaReflection.schemaFor[A].dataType.asInstanceOf[StructType]
val attributeSeq = toAttributes(schema)
@@ -403,14 +355,7 @@ class SparkSession private(
createDataFrame(rowRDD.rdd, replaced)
}
- /**
- * :: DeveloperApi ::
- * Creates a `DataFrame` from a `java.util.List` containing [[Row]]s using the given schema.
- * It is important to make sure that the structure of every [[Row]] of the provided List matches
- * the provided schema. Otherwise, there will be runtime exception.
- *
- * @since 2.0.0
- */
+ /** @inheritdoc */
@DeveloperApi
def createDataFrame(rows: java.util.List[Row], schema: StructType): DataFrame = withActive {
val replaced = CharVarcharUtils.failIfHasCharVarchar(schema).asInstanceOf[StructType]
@@ -447,13 +392,7 @@ class SparkSession private(
createDataFrame(rdd.rdd, beanClass)
}
- /**
- * 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.
- * @since 1.6.0
- */
+ /** @inheritdoc */
def createDataFrame(data: java.util.List[_], beanClass: Class[_]): DataFrame = withActive {
val attrSeq = getSchema(beanClass)
val rows = SQLContext.beansToRows(data.asScala.iterator, beanClass, attrSeq)
@@ -473,33 +412,7 @@ class SparkSession private(
| Methods for creating DataSets |
* ------------------------------- */
- /**
- * 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]].
- *
- * == Example ==
- *
- * {{{
- *
- * import spark.implicits._
- * case class Person(name: String, age: Long)
- * val data = Seq(Person("Michael", 29), Person("Andy", 30), Person("Justin", 19))
- * val ds = spark.createDataset(data)
- *
- * ds.show()
- * // +-------+---+
- * // | name|age|
- * // +-------+---+
- * // |Michael| 29|
- * // | Andy| 30|
- * // | Justin| 19|
- * // +-------+---+
- * }}}
- *
- * @since 2.0.0
- */
+ /** @inheritdoc */
def createDataset[T : Encoder](data: Seq[T]): Dataset[T] = {
val enc = encoderFor[T]
val toRow = enc.createSerializer()
@@ -521,60 +434,25 @@ class SparkSession private(
Dataset[T](self, ExternalRDD(data, self))
}
- /**
- * 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]].
- *
- * == Java Example ==
- *
- * {{{
- * List data = Arrays.asList("hello", "world");
- * Dataset ds = spark.createDataset(data, Encoders.STRING());
- * }}}
- *
- * @since 2.0.0
- */
+ /** @inheritdoc */
def createDataset[T : Encoder](data: java.util.List[T]): Dataset[T] = {
createDataset(data.asScala.toSeq)
}
- /**
- * 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
- */
+ /** @inheritdoc */
def range(end: Long): Dataset[java.lang.Long] = range(0, end)
- /**
- * 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
- */
+ /** @inheritdoc */
def range(start: Long, end: Long): Dataset[java.lang.Long] = {
range(start, end, step = 1, numPartitions = leafNodeDefaultParallelism)
}
- /**
- * 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
- */
+ /** @inheritdoc */
def range(start: Long, end: Long, step: Long): Dataset[java.lang.Long] = {
range(start, end, step, numPartitions = leafNodeDefaultParallelism)
}
- /**
- * 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
- */
+ /** @inheritdoc */
def range(start: Long, end: Long, step: Long, numPartitions: Int): Dataset[java.lang.Long] = {
new Dataset(self, Range(start, end, step, numPartitions), Encoders.LONG)
}
@@ -608,19 +486,7 @@ class SparkSession private(
*/
@transient lazy val catalog: Catalog = new CatalogImpl(self)
- /**
- * 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.
- * @since 2.0.0
- */
+ /** @inheritdoc */
def table(tableName: String): DataFrame = {
read.table(tableName)
}
@@ -661,22 +527,7 @@ class SparkSession private(
Dataset.ofRows(self, plan, tracker)
}
- /**
- * 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
- */
+ /** @inheritdoc */
@Experimental
def sql(sqlText: String, args: Array[_]): DataFrame = {
sql(sqlText, args, new QueryPlanningTracker)
@@ -714,57 +565,20 @@ class SparkSession private(
Dataset.ofRows(self, plan, tracker)
}
- /**
- * 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
- */
+ /** @inheritdoc */
@Experimental
def sql(sqlText: String, args: Map[String, Any]): DataFrame = {
sql(sqlText, args, new QueryPlanningTracker)
}
- /**
- * 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
- */
+ /** @inheritdoc */
@Experimental
- def sql(sqlText: String, args: java.util.Map[String, Any]): DataFrame = {
+ override def sql(sqlText: String, args: java.util.Map[String, Any]): DataFrame = {
sql(sqlText, args.asScala.toMap)
}
- /**
- * 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
- */
- def sql(sqlText: String): DataFrame = sql(sqlText, Map.empty[String, Any])
+ /** @inheritdoc */
+ override def sql(sqlText: String): DataFrame = sql(sqlText, Map.empty[String, Any])
/**
* Execute an arbitrary string command inside an external execution engine rather than Spark.
@@ -817,22 +631,6 @@ class SparkSession private(
*/
def readStream: DataStreamReader = new DataStreamReader(self)
- /**
- * Executes some code block and prints to stdout the time taken to execute the block. This is
- * available in Scala only and is used primarily for interactive testing and debugging.
- *
- * @since 2.1.0
- */
- def time[T](f: => T): T = {
- val start = System.nanoTime()
- val ret = f
- val end = System.nanoTime()
- // scalastyle:off println
- println(s"Time taken: ${NANOSECONDS.toMillis(end - start)} ms")
- // scalastyle:on println
- ret
- }
-
// scalastyle:off
// Disable style checker so "implicits" object can start with lowercase i
/**
@@ -854,19 +652,12 @@ class SparkSession private(
/**
* Stop the underlying `SparkContext`.
*
- * @since 2.0.0
+ * @since 2.1.0
*/
- def stop(): Unit = {
+ override def close(): Unit = {
sparkContext.stop()
}
- /**
- * Synonym for `stop()`.
- *
- * @since 2.1.0
- */
- override def close(): Unit = stop()
-
/**
* Parses the data type in our internal string representation. The data type string should
* have the same format as the one generated by `toString` in scala.
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala
index 3ae76a1db22b2..5d59a48d544a0 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala
@@ -138,9 +138,8 @@ trait HashJoin extends JoinCodegenSupport {
UnsafeProjection.create(streamedBoundKeys)
@transient protected[this] lazy val boundCondition = if (condition.isDefined) {
- if (joinType == FullOuter && buildSide == BuildLeft) {
- // Put join left side before right side. This is to be consistent with
- // `ShuffledHashJoinExec.fullOuterJoin`.
+ if ((joinType == FullOuter || joinType == LeftOuter) && buildSide == BuildLeft) {
+ // Put join left side before right side.
Predicate.create(condition.get, buildPlan.output ++ streamedPlan.output).eval _
} else {
Predicate.create(condition.get, streamedPlan.output ++ buildPlan.output).eval _
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala
index cf8dcc0b8b2f0..739cef035c38c 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala
@@ -23,10 +23,10 @@ import java.sql.{Date, Timestamp}
import scala.util.Random
-import org.apache.spark.{SPARK_DOC_ROOT, SparkException, SparkRuntimeException}
+import org.apache.spark.{QueryContextType, SPARK_DOC_ROOT, SparkException, SparkRuntimeException}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.analysis.FunctionRegistry
-import org.apache.spark.sql.catalyst.expressions.{Expression, UnaryExpression}
+import org.apache.spark.sql.catalyst.expressions.{Expression, Literal, UnaryExpression}
import org.apache.spark.sql.catalyst.expressions.Cast._
import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
import org.apache.spark.sql.catalyst.plans.logical.OneRowRelation
@@ -331,6 +331,66 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession {
checkAnswer(df.select(nullif(lit(5), lit(5))), Seq(Row(null)))
}
+ test("nullifzero function") {
+ withTable("t") {
+ // Here we exercise a non-nullable, non-foldable column.
+ sql("create table t(col int not null) using csv")
+ sql("insert into t values (0)")
+ val df = sql("select col from t")
+ checkAnswer(df.select(nullifzero($"col")), Seq(Row(null)))
+ }
+ // Here we exercise invalid cases including types that do not support ordering.
+ val df = Seq((0)).toDF("a")
+ var expr = nullifzero(map(lit(1), lit("a")))
+ checkError(
+ intercept[AnalysisException](df.select(expr)),
+ errorClass = "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ parameters = Map(
+ "left" -> "\"MAP\"",
+ "right" -> "\"INT\"",
+ "sqlExpr" -> "\"(map(1, a) = 0)\""),
+ context = ExpectedContext(
+ contextType = QueryContextType.DataFrame,
+ fragment = "nullifzero",
+ objectType = "",
+ objectName = "",
+ callSitePattern = "",
+ startIndex = 0,
+ stopIndex = 0))
+ expr = nullifzero(array(lit(1), lit(2)))
+ checkError(
+ intercept[AnalysisException](df.select(expr)),
+ errorClass = "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ parameters = Map(
+ "left" -> "\"ARRAY\"",
+ "right" -> "\"INT\"",
+ "sqlExpr" -> "\"(array(1, 2) = 0)\""),
+ context = ExpectedContext(
+ contextType = QueryContextType.DataFrame,
+ fragment = "nullifzero",
+ objectType = "",
+ objectName = "",
+ callSitePattern = "",
+ startIndex = 0,
+ stopIndex = 0))
+ expr = nullifzero(Literal.create(20201231, DateType))
+ checkError(
+ intercept[AnalysisException](df.select(expr)),
+ errorClass = "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ parameters = Map(
+ "left" -> "\"DATE\"",
+ "right" -> "\"INT\"",
+ "sqlExpr" -> "\"(DATE '+57279-02-03' = 0)\""),
+ context = ExpectedContext(
+ contextType = QueryContextType.DataFrame,
+ fragment = "nullifzero",
+ objectType = "",
+ objectName = "",
+ callSitePattern = "",
+ startIndex = 0,
+ stopIndex = 0))
+ }
+
test("nvl") {
val df = Seq[(Integer, Integer)]((null, 8)).toDF("a", "b")
checkAnswer(df.selectExpr("nvl(a, b)"), Seq(Row(8)))
@@ -349,6 +409,66 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession {
checkAnswer(df.select(nvl2(col("b"), col("a"), col("c"))), Seq(Row(null)))
}
+ test("zeroifnull function") {
+ withTable("t") {
+ // Here we exercise a non-nullable, non-foldable column.
+ sql("create table t(col int not null) using csv")
+ sql("insert into t values (0)")
+ val df = sql("select col from t")
+ checkAnswer(df.select(zeroifnull($"col")), Seq(Row(0)))
+ }
+ // Here we exercise invalid cases including types that do not support ordering.
+ val df = Seq((0)).toDF("a")
+ var expr = zeroifnull(map(lit(1), lit("a")))
+ checkError(
+ intercept[AnalysisException](df.select(expr)),
+ errorClass = "DATATYPE_MISMATCH.DATA_DIFF_TYPES",
+ parameters = Map(
+ "functionName" -> "`coalesce`",
+ "dataType" -> "(\"MAP\" or \"INT\")",
+ "sqlExpr" -> "\"coalesce(map(1, a), 0)\""),
+ context = ExpectedContext(
+ contextType = QueryContextType.DataFrame,
+ fragment = "zeroifnull",
+ objectType = "",
+ objectName = "",
+ callSitePattern = "",
+ startIndex = 0,
+ stopIndex = 0))
+ expr = zeroifnull(array(lit(1), lit(2)))
+ checkError(
+ intercept[AnalysisException](df.select(expr)),
+ errorClass = "DATATYPE_MISMATCH.DATA_DIFF_TYPES",
+ parameters = Map(
+ "functionName" -> "`coalesce`",
+ "dataType" -> "(\"ARRAY\" or \"INT\")",
+ "sqlExpr" -> "\"coalesce(array(1, 2), 0)\""),
+ context = ExpectedContext(
+ contextType = QueryContextType.DataFrame,
+ fragment = "zeroifnull",
+ objectType = "",
+ objectName = "",
+ callSitePattern = "",
+ startIndex = 0,
+ stopIndex = 0))
+ expr = zeroifnull(Literal.create(20201231, DateType))
+ checkError(
+ intercept[AnalysisException](df.select(expr)),
+ errorClass = "DATATYPE_MISMATCH.DATA_DIFF_TYPES",
+ parameters = Map(
+ "functionName" -> "`coalesce`",
+ "dataType" -> "(\"DATE\" or \"INT\")",
+ "sqlExpr" -> "\"coalesce(DATE '+57279-02-03', 0)\""),
+ context = ExpectedContext(
+ contextType = QueryContextType.DataFrame,
+ fragment = "zeroifnull",
+ objectType = "",
+ objectName = "",
+ callSitePattern = "",
+ startIndex = 0,
+ stopIndex = 0))
+ }
+
test("misc md5 function") {
val df = Seq(("ABC", Array[Byte](1, 2, 3, 4, 5, 6))).toDF("a", "b")
checkAnswer(
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala
index e4ea88067c7c2..7ba93ee13e182 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala
@@ -26,11 +26,12 @@ import org.apache.spark.sql.catalyst.plans.logical.{Join, JoinHint}
import org.apache.spark.sql.execution.{SparkPlan, SparkPlanTest}
import org.apache.spark.sql.execution.exchange.EnsureRequirements
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.test.{SharedSparkSession, SQLTestData}
import org.apache.spark.sql.types.{DoubleType, IntegerType, StructType}
-class OuterJoinSuite extends SparkPlanTest with SharedSparkSession {
+class OuterJoinSuite extends SparkPlanTest with SharedSparkSession with SQLTestData {
import testImplicits.toRichColumn
+ setupTestData()
private val EnsureRequirements = new EnsureRequirements()
@@ -326,4 +327,21 @@ class OuterJoinSuite extends SparkPlanTest with SharedSparkSession {
(null, null, 7, 7.0)
)
)
+
+ testWithWholeStageCodegenOnAndOff(
+ "SPARK-46037: ShuffledHashJoin build left with left outer join, codegen off") { _ =>
+ def join(hint: String): DataFrame = {
+ sql(
+ s"""
+ |SELECT /*+ $hint */ *
+ |FROM testData t1
+ |LEFT OUTER JOIN
+ |testData2 t2
+ |ON key = a AND concat(value, b) = '12'
+ |""".stripMargin)
+ }
+ val df1 = join("SHUFFLE_HASH(t1)")
+ val df2 = join("SHUFFLE_MERGE(t1)")
+ checkAnswer(df1, identity, df2.collect().toSeq)
+ }
}