diff --git a/.github/workflows/publish_snapshot.yml b/.github/workflows/publish_snapshot.yml index 3354ab88a395e..c78b633a980c5 100644 --- a/.github/workflows/publish_snapshot.yml +++ b/.github/workflows/publish_snapshot.yml @@ -70,12 +70,5 @@ jobs: GPG_KEY: "not_used" GPG_PASSPHRASE: "not_used" GIT_REF: ${{ matrix.branch }} - run: | - while true - do - date - top -b -n 1 -i - sleep 1 - echo - done | sed "s/^/mem: /" & - ./dev/create-release/release-build.sh publish-snapshot + MAVEN_MXM_OPT: 2g + run: ./dev/create-release/release-build.sh publish-snapshot diff --git a/binder/postBuild b/binder/postBuild index b6bdf72324c90..b1eed6a8b5575 100644 --- a/binder/postBuild +++ b/binder/postBuild @@ -20,6 +20,11 @@ # This file is used for Binder integration to install PySpark available in # Jupyter notebook. +# SPARK-45706: Should fail fast. Otherwise, the Binder image is successfully +# built, and it cannot be rebuilt. +set -o pipefail +set -e + VERSION=$(python -c "exec(open('python/pyspark/version.py').read()); print(__version__)") TAG=$(git describe --tags --exact-match 2>/dev/null) diff --git a/common/utils/src/main/resources/error/error-classes.json b/common/utils/src/main/resources/error/error-classes.json index 2ab4af73a8e79..f9cc0a8652161 100644 --- a/common/utils/src/main/resources/error/error-classes.json +++ b/common/utils/src/main/resources/error/error-classes.json @@ -1974,6 +1974,11 @@ "expects one of the units without quotes YEAR, QUARTER, MONTH, WEEK, DAY, DAYOFYEAR, HOUR, MINUTE, SECOND, MILLISECOND, MICROSECOND, but got the string literal ." ] }, + "LENGTH" : { + "message" : [ + "Expects `length` greater than or equal to 0, but got ." + ] + }, "NULL" : { "message" : [ "expects a non-NULL value." @@ -1989,6 +1994,11 @@ "Expects group index between 0 and , but got ." ] }, + "START" : { + "message" : [ + "Expects a positive or a negative value for `start`, but got 0." + ] + }, "ZERO_INDEX" : { "message" : [ "expects %1$, %2$ and so on, but got %0$." @@ -2192,6 +2202,12 @@ ], "sqlState" : "42K0F" }, + "INVALID_TIME_TRAVEL_SPEC" : { + "message" : [ + "Cannot specify both version and timestamp when time travelling the table." + ], + "sqlState" : "42K0E" + }, "INVALID_TIME_TRAVEL_TIMESTAMP_EXPR" : { "message" : [ "The time travel timestamp expression is invalid." @@ -2207,6 +2223,11 @@ "Must be deterministic." ] }, + "OPTION" : { + "message" : [ + "Timestamp string in the options must be able to cast to TIMESTAMP type." + ] + }, "UNEVALUABLE" : { "message" : [ "Must be evaluable." @@ -2386,6 +2407,12 @@ ], "sqlState" : "42803" }, + "MULTIPLE_TIME_TRAVEL_SPEC" : { + "message" : [ + "Cannot specify time travel in both the time travel clause and options." + ], + "sqlState" : "42K0E" + }, "MULTI_SOURCES_UNSUPPORTED_FOR_EXPRESSION" : { "message" : [ "The expression does not support more than one source." @@ -5132,11 +5159,6 @@ "" ] }, - "_LEGACY_ERROR_TEMP_1334" : { - "message" : [ - "Cannot specify both version and timestamp when time travelling the table." - ] - }, "_LEGACY_ERROR_TEMP_1338" : { "message" : [ "Sinks cannot request distribution and ordering in continuous execution mode." @@ -5737,21 +5759,6 @@ " is not annotated with SQLUserDefinedType nor registered with UDTRegistration.}" ] }, - "_LEGACY_ERROR_TEMP_2156" : { - "message" : [ - "The size function doesn't support the operand type ." - ] - }, - "_LEGACY_ERROR_TEMP_2157" : { - "message" : [ - "Unexpected value for start in function : SQL array indices start at 1." - ] - }, - "_LEGACY_ERROR_TEMP_2158" : { - "message" : [ - "Unexpected value for length in function : length must be greater than or equal to 0." - ] - }, "_LEGACY_ERROR_TEMP_2159" : { "message" : [ "Unsuccessful try to concat arrays with elements due to exceeding the array size limit ." diff --git a/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala b/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala index 59f2999bdd395..2c2a45fc3f14f 100644 --- a/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala +++ b/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala @@ -61,7 +61,8 @@ private[sql] case class AvroDataToCatalyst( @transient private lazy val reader = new GenericDatumReader[Any](actualSchema, expectedSchema) @transient private lazy val deserializer = - new AvroDeserializer(expectedSchema, dataType, avroOptions.datetimeRebaseModeInRead) + new AvroDeserializer(expectedSchema, dataType, + avroOptions.datetimeRebaseModeInRead, avroOptions.useStableIdForUnionType) @transient private var decoder: BinaryDecoder = _ diff --git a/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala b/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala index d9c8fba3b8766..c04fe820f0bef 100644 --- a/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala +++ b/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala @@ -49,18 +49,21 @@ private[sql] class AvroDeserializer( rootCatalystType: DataType, positionalFieldMatch: Boolean, datetimeRebaseSpec: RebaseSpec, - filters: StructFilters) { + filters: StructFilters, + useStableIdForUnionType: Boolean) { def this( rootAvroType: Schema, rootCatalystType: DataType, - datetimeRebaseMode: String) = { + datetimeRebaseMode: String, + useStableIdForUnionType: Boolean) = { this( rootAvroType, rootCatalystType, positionalFieldMatch = false, RebaseSpec(LegacyBehaviorPolicy.withName(datetimeRebaseMode)), - new NoopFilters) + new NoopFilters, + useStableIdForUnionType) } private lazy val decimalConversions = new DecimalConversion() @@ -118,7 +121,7 @@ private[sql] class AvroDeserializer( val incompatibleMsg = errorPrefix + s"schema is incompatible (avroType = $avroType, sqlType = ${catalystType.sql})" - val realDataType = SchemaConverters.toSqlType(avroType).dataType + val realDataType = SchemaConverters.toSqlType(avroType, useStableIdForUnionType).dataType val confKey = SQLConf.LEGACY_AVRO_ALLOW_INCOMPATIBLE_SCHEMA val preventReadingIncorrectType = !SQLConf.get.getConf(confKey) diff --git a/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala b/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala index 53562a3afdb5b..7b0292df43c2f 100755 --- a/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala +++ b/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala @@ -141,7 +141,8 @@ private[sql] class AvroFileFormat extends FileFormat requiredSchema, parsedOptions.positionalFieldMatching, datetimeRebaseMode, - avroFilters) + avroFilters, + parsedOptions.useStableIdForUnionType) override val stopPosition = file.start + file.length override def hasNext: Boolean = hasNextRow diff --git a/connector/avro/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala b/connector/avro/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala index 90d09aa1e41ed..ba01a18d76f79 100644 --- a/connector/avro/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala +++ b/connector/avro/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala @@ -46,16 +46,24 @@ object SchemaConverters { */ case class SchemaType(dataType: DataType, nullable: Boolean) + /** + * Converts an Avro schema to a corresponding Spark SQL schema. + * + * @since 4.0.0 + */ + def toSqlType(avroSchema: Schema, useStableIdForUnionType: Boolean): SchemaType = { + toSqlTypeHelper(avroSchema, Set.empty, useStableIdForUnionType) + } /** * Converts an Avro schema to a corresponding Spark SQL schema. * * @since 2.4.0 */ def toSqlType(avroSchema: Schema): SchemaType = { - toSqlTypeHelper(avroSchema, Set.empty, AvroOptions(Map())) + toSqlType(avroSchema, false) } def toSqlType(avroSchema: Schema, options: Map[String, String]): SchemaType = { - toSqlTypeHelper(avroSchema, Set.empty, AvroOptions(options)) + toSqlTypeHelper(avroSchema, Set.empty, AvroOptions(options).useStableIdForUnionType) } // The property specifies Catalyst type of the given field @@ -64,7 +72,7 @@ object SchemaConverters { private def toSqlTypeHelper( avroSchema: Schema, existingRecordNames: Set[String], - avroOptions: AvroOptions): SchemaType = { + useStableIdForUnionType: Boolean): SchemaType = { avroSchema.getType match { case INT => avroSchema.getLogicalType match { case _: Date => SchemaType(DateType, nullable = false) @@ -117,7 +125,7 @@ object SchemaConverters { } val newRecordNames = existingRecordNames + avroSchema.getFullName val fields = avroSchema.getFields.asScala.map { f => - val schemaType = toSqlTypeHelper(f.schema(), newRecordNames, avroOptions) + val schemaType = toSqlTypeHelper(f.schema(), newRecordNames, useStableIdForUnionType) StructField(f.name, schemaType.dataType, schemaType.nullable) } @@ -127,13 +135,14 @@ object SchemaConverters { val schemaType = toSqlTypeHelper( avroSchema.getElementType, existingRecordNames, - avroOptions) + useStableIdForUnionType) SchemaType( ArrayType(schemaType.dataType, containsNull = schemaType.nullable), nullable = false) case MAP => - val schemaType = toSqlTypeHelper(avroSchema.getValueType, existingRecordNames, avroOptions) + val schemaType = toSqlTypeHelper(avroSchema.getValueType, + existingRecordNames, useStableIdForUnionType) SchemaType( MapType(StringType, schemaType.dataType, valueContainsNull = schemaType.nullable), nullable = false) @@ -143,17 +152,18 @@ object SchemaConverters { // In case of a union with null, eliminate it and make a recursive call val remainingUnionTypes = AvroUtils.nonNullUnionBranches(avroSchema) if (remainingUnionTypes.size == 1) { - toSqlTypeHelper(remainingUnionTypes.head, existingRecordNames, avroOptions) + toSqlTypeHelper(remainingUnionTypes.head, existingRecordNames, useStableIdForUnionType) .copy(nullable = true) } else { toSqlTypeHelper( Schema.createUnion(remainingUnionTypes.asJava), existingRecordNames, - avroOptions).copy(nullable = true) + useStableIdForUnionType).copy(nullable = true) } } else avroSchema.getTypes.asScala.map(_.getType).toSeq match { case Seq(t1) => - toSqlTypeHelper(avroSchema.getTypes.get(0), existingRecordNames, avroOptions) + toSqlTypeHelper(avroSchema.getTypes.get(0), + existingRecordNames, useStableIdForUnionType) case Seq(t1, t2) if Set(t1, t2) == Set(INT, LONG) => SchemaType(LongType, nullable = false) case Seq(t1, t2) if Set(t1, t2) == Set(FLOAT, DOUBLE) => @@ -167,9 +177,9 @@ object SchemaConverters { val fieldNameSet : mutable.Set[String] = mutable.Set() val fields = avroSchema.getTypes.asScala.zipWithIndex.map { case (s, i) => - val schemaType = toSqlTypeHelper(s, existingRecordNames, avroOptions) + val schemaType = toSqlTypeHelper(s, existingRecordNames, useStableIdForUnionType) - val fieldName = if (avroOptions.useStableIdForUnionType) { + val fieldName = if (useStableIdForUnionType) { // Avro's field name may be case sensitive, so field names for two named type // could be "a" and "A" and we need to distinguish them. In this case, we throw // an exception. diff --git a/connector/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroPartitionReaderFactory.scala b/connector/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroPartitionReaderFactory.scala index cc7bd180e8477..2c85c1b067392 100644 --- a/connector/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroPartitionReaderFactory.scala +++ b/connector/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroPartitionReaderFactory.scala @@ -103,7 +103,8 @@ case class AvroPartitionReaderFactory( readDataSchema, options.positionalFieldMatching, datetimeRebaseMode, - avroFilters) + avroFilters, + options.useStableIdForUnionType) override val stopPosition = partitionedFile.start + partitionedFile.length override def next(): Boolean = hasNextRow diff --git a/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroCatalystDataConversionSuite.scala b/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroCatalystDataConversionSuite.scala index 1cb34a0bc4dc5..250b5e0615ad8 100644 --- a/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroCatalystDataConversionSuite.scala +++ b/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroCatalystDataConversionSuite.scala @@ -59,7 +59,7 @@ class AvroCatalystDataConversionSuite extends SparkFunSuite val expected = { val avroSchema = new Schema.Parser().parse(schema) - SchemaConverters.toSqlType(avroSchema).dataType match { + SchemaConverters.toSqlType(avroSchema, false).dataType match { case st: StructType => Row.fromSeq((0 until st.length).map(_ => null)) case _ => null } @@ -281,13 +281,14 @@ class AvroCatalystDataConversionSuite extends SparkFunSuite data: GenericData.Record, expected: Option[Any], filters: StructFilters = new NoopFilters): Unit = { - val dataType = SchemaConverters.toSqlType(schema).dataType + val dataType = SchemaConverters.toSqlType(schema, false).dataType val deserializer = new AvroDeserializer( schema, dataType, false, RebaseSpec(LegacyBehaviorPolicy.CORRECTED), - filters) + filters, + false) val deserialized = deserializer.deserialize(data) expected match { case None => assert(deserialized == None) diff --git a/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroRowReaderSuite.scala b/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroRowReaderSuite.scala index 1c450d4f16674..7117ef4b21e83 100644 --- a/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroRowReaderSuite.scala +++ b/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroRowReaderSuite.scala @@ -75,7 +75,8 @@ class AvroRowReaderSuite StructType(new StructField("value", IntegerType, true) :: Nil), false, RebaseSpec(CORRECTED), - new NoopFilters) + new NoopFilters, + false) override val stopPosition = fileSize override def hasNext: Boolean = hasNextRow diff --git a/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroSerdeSuite.scala b/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroSerdeSuite.scala index d9d20b8732fbb..c6d0398017ef3 100644 --- a/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroSerdeSuite.scala +++ b/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroSerdeSuite.scala @@ -226,7 +226,8 @@ object AvroSerdeSuite { sql, isPositional(matchType), RebaseSpec(CORRECTED), - new NoopFilters) + new NoopFilters, + false) } /** diff --git a/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala b/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala index a08452c88308a..3a25d4d9f709a 100644 --- a/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala +++ b/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala @@ -2147,7 +2147,7 @@ abstract class AvroSuite private def checkSchemaWithRecursiveLoop(avroSchema: String): Unit = { val message = intercept[IncompatibleSchemaException] { - SchemaConverters.toSqlType(new Schema.Parser().parse(avroSchema)) + SchemaConverters.toSqlType(new Schema.Parser().parse(avroSchema), false) }.getMessage assert(message.contains("Found recursive reference in Avro schema")) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala index e60bda0a838e3..5e640bea57000 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -39,7 +39,6 @@ import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{BoxedLongEncoder import org.apache.spark.sql.connect.client.{ClassFinder, SparkConnectClient, SparkResult} import org.apache.spark.sql.connect.client.SparkConnectClient.Configuration import org.apache.spark.sql.connect.client.arrow.ArrowSerializer -import org.apache.spark.sql.connect.client.util.Cleaner import org.apache.spark.sql.functions.lit import org.apache.spark.sql.internal.{CatalogImpl, SqlApiConf} import org.apache.spark.sql.streaming.DataStreamReader @@ -66,7 +65,6 @@ import org.apache.spark.sql.types.StructType */ class SparkSession private[sql] ( private[sql] val client: SparkConnectClient, - private val cleaner: Cleaner, private val planIdGenerator: AtomicLong) extends Serializable with Closeable @@ -536,7 +534,6 @@ class SparkSession private[sql] ( private[sql] def execute[T](plan: proto.Plan, encoder: AgnosticEncoder[T]): SparkResult[T] = { val value = client.execute(plan) val result = new SparkResult(value, allocator, encoder, timeZoneId) - cleaner.register(result) result } @@ -774,7 +771,7 @@ object SparkSession extends Logging { * Create a new [[SparkSession]] based on the connect client [[Configuration]]. */ private[sql] def create(configuration: Configuration): SparkSession = { - new SparkSession(configuration.toSparkConnectClient, cleaner, planIdGenerator) + new SparkSession(configuration.toSparkConnectClient, planIdGenerator) } /** @@ -795,12 +792,6 @@ object SparkSession extends Logging { */ def builder(): Builder = new Builder() - private[sql] lazy val cleaner = { - val cleaner = new Cleaner - cleaner.start() - cleaner - } - class Builder() extends Logging { // Initialize the connection string of the Spark Connect client builder from SPARK_REMOTE // by default, if it exists. The connection string can be overridden using @@ -911,7 +902,7 @@ object SparkSession extends Logging { private def tryCreateSessionFromClient(): Option[SparkSession] = { if (client != null) { - Option(new SparkSession(client, cleaner, planIdGenerator)) + Option(new SparkSession(client, planIdGenerator)) } else { None } diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientDatasetSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientDatasetSuite.scala index aab31d97e8c9d..041b092836588 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientDatasetSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientDatasetSuite.scala @@ -43,7 +43,7 @@ class ClientDatasetSuite extends ConnectFunSuite with BeforeAndAfterEach { private def newSparkSession(): SparkSession = { val client = SparkConnectClient( InProcessChannelBuilder.forName(getClass.getName).directExecutor().build()) - new SparkSession(client, cleaner = SparkSession.cleaner, planIdGenerator = new AtomicLong) + new SparkSession(client, planIdGenerator = new AtomicLong) } private def startDummyServer(): Unit = { diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/PlanGenerationTestSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/PlanGenerationTestSuite.scala index e4c5b851b1306..cf287088b59fb 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/PlanGenerationTestSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/PlanGenerationTestSuite.scala @@ -112,8 +112,7 @@ class PlanGenerationTestSuite override protected def beforeAll(): Unit = { super.beforeAll() val client = SparkConnectClient(InProcessChannelBuilder.forName("/dev/null").build()) - session = - new SparkSession(client, cleaner = SparkSession.cleaner, planIdGenerator = new AtomicLong) + session = new SparkSession(client, planIdGenerator = new AtomicLong) } override protected def beforeEach(): Unit = { diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/SQLImplicitsTestSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/SQLImplicitsTestSuite.scala index 53743feb03bbc..b2c13850a13a0 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/SQLImplicitsTestSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/SQLImplicitsTestSuite.scala @@ -39,8 +39,7 @@ class SQLImplicitsTestSuite extends ConnectFunSuite with BeforeAndAfterAll { override protected def beforeAll(): Unit = { super.beforeAll() val client = SparkConnectClient(InProcessChannelBuilder.forName("/dev/null").build()) - session = - new SparkSession(client, cleaner = SparkSession.cleaner, planIdGenerator = new AtomicLong) + session = new SparkSession(client, planIdGenerator = new AtomicLong) } test("column resolution") { diff --git a/connector/connect/common/src/main/protobuf/spark/connect/commands.proto b/connector/connect/common/src/main/protobuf/spark/connect/commands.proto index 89a82c2c7e8fc..d120fb7b530c6 100644 --- a/connector/connect/common/src/main/protobuf/spark/connect/commands.proto +++ b/connector/connect/common/src/main/protobuf/spark/connect/commands.proto @@ -127,6 +127,9 @@ message WriteOperation { // (Optional) A list of configuration options. map options = 9; + // (Optional) Columns used for clustering the table. + repeated string clustering_columns = 10; + message SaveTable { // (Required) The table name. string table_name = 1; @@ -191,6 +194,9 @@ message WriteOperationV2 { // (Optional) A condition for overwrite saving mode Expression overwrite_condition = 8; + + // (Optional) Columns used for clustering the table. + repeated string clustering_columns = 9; } // Starts write stream operation as streaming query. Query ID and Run ID of the streaming diff --git a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala index d4b40624be7e7..7a7c6a2d6c925 100644 --- a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala +++ b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala @@ -16,6 +16,7 @@ */ package org.apache.spark.sql.connect.client +import java.lang.ref.Cleaner import java.util.Objects import scala.collection.mutable @@ -28,7 +29,6 @@ import org.apache.spark.connect.proto import org.apache.spark.sql.catalyst.encoders.{AgnosticEncoder, RowEncoder} import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ProductEncoder, UnboundRowEncoder} import org.apache.spark.sql.connect.client.arrow.{AbstractMessageIterator, ArrowDeserializingIterator, ConcatenatingArrowStreamReader, MessageIterator} -import org.apache.spark.sql.connect.client.util.Cleanable import org.apache.spark.sql.connect.common.DataTypeProtoConverter import org.apache.spark.sql.types.{DataType, StructType} import org.apache.spark.sql.util.ArrowUtils @@ -38,8 +38,7 @@ private[sql] class SparkResult[T]( allocator: BufferAllocator, encoder: AgnosticEncoder[T], timeZoneId: String) - extends AutoCloseable - with Cleanable { self => + extends AutoCloseable { self => private[this] var opId: String = _ private[this] var numRecords: Int = 0 @@ -47,6 +46,8 @@ private[sql] class SparkResult[T]( private[this] var arrowSchema: pojo.Schema = _ private[this] var nextResultIndex: Int = 0 private val resultMap = mutable.Map.empty[Int, (Long, Seq[ArrowMessage])] + private val cleanable = + SparkResult.cleaner.register(this, new SparkResultCloseable(resultMap, responses)) /** * Update RowEncoder and recursively update the fields of the ProductEncoder if found. @@ -257,9 +258,7 @@ private[sql] class SparkResult[T]( /** * Close this result, freeing any underlying resources. */ - override def close(): Unit = cleaner.close() - - override val cleaner: AutoCloseable = new SparkResultCloseable(resultMap, responses) + override def close(): Unit = cleanable.clean() private class ResultMessageIterator(destructive: Boolean) extends AbstractMessageIterator { private[this] var totalBytesRead = 0L @@ -309,12 +308,21 @@ private[sql] class SparkResult[T]( } } +private object SparkResult { + private val cleaner: Cleaner = Cleaner.create() +} + private[client] class SparkResultCloseable( resultMap: mutable.Map[Int, (Long, Seq[ArrowMessage])], responses: CloseableIterator[proto.ExecutePlanResponse]) - extends AutoCloseable { + extends AutoCloseable + with Runnable { override def close(): Unit = { resultMap.values.foreach(_._2.foreach(_.close())) responses.close() } + + override def run(): Unit = { + close() + } } diff --git a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/util/Cleaner.scala b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/util/Cleaner.scala deleted file mode 100644 index 4eecc88135665..0000000000000 --- a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/util/Cleaner.scala +++ /dev/null @@ -1,113 +0,0 @@ -/* - * 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.connect.client.util - -import java.lang.ref.{ReferenceQueue, WeakReference} -import java.util.Collections -import java.util.concurrent.ConcurrentHashMap - -import scala.collection.mutable -import scala.util.control.NonFatal - -/** - * Helper class for cleaning up an object's resources after the object itself has been garbage - * collected. - * - * When we move to Java 9+ we should replace this class by [[java.lang.ref.Cleaner]]. - */ -private[sql] class Cleaner { - class Ref(pin: AnyRef, val resource: AutoCloseable) - extends WeakReference[AnyRef](pin, referenceQueue) - with AutoCloseable { - override def close(): Unit = resource.close() - } - - def register(pin: Cleanable): Unit = { - register(pin, pin.cleaner) - } - - /** - * Register an objects' resources for clean-up. Note that it is absolutely pivotal that resource - * itself does not contain any reference to the object, if it does the object will never be - * garbage collected and the clean-up will never be performed. - * - * @param pin - * who's resources need to be cleaned up after GC. - * @param resource - * to clean-up. - */ - def register(pin: AnyRef, resource: AutoCloseable): Unit = { - referenceBuffer.add(new Ref(pin, resource)) - } - - @volatile private var stopped = false - private val referenceBuffer = Collections.newSetFromMap[Ref](new ConcurrentHashMap) - private val referenceQueue = new ReferenceQueue[AnyRef] - - private val cleanerThread = { - val thread = new Thread(() => cleanUp()) - thread.setName("cleaner") - thread.setDaemon(true) - thread - } - - def start(): Unit = { - require(!stopped) - cleanerThread.start() - } - - def stop(): Unit = { - stopped = true - cleanerThread.interrupt() - } - - private def cleanUp(): Unit = { - while (!stopped) { - try { - val ref = referenceQueue.remove().asInstanceOf[Ref] - referenceBuffer.remove(ref) - ref.close() - } catch { - case NonFatal(e) => - // Perhaps log this? - e.printStackTrace() - } - } - } -} - -trait Cleanable { - def cleaner: AutoCloseable -} - -object AutoCloseables { - def apply(resources: Seq[AutoCloseable]): AutoCloseable = { () => - val throwables = mutable.Buffer.empty[Throwable] - resources.foreach { resource => - try { - resource.close() - } catch { - case NonFatal(e) => throwables += e - } - } - if (throwables.nonEmpty) { - val t = throwables.head - throwables.tail.foreach(t.addSuppressed) - throw t - } - } -} diff --git a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/common/ProtoUtils.scala b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/common/ProtoUtils.scala index c7bf3f93bd0f2..4d1be169ae1ba 100644 --- a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/common/ProtoUtils.scala +++ b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/common/ProtoUtils.scala @@ -43,9 +43,12 @@ private[connect] object ProtoUtils { case (field: FieldDescriptor, byteString: ByteString) if field.getJavaType == FieldDescriptor.JavaType.BYTE_STRING && byteString != null => val size = byteString.size - if (size > maxStringSize) { - val prefix = Array.tabulate(maxStringSize)(byteString.byteAt) - builder.setField(field, createByteString(prefix, size)) + if (size > MAX_BYTES_SIZE) { + builder.setField( + field, + byteString + .substring(0, MAX_BYTES_SIZE) + .concat(createTruncatedByteString(size))) } else { builder.setField(field, byteString) } @@ -54,8 +57,11 @@ private[connect] object ProtoUtils { if field.getJavaType == FieldDescriptor.JavaType.BYTE_STRING && byteArray != null => val size = byteArray.size if (size > MAX_BYTES_SIZE) { - val prefix = byteArray.take(MAX_BYTES_SIZE) - builder.setField(field, createByteString(prefix, size)) + builder.setField( + field, + ByteString + .copyFrom(byteArray, 0, MAX_BYTES_SIZE) + .concat(createTruncatedByteString(size))) } else { builder.setField(field, byteArray) } @@ -63,7 +69,7 @@ private[connect] object ProtoUtils { // TODO(SPARK-43117): should also support 1, repeated msg; 2, map case (field: FieldDescriptor, msg: Message) if field.getJavaType == FieldDescriptor.JavaType.MESSAGE && msg != null => - builder.setField(field, abbreviate(msg)) + builder.setField(field, abbreviate(msg, maxStringSize)) case (field: FieldDescriptor, value: Any) => builder.setField(field, value) } @@ -71,11 +77,8 @@ private[connect] object ProtoUtils { builder.build() } - private def createByteString(prefix: Array[Byte], size: Int): ByteString = { - ByteString.copyFrom( - List( - ByteString.copyFrom(prefix), - ByteString.copyFromUtf8(s"[truncated(size=${format.format(size)})]")).asJava) + private def createTruncatedByteString(size: Int): ByteString = { + ByteString.copyFromUtf8(s"[truncated(size=${format.format(size)})]") } private def createString(prefix: String, size: Int): String = { diff --git a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/common/UdfUtils.scala b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/common/UdfUtils.scala index 7050e62d549c8..adcc6e3d6bf1e 100644 --- a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/common/UdfUtils.scala +++ b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/common/UdfUtils.scala @@ -132,7 +132,7 @@ private[sql] object UdfUtils extends Serializable { def noOp[V, K](): V => K = _ => null.asInstanceOf[K] def iterableOnceToSeq[A, B](f: A => IterableOnce[B]): A => Seq[B] = { value => - f(value).toSeq + f(value).iterator.to(Seq) } // (1 to 22).foreach { i => diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index f101a180ea5a3..883ca62ae22b4 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -674,7 +674,7 @@ private[spark] class SparkSubmit extends Logging { confKey = EXECUTOR_CORES.key), OptionAssigner(args.executorMemory, STANDALONE | YARN | KUBERNETES, ALL_DEPLOY_MODES, confKey = EXECUTOR_MEMORY.key), - OptionAssigner(args.totalExecutorCores, STANDALONE | KUBERNETES, ALL_DEPLOY_MODES, + OptionAssigner(args.totalExecutorCores, STANDALONE, ALL_DEPLOY_MODES, confKey = CORES_MAX.key), OptionAssigner(args.files, LOCAL | STANDALONE | KUBERNETES, ALL_DEPLOY_MODES, confKey = FILES.key), diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index bd6507c8ac49b..534c14000614d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -566,7 +566,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S | --kill SUBMISSION_ID If given, kills the driver specified. | --status SUBMISSION_ID If given, requests the status of the driver specified. | - | Spark standalone and Kubernetes only: + | Spark standalone only: | --total-executor-cores NUM Total cores for all executors. | | Spark standalone, YARN and Kubernetes only: diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 46503e017ea4a..29022c7419b4b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -1042,7 +1042,7 @@ private[deploy] class Master( completedApps.take(toRemove).foreach { a => applicationMetricsSystem.removeSource(a.appSource) } - completedApps.trimStart(toRemove) + completedApps.dropInPlace(toRemove) } completedApps += app // Remember it in our history waitingApps -= app @@ -1204,7 +1204,7 @@ private[deploy] class Master( drivers -= driver if (completedDrivers.size >= retainedDrivers) { val toRemove = math.max(retainedDrivers / 10, 1) - completedDrivers.trimStart(toRemove) + completedDrivers.dropInPlace(toRemove) } completedDrivers += driver persistenceEngine.removeDriver(driver) diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 0739367ec79d8..a40d6636ff061 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -761,7 +761,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) val cleanF = self.context.clean(f) new MapPartitionsRDD[(K, U), (K, V)](self, (context, pid, iter) => iter.flatMap { case (k, v) => - cleanF(v).map(x => (k, x)) + cleanF(v).iterator.map(x => (k, x)) }, preservesPartitioning = true) } diff --git a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala index 3c1451a01850d..7c6eaebbc6bd1 100644 --- a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala @@ -84,7 +84,7 @@ class UnionRDD[T: ClassTag]( } else { rdds } - val array = new Array[Partition](parRDDs.map(_.partitions.length).sum) + val array = new Array[Partition](parRDDs.iterator.map(_.partitions.length).sum) var pos = 0 for ((rdd, rddIndex) <- rdds.zipWithIndex; split <- rdd.partitions) { array(pos) = new UnionPartition(pos, rdd, rddIndex, split.index) diff --git a/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala b/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala index 074dab847265c..ccb4d2063ff3b 100644 --- a/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala +++ b/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala @@ -42,7 +42,7 @@ private[spark] class BoundedPriorityQueue[A](maxSize: Int)(implicit ord: Orderin override def knownSize: Int = size override def addAll(xs: IterableOnce[A]): this.type = { - xs.foreach { this += _ } + xs.iterator.foreach { this += _ } this } diff --git a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala index 704aeaefa55eb..1447a3e752de7 100644 --- a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala +++ b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala @@ -180,7 +180,7 @@ object SizeEstimator extends Logging { def dequeue(): AnyRef = { val elem = stack.last - stack.trimEnd(1) + stack.dropRightInPlace(1) elem } } diff --git a/core/src/main/scala/org/apache/spark/util/StatCounter.scala b/core/src/main/scala/org/apache/spark/util/StatCounter.scala index 0960a47415589..7df25fd78c403 100644 --- a/core/src/main/scala/org/apache/spark/util/StatCounter.scala +++ b/core/src/main/scala/org/apache/spark/util/StatCounter.scala @@ -52,7 +52,7 @@ class StatCounter(values: IterableOnce[Double]) extends Serializable { /** Add multiple values into this StatCounter, updating the internal statistics. */ def merge(values: IterableOnce[Double]): StatCounter = { - values.foreach(v => merge(v)) + values.iterator.foreach(v => merge(v)) this } diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index f22bec5c2bebb..6e3f42bd16db6 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -838,7 +838,7 @@ private[spark] object Utils * uses a local random number generator, avoiding inter-thread contention. */ def randomize[T: ClassTag](seq: IterableOnce[T]): Seq[T] = { - randomizeInPlace(seq.toArray) + randomizeInPlace(seq.iterator.toArray) } /** diff --git a/core/src/main/scala/org/apache/spark/util/collection/CompactBuffer.scala b/core/src/main/scala/org/apache/spark/util/collection/CompactBuffer.scala index 8d9fb85309b15..f667aea63c7f8 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/CompactBuffer.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/CompactBuffer.scala @@ -105,7 +105,7 @@ private[spark] class CompactBuffer[T: ClassTag] extends Seq[T] with Serializable } case _ => - values.foreach(e => this += e) + values.iterator.foreach(e => this += e) } this } diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index 3afbe322b6e14..8224472b75458 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -362,7 +362,7 @@ class ExternalAppendOnlyMap[K, V, C]( private def removeFromBuffer[T](buffer: ArrayBuffer[T], index: Int): T = { val elem = buffer(index) buffer(index) = buffer(buffer.size - 1) // This also works if index == buffer.size - 1 - buffer.trimEnd(1) + buffer.dropRightInPlace(1) elem } diff --git a/core/src/test/scala/org/apache/spark/deploy/IvyTestUtils.scala b/core/src/test/scala/org/apache/spark/deploy/IvyTestUtils.scala index 0dcdba3dfb86b..0b970a03ad876 100644 --- a/core/src/test/scala/org/apache/spark/deploy/IvyTestUtils.scala +++ b/core/src/test/scala/org/apache/spark/deploy/IvyTestUtils.scala @@ -316,7 +316,7 @@ private[deploy] object IvyTestUtils { } if (withR) { val rFiles = createRFiles(root, className, artifact.groupId) - allFiles.append(rFiles: _*) + allFiles.appendAll(rFiles) } val jarFile = packJar(jarPath, artifact, allFiles.toSeq, useIvyLayout, withR) assert(jarFile.exists(), "Problem creating Jar file") diff --git a/dev/create-release/release-build.sh b/dev/create-release/release-build.sh index f3571c4e48cf9..69494dc5107c5 100755 --- a/dev/create-release/release-build.sh +++ b/dev/create-release/release-build.sh @@ -227,7 +227,7 @@ git clean -d -f -x rm -f .gitignore cd .. -export MAVEN_OPTS="-Xss128m -Xmx12g -XX:ReservedCodeCacheSize=1g" +export MAVEN_OPTS="-Xss128m -Xmx${MAVEN_MXM_OPT:-12g} -XX:ReservedCodeCacheSize=1g" if [[ "$1" == "package" ]]; then # Source and binary tarballs diff --git a/docs/sql-data-sources-hive-tables.md b/docs/sql-data-sources-hive-tables.md index 0de573ec64b89..a2c37e69f9a42 100644 --- a/docs/sql-data-sources-hive-tables.md +++ b/docs/sql-data-sources-hive-tables.md @@ -130,7 +130,7 @@ The following options can be used to configure the version of Hive that is used 2.3.9 Version of the Hive metastore. Available - options are 0.12.0 through 2.3.9 and 3.0.0 through 3.1.3. + options are 2.0.0 through 2.3.9 and 3.0.0 through 3.1.3. 1.4.0 diff --git a/docs/sql-error-conditions-invalid-parameter-value-error-class.md b/docs/sql-error-conditions-invalid-parameter-value-error-class.md index 8186a56314dea..d58d4fc2f5992 100644 --- a/docs/sql-error-conditions-invalid-parameter-value-error-class.md +++ b/docs/sql-error-conditions-invalid-parameter-value-error-class.md @@ -49,6 +49,10 @@ expects an integer value in [0, ``), but got ``. expects one of the units without quotes YEAR, QUARTER, MONTH, WEEK, DAY, DAYOFYEAR, HOUR, MINUTE, SECOND, MILLISECOND, MICROSECOND, but got the string literal ``. +## LENGTH + +Expects `length` greater than or equal to 0, but got ``. + ## NULL expects a non-NULL value. @@ -61,6 +65,10 @@ expects a non-NULL value. Expects group index between 0 and ``, but got ``. +## START + +Expects a positive or a negative value for `start`, but got 0. + ## ZERO_INDEX expects `%1$`, `%2$` and so on, but got `%0$`. diff --git a/docs/sql-error-conditions-invalid-time-travel-timestamp-expr-error-class.md b/docs/sql-error-conditions-invalid-time-travel-timestamp-expr-error-class.md index 42513bf989bcf..80344662d0909 100644 --- a/docs/sql-error-conditions-invalid-time-travel-timestamp-expr-error-class.md +++ b/docs/sql-error-conditions-invalid-time-travel-timestamp-expr-error-class.md @@ -33,6 +33,10 @@ Cannot be casted to the "TIMESTAMP" type. Must be deterministic. +## OPTION + +Timestamp string in the options must be able to cast to TIMESTAMP type. + ## UNEVALUABLE Must be evaluable. diff --git a/docs/sql-error-conditions.md b/docs/sql-error-conditions.md index 007c95297f7ce..7c537f6fe20e5 100644 --- a/docs/sql-error-conditions.md +++ b/docs/sql-error-conditions.md @@ -1222,6 +1222,12 @@ For more details see [INVALID_SUBQUERY_EXPRESSION](sql-error-conditions-invalid- Cannot create the persistent object `` of the type `` because it references to the temporary object `` of the type ``. Please make the temporary object `` persistent, or make the persistent object `` temporary. +### INVALID_TIME_TRAVEL_SPEC + +[SQLSTATE: 42K0E](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation) + +Cannot specify both version and timestamp when time travelling the table. + ### [INVALID_TIME_TRAVEL_TIMESTAMP_EXPR](sql-error-conditions-invalid-time-travel-timestamp-expr-error-class.html) [SQLSTATE: 42K0E](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation) @@ -1360,6 +1366,12 @@ For more details see [MISSING_ATTRIBUTES](sql-error-conditions-missing-attribute The query does not include a GROUP BY clause. Add GROUP BY or turn it into the window functions using OVER clauses. +### MULTIPLE_TIME_TRAVEL_SPEC + +[SQLSTATE: 42K0E](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation) + +Cannot specify time travel in both the time travel clause and options. + ### MULTI_SOURCES_UNSUPPORTED_FOR_EXPRESSION [SQLSTATE: 42K0E](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation) diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java index bcff454b99acf..5d36ef56d2cf0 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java @@ -99,7 +99,7 @@ public class SparkLauncher extends AbstractLauncher { * @deprecated use `CHILD_CONNECTION_TIMEOUT` * @since 1.6.0 */ - @Deprecated + @Deprecated(since = "3.2.0") public static final String DEPRECATED_CHILD_CONNECTION_TIMEOUT = "spark.launcher.childConectionTimeout"; diff --git a/python/pyspark/sql/connect/proto/commands_pb2.py b/python/pyspark/sql/connect/proto/commands_pb2.py index a1cbea09b3ca0..309d34852d20e 100644 --- a/python/pyspark/sql/connect/proto/commands_pb2.py +++ b/python/pyspark/sql/connect/proto/commands_pb2.py @@ -35,7 +35,7 @@ DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x1cspark/connect/commands.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1aspark/connect/common.proto\x1a\x1fspark/connect/expressions.proto\x1a\x1dspark/connect/relations.proto"\xf5\x07\n\x07\x43ommand\x12]\n\x11register_function\x18\x01 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionH\x00R\x10registerFunction\x12H\n\x0fwrite_operation\x18\x02 \x01(\x0b\x32\x1d.spark.connect.WriteOperationH\x00R\x0ewriteOperation\x12_\n\x15\x63reate_dataframe_view\x18\x03 \x01(\x0b\x32).spark.connect.CreateDataFrameViewCommandH\x00R\x13\x63reateDataframeView\x12O\n\x12write_operation_v2\x18\x04 \x01(\x0b\x32\x1f.spark.connect.WriteOperationV2H\x00R\x10writeOperationV2\x12<\n\x0bsql_command\x18\x05 \x01(\x0b\x32\x19.spark.connect.SqlCommandH\x00R\nsqlCommand\x12k\n\x1cwrite_stream_operation_start\x18\x06 \x01(\x0b\x32(.spark.connect.WriteStreamOperationStartH\x00R\x19writeStreamOperationStart\x12^\n\x17streaming_query_command\x18\x07 \x01(\x0b\x32$.spark.connect.StreamingQueryCommandH\x00R\x15streamingQueryCommand\x12X\n\x15get_resources_command\x18\x08 \x01(\x0b\x32".spark.connect.GetResourcesCommandH\x00R\x13getResourcesCommand\x12t\n\x1fstreaming_query_manager_command\x18\t \x01(\x0b\x32+.spark.connect.StreamingQueryManagerCommandH\x00R\x1cstreamingQueryManagerCommand\x12m\n\x17register_table_function\x18\n \x01(\x0b\x32\x33.spark.connect.CommonInlineUserDefinedTableFunctionH\x00R\x15registerTableFunction\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textensionB\x0e\n\x0c\x63ommand_type"\xef\x03\n\nSqlCommand\x12\x10\n\x03sql\x18\x01 \x01(\tR\x03sql\x12;\n\x04\x61rgs\x18\x02 \x03(\x0b\x32#.spark.connect.SqlCommand.ArgsEntryB\x02\x18\x01R\x04\x61rgs\x12@\n\x08pos_args\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralB\x02\x18\x01R\x07posArgs\x12V\n\x0fnamed_arguments\x18\x04 \x03(\x0b\x32-.spark.connect.SqlCommand.NamedArgumentsEntryR\x0enamedArguments\x12>\n\rpos_arguments\x18\x05 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0cposArguments\x1aZ\n\tArgsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x37\n\x05value\x18\x02 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x05value:\x02\x38\x01\x1a\\\n\x13NamedArgumentsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value:\x02\x38\x01"\x96\x01\n\x1a\x43reateDataFrameViewCommand\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x1b\n\tis_global\x18\x03 \x01(\x08R\x08isGlobal\x12\x18\n\x07replace\x18\x04 \x01(\x08R\x07replace"\x9b\x08\n\x0eWriteOperation\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1b\n\x06source\x18\x02 \x01(\tH\x01R\x06source\x88\x01\x01\x12\x14\n\x04path\x18\x03 \x01(\tH\x00R\x04path\x12?\n\x05table\x18\x04 \x01(\x0b\x32\'.spark.connect.WriteOperation.SaveTableH\x00R\x05table\x12:\n\x04mode\x18\x05 \x01(\x0e\x32&.spark.connect.WriteOperation.SaveModeR\x04mode\x12*\n\x11sort_column_names\x18\x06 \x03(\tR\x0fsortColumnNames\x12\x31\n\x14partitioning_columns\x18\x07 \x03(\tR\x13partitioningColumns\x12\x43\n\tbucket_by\x18\x08 \x01(\x0b\x32&.spark.connect.WriteOperation.BucketByR\x08\x62ucketBy\x12\x44\n\x07options\x18\t \x03(\x0b\x32*.spark.connect.WriteOperation.OptionsEntryR\x07options\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x82\x02\n\tSaveTable\x12\x1d\n\ntable_name\x18\x01 \x01(\tR\ttableName\x12X\n\x0bsave_method\x18\x02 \x01(\x0e\x32\x37.spark.connect.WriteOperation.SaveTable.TableSaveMethodR\nsaveMethod"|\n\x0fTableSaveMethod\x12!\n\x1dTABLE_SAVE_METHOD_UNSPECIFIED\x10\x00\x12#\n\x1fTABLE_SAVE_METHOD_SAVE_AS_TABLE\x10\x01\x12!\n\x1dTABLE_SAVE_METHOD_INSERT_INTO\x10\x02\x1a[\n\x08\x42ucketBy\x12.\n\x13\x62ucket_column_names\x18\x01 \x03(\tR\x11\x62ucketColumnNames\x12\x1f\n\x0bnum_buckets\x18\x02 \x01(\x05R\nnumBuckets"\x89\x01\n\x08SaveMode\x12\x19\n\x15SAVE_MODE_UNSPECIFIED\x10\x00\x12\x14\n\x10SAVE_MODE_APPEND\x10\x01\x12\x17\n\x13SAVE_MODE_OVERWRITE\x10\x02\x12\x1d\n\x19SAVE_MODE_ERROR_IF_EXISTS\x10\x03\x12\x14\n\x10SAVE_MODE_IGNORE\x10\x04\x42\x0b\n\tsave_typeB\t\n\x07_source"\xad\x06\n\x10WriteOperationV2\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1d\n\ntable_name\x18\x02 \x01(\tR\ttableName\x12\x1f\n\x08provider\x18\x03 \x01(\tH\x00R\x08provider\x88\x01\x01\x12L\n\x14partitioning_columns\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13partitioningColumns\x12\x46\n\x07options\x18\x05 \x03(\x0b\x32,.spark.connect.WriteOperationV2.OptionsEntryR\x07options\x12_\n\x10table_properties\x18\x06 \x03(\x0b\x32\x34.spark.connect.WriteOperationV2.TablePropertiesEntryR\x0ftableProperties\x12\x38\n\x04mode\x18\x07 \x01(\x0e\x32$.spark.connect.WriteOperationV2.ModeR\x04mode\x12J\n\x13overwrite_condition\x18\x08 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x12overwriteCondition\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x42\n\x14TablePropertiesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01"\x9f\x01\n\x04Mode\x12\x14\n\x10MODE_UNSPECIFIED\x10\x00\x12\x0f\n\x0bMODE_CREATE\x10\x01\x12\x12\n\x0eMODE_OVERWRITE\x10\x02\x12\x1d\n\x19MODE_OVERWRITE_PARTITIONS\x10\x03\x12\x0f\n\x0bMODE_APPEND\x10\x04\x12\x10\n\x0cMODE_REPLACE\x10\x05\x12\x1a\n\x16MODE_CREATE_OR_REPLACE\x10\x06\x42\x0b\n\t_provider"\xa0\x06\n\x19WriteStreamOperationStart\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x16\n\x06\x66ormat\x18\x02 \x01(\tR\x06\x66ormat\x12O\n\x07options\x18\x03 \x03(\x0b\x32\x35.spark.connect.WriteStreamOperationStart.OptionsEntryR\x07options\x12:\n\x19partitioning_column_names\x18\x04 \x03(\tR\x17partitioningColumnNames\x12:\n\x18processing_time_interval\x18\x05 \x01(\tH\x00R\x16processingTimeInterval\x12%\n\ravailable_now\x18\x06 \x01(\x08H\x00R\x0c\x61vailableNow\x12\x14\n\x04once\x18\x07 \x01(\x08H\x00R\x04once\x12\x46\n\x1e\x63ontinuous_checkpoint_interval\x18\x08 \x01(\tH\x00R\x1c\x63ontinuousCheckpointInterval\x12\x1f\n\x0boutput_mode\x18\t \x01(\tR\noutputMode\x12\x1d\n\nquery_name\x18\n \x01(\tR\tqueryName\x12\x14\n\x04path\x18\x0b \x01(\tH\x01R\x04path\x12\x1f\n\ntable_name\x18\x0c \x01(\tH\x01R\ttableName\x12N\n\x0e\x66oreach_writer\x18\r \x01(\x0b\x32\'.spark.connect.StreamingForeachFunctionR\rforeachWriter\x12L\n\rforeach_batch\x18\x0e \x01(\x0b\x32\'.spark.connect.StreamingForeachFunctionR\x0c\x66oreachBatch\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\t\n\x07triggerB\x12\n\x10sink_destination"\xb3\x01\n\x18StreamingForeachFunction\x12\x43\n\x0fpython_function\x18\x01 \x01(\x0b\x32\x18.spark.connect.PythonUDFH\x00R\x0epythonFunction\x12\x46\n\x0escala_function\x18\x02 \x01(\x0b\x32\x1d.spark.connect.ScalarScalaUDFH\x00R\rscalaFunctionB\n\n\x08\x66unction"y\n\x1fWriteStreamOperationStartResult\x12\x42\n\x08query_id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x07queryId\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name"A\n\x18StreamingQueryInstanceId\x12\x0e\n\x02id\x18\x01 \x01(\tR\x02id\x12\x15\n\x06run_id\x18\x02 \x01(\tR\x05runId"\xf8\x04\n\x15StreamingQueryCommand\x12\x42\n\x08query_id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x07queryId\x12\x18\n\x06status\x18\x02 \x01(\x08H\x00R\x06status\x12%\n\rlast_progress\x18\x03 \x01(\x08H\x00R\x0clastProgress\x12)\n\x0frecent_progress\x18\x04 \x01(\x08H\x00R\x0erecentProgress\x12\x14\n\x04stop\x18\x05 \x01(\x08H\x00R\x04stop\x12\x34\n\x15process_all_available\x18\x06 \x01(\x08H\x00R\x13processAllAvailable\x12O\n\x07\x65xplain\x18\x07 \x01(\x0b\x32\x33.spark.connect.StreamingQueryCommand.ExplainCommandH\x00R\x07\x65xplain\x12\x1e\n\texception\x18\x08 \x01(\x08H\x00R\texception\x12k\n\x11\x61wait_termination\x18\t \x01(\x0b\x32<.spark.connect.StreamingQueryCommand.AwaitTerminationCommandH\x00R\x10\x61waitTermination\x1a,\n\x0e\x45xplainCommand\x12\x1a\n\x08\x65xtended\x18\x01 \x01(\x08R\x08\x65xtended\x1aL\n\x17\x41waitTerminationCommand\x12"\n\ntimeout_ms\x18\x02 \x01(\x03H\x00R\ttimeoutMs\x88\x01\x01\x42\r\n\x0b_timeout_msB\t\n\x07\x63ommand"\xf5\x08\n\x1bStreamingQueryCommandResult\x12\x42\n\x08query_id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x07queryId\x12Q\n\x06status\x18\x02 \x01(\x0b\x32\x37.spark.connect.StreamingQueryCommandResult.StatusResultH\x00R\x06status\x12j\n\x0frecent_progress\x18\x03 \x01(\x0b\x32?.spark.connect.StreamingQueryCommandResult.RecentProgressResultH\x00R\x0erecentProgress\x12T\n\x07\x65xplain\x18\x04 \x01(\x0b\x32\x38.spark.connect.StreamingQueryCommandResult.ExplainResultH\x00R\x07\x65xplain\x12Z\n\texception\x18\x05 \x01(\x0b\x32:.spark.connect.StreamingQueryCommandResult.ExceptionResultH\x00R\texception\x12p\n\x11\x61wait_termination\x18\x06 \x01(\x0b\x32\x41.spark.connect.StreamingQueryCommandResult.AwaitTerminationResultH\x00R\x10\x61waitTermination\x1a\xaa\x01\n\x0cStatusResult\x12%\n\x0estatus_message\x18\x01 \x01(\tR\rstatusMessage\x12*\n\x11is_data_available\x18\x02 \x01(\x08R\x0fisDataAvailable\x12*\n\x11is_trigger_active\x18\x03 \x01(\x08R\x0fisTriggerActive\x12\x1b\n\tis_active\x18\x04 \x01(\x08R\x08isActive\x1aH\n\x14RecentProgressResult\x12\x30\n\x14recent_progress_json\x18\x05 \x03(\tR\x12recentProgressJson\x1a\'\n\rExplainResult\x12\x16\n\x06result\x18\x01 \x01(\tR\x06result\x1a\xc5\x01\n\x0f\x45xceptionResult\x12\x30\n\x11\x65xception_message\x18\x01 \x01(\tH\x00R\x10\x65xceptionMessage\x88\x01\x01\x12$\n\x0b\x65rror_class\x18\x02 \x01(\tH\x01R\nerrorClass\x88\x01\x01\x12$\n\x0bstack_trace\x18\x03 \x01(\tH\x02R\nstackTrace\x88\x01\x01\x42\x14\n\x12_exception_messageB\x0e\n\x0c_error_classB\x0e\n\x0c_stack_trace\x1a\x38\n\x16\x41waitTerminationResult\x12\x1e\n\nterminated\x18\x01 \x01(\x08R\nterminatedB\r\n\x0bresult_type"\xbd\x06\n\x1cStreamingQueryManagerCommand\x12\x18\n\x06\x61\x63tive\x18\x01 \x01(\x08H\x00R\x06\x61\x63tive\x12\x1d\n\tget_query\x18\x02 \x01(\tH\x00R\x08getQuery\x12|\n\x15\x61wait_any_termination\x18\x03 \x01(\x0b\x32\x46.spark.connect.StreamingQueryManagerCommand.AwaitAnyTerminationCommandH\x00R\x13\x61waitAnyTermination\x12+\n\x10reset_terminated\x18\x04 \x01(\x08H\x00R\x0fresetTerminated\x12n\n\x0c\x61\x64\x64_listener\x18\x05 \x01(\x0b\x32I.spark.connect.StreamingQueryManagerCommand.StreamingQueryListenerCommandH\x00R\x0b\x61\x64\x64Listener\x12t\n\x0fremove_listener\x18\x06 \x01(\x0b\x32I.spark.connect.StreamingQueryManagerCommand.StreamingQueryListenerCommandH\x00R\x0eremoveListener\x12\'\n\x0elist_listeners\x18\x07 \x01(\x08H\x00R\rlistListeners\x1aO\n\x1a\x41waitAnyTerminationCommand\x12"\n\ntimeout_ms\x18\x01 \x01(\x03H\x00R\ttimeoutMs\x88\x01\x01\x42\r\n\x0b_timeout_ms\x1a\xcd\x01\n\x1dStreamingQueryListenerCommand\x12)\n\x10listener_payload\x18\x01 \x01(\x0cR\x0flistenerPayload\x12U\n\x17python_listener_payload\x18\x02 \x01(\x0b\x32\x18.spark.connect.PythonUDFH\x00R\x15pythonListenerPayload\x88\x01\x01\x12\x0e\n\x02id\x18\x03 \x01(\tR\x02idB\x1a\n\x18_python_listener_payloadB\t\n\x07\x63ommand"\xb4\x08\n"StreamingQueryManagerCommandResult\x12X\n\x06\x61\x63tive\x18\x01 \x01(\x0b\x32>.spark.connect.StreamingQueryManagerCommandResult.ActiveResultH\x00R\x06\x61\x63tive\x12`\n\x05query\x18\x02 \x01(\x0b\x32H.spark.connect.StreamingQueryManagerCommandResult.StreamingQueryInstanceH\x00R\x05query\x12\x81\x01\n\x15\x61wait_any_termination\x18\x03 \x01(\x0b\x32K.spark.connect.StreamingQueryManagerCommandResult.AwaitAnyTerminationResultH\x00R\x13\x61waitAnyTermination\x12+\n\x10reset_terminated\x18\x04 \x01(\x08H\x00R\x0fresetTerminated\x12#\n\x0c\x61\x64\x64_listener\x18\x05 \x01(\x08H\x00R\x0b\x61\x64\x64Listener\x12)\n\x0fremove_listener\x18\x06 \x01(\x08H\x00R\x0eremoveListener\x12{\n\x0elist_listeners\x18\x07 \x01(\x0b\x32R.spark.connect.StreamingQueryManagerCommandResult.ListStreamingQueryListenerResultH\x00R\rlistListeners\x1a\x7f\n\x0c\x41\x63tiveResult\x12o\n\x0e\x61\x63tive_queries\x18\x01 \x03(\x0b\x32H.spark.connect.StreamingQueryManagerCommandResult.StreamingQueryInstanceR\ractiveQueries\x1as\n\x16StreamingQueryInstance\x12\x37\n\x02id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x02id\x12\x17\n\x04name\x18\x02 \x01(\tH\x00R\x04name\x88\x01\x01\x42\x07\n\x05_name\x1a;\n\x19\x41waitAnyTerminationResult\x12\x1e\n\nterminated\x18\x01 \x01(\x08R\nterminated\x1aK\n\x1eStreamingQueryListenerInstance\x12)\n\x10listener_payload\x18\x01 \x01(\x0cR\x0flistenerPayload\x1a\x45\n ListStreamingQueryListenerResult\x12!\n\x0clistener_ids\x18\x01 \x03(\tR\x0blistenerIdsB\r\n\x0bresult_type"\x15\n\x13GetResourcesCommand"\xd4\x01\n\x19GetResourcesCommandResult\x12U\n\tresources\x18\x01 \x03(\x0b\x32\x37.spark.connect.GetResourcesCommandResult.ResourcesEntryR\tresources\x1a`\n\x0eResourcesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x38\n\x05value\x18\x02 \x01(\x0b\x32".spark.connect.ResourceInformationR\x05value:\x02\x38\x01\x42\x36\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' + b'\n\x1cspark/connect/commands.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1aspark/connect/common.proto\x1a\x1fspark/connect/expressions.proto\x1a\x1dspark/connect/relations.proto"\xf5\x07\n\x07\x43ommand\x12]\n\x11register_function\x18\x01 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionH\x00R\x10registerFunction\x12H\n\x0fwrite_operation\x18\x02 \x01(\x0b\x32\x1d.spark.connect.WriteOperationH\x00R\x0ewriteOperation\x12_\n\x15\x63reate_dataframe_view\x18\x03 \x01(\x0b\x32).spark.connect.CreateDataFrameViewCommandH\x00R\x13\x63reateDataframeView\x12O\n\x12write_operation_v2\x18\x04 \x01(\x0b\x32\x1f.spark.connect.WriteOperationV2H\x00R\x10writeOperationV2\x12<\n\x0bsql_command\x18\x05 \x01(\x0b\x32\x19.spark.connect.SqlCommandH\x00R\nsqlCommand\x12k\n\x1cwrite_stream_operation_start\x18\x06 \x01(\x0b\x32(.spark.connect.WriteStreamOperationStartH\x00R\x19writeStreamOperationStart\x12^\n\x17streaming_query_command\x18\x07 \x01(\x0b\x32$.spark.connect.StreamingQueryCommandH\x00R\x15streamingQueryCommand\x12X\n\x15get_resources_command\x18\x08 \x01(\x0b\x32".spark.connect.GetResourcesCommandH\x00R\x13getResourcesCommand\x12t\n\x1fstreaming_query_manager_command\x18\t \x01(\x0b\x32+.spark.connect.StreamingQueryManagerCommandH\x00R\x1cstreamingQueryManagerCommand\x12m\n\x17register_table_function\x18\n \x01(\x0b\x32\x33.spark.connect.CommonInlineUserDefinedTableFunctionH\x00R\x15registerTableFunction\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textensionB\x0e\n\x0c\x63ommand_type"\xef\x03\n\nSqlCommand\x12\x10\n\x03sql\x18\x01 \x01(\tR\x03sql\x12;\n\x04\x61rgs\x18\x02 \x03(\x0b\x32#.spark.connect.SqlCommand.ArgsEntryB\x02\x18\x01R\x04\x61rgs\x12@\n\x08pos_args\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralB\x02\x18\x01R\x07posArgs\x12V\n\x0fnamed_arguments\x18\x04 \x03(\x0b\x32-.spark.connect.SqlCommand.NamedArgumentsEntryR\x0enamedArguments\x12>\n\rpos_arguments\x18\x05 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0cposArguments\x1aZ\n\tArgsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x37\n\x05value\x18\x02 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x05value:\x02\x38\x01\x1a\\\n\x13NamedArgumentsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value:\x02\x38\x01"\x96\x01\n\x1a\x43reateDataFrameViewCommand\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x1b\n\tis_global\x18\x03 \x01(\x08R\x08isGlobal\x12\x18\n\x07replace\x18\x04 \x01(\x08R\x07replace"\xca\x08\n\x0eWriteOperation\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1b\n\x06source\x18\x02 \x01(\tH\x01R\x06source\x88\x01\x01\x12\x14\n\x04path\x18\x03 \x01(\tH\x00R\x04path\x12?\n\x05table\x18\x04 \x01(\x0b\x32\'.spark.connect.WriteOperation.SaveTableH\x00R\x05table\x12:\n\x04mode\x18\x05 \x01(\x0e\x32&.spark.connect.WriteOperation.SaveModeR\x04mode\x12*\n\x11sort_column_names\x18\x06 \x03(\tR\x0fsortColumnNames\x12\x31\n\x14partitioning_columns\x18\x07 \x03(\tR\x13partitioningColumns\x12\x43\n\tbucket_by\x18\x08 \x01(\x0b\x32&.spark.connect.WriteOperation.BucketByR\x08\x62ucketBy\x12\x44\n\x07options\x18\t \x03(\x0b\x32*.spark.connect.WriteOperation.OptionsEntryR\x07options\x12-\n\x12\x63lustering_columns\x18\n \x03(\tR\x11\x63lusteringColumns\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x82\x02\n\tSaveTable\x12\x1d\n\ntable_name\x18\x01 \x01(\tR\ttableName\x12X\n\x0bsave_method\x18\x02 \x01(\x0e\x32\x37.spark.connect.WriteOperation.SaveTable.TableSaveMethodR\nsaveMethod"|\n\x0fTableSaveMethod\x12!\n\x1dTABLE_SAVE_METHOD_UNSPECIFIED\x10\x00\x12#\n\x1fTABLE_SAVE_METHOD_SAVE_AS_TABLE\x10\x01\x12!\n\x1dTABLE_SAVE_METHOD_INSERT_INTO\x10\x02\x1a[\n\x08\x42ucketBy\x12.\n\x13\x62ucket_column_names\x18\x01 \x03(\tR\x11\x62ucketColumnNames\x12\x1f\n\x0bnum_buckets\x18\x02 \x01(\x05R\nnumBuckets"\x89\x01\n\x08SaveMode\x12\x19\n\x15SAVE_MODE_UNSPECIFIED\x10\x00\x12\x14\n\x10SAVE_MODE_APPEND\x10\x01\x12\x17\n\x13SAVE_MODE_OVERWRITE\x10\x02\x12\x1d\n\x19SAVE_MODE_ERROR_IF_EXISTS\x10\x03\x12\x14\n\x10SAVE_MODE_IGNORE\x10\x04\x42\x0b\n\tsave_typeB\t\n\x07_source"\xdc\x06\n\x10WriteOperationV2\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1d\n\ntable_name\x18\x02 \x01(\tR\ttableName\x12\x1f\n\x08provider\x18\x03 \x01(\tH\x00R\x08provider\x88\x01\x01\x12L\n\x14partitioning_columns\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13partitioningColumns\x12\x46\n\x07options\x18\x05 \x03(\x0b\x32,.spark.connect.WriteOperationV2.OptionsEntryR\x07options\x12_\n\x10table_properties\x18\x06 \x03(\x0b\x32\x34.spark.connect.WriteOperationV2.TablePropertiesEntryR\x0ftableProperties\x12\x38\n\x04mode\x18\x07 \x01(\x0e\x32$.spark.connect.WriteOperationV2.ModeR\x04mode\x12J\n\x13overwrite_condition\x18\x08 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x12overwriteCondition\x12-\n\x12\x63lustering_columns\x18\t \x03(\tR\x11\x63lusteringColumns\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x42\n\x14TablePropertiesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01"\x9f\x01\n\x04Mode\x12\x14\n\x10MODE_UNSPECIFIED\x10\x00\x12\x0f\n\x0bMODE_CREATE\x10\x01\x12\x12\n\x0eMODE_OVERWRITE\x10\x02\x12\x1d\n\x19MODE_OVERWRITE_PARTITIONS\x10\x03\x12\x0f\n\x0bMODE_APPEND\x10\x04\x12\x10\n\x0cMODE_REPLACE\x10\x05\x12\x1a\n\x16MODE_CREATE_OR_REPLACE\x10\x06\x42\x0b\n\t_provider"\xa0\x06\n\x19WriteStreamOperationStart\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x16\n\x06\x66ormat\x18\x02 \x01(\tR\x06\x66ormat\x12O\n\x07options\x18\x03 \x03(\x0b\x32\x35.spark.connect.WriteStreamOperationStart.OptionsEntryR\x07options\x12:\n\x19partitioning_column_names\x18\x04 \x03(\tR\x17partitioningColumnNames\x12:\n\x18processing_time_interval\x18\x05 \x01(\tH\x00R\x16processingTimeInterval\x12%\n\ravailable_now\x18\x06 \x01(\x08H\x00R\x0c\x61vailableNow\x12\x14\n\x04once\x18\x07 \x01(\x08H\x00R\x04once\x12\x46\n\x1e\x63ontinuous_checkpoint_interval\x18\x08 \x01(\tH\x00R\x1c\x63ontinuousCheckpointInterval\x12\x1f\n\x0boutput_mode\x18\t \x01(\tR\noutputMode\x12\x1d\n\nquery_name\x18\n \x01(\tR\tqueryName\x12\x14\n\x04path\x18\x0b \x01(\tH\x01R\x04path\x12\x1f\n\ntable_name\x18\x0c \x01(\tH\x01R\ttableName\x12N\n\x0e\x66oreach_writer\x18\r \x01(\x0b\x32\'.spark.connect.StreamingForeachFunctionR\rforeachWriter\x12L\n\rforeach_batch\x18\x0e \x01(\x0b\x32\'.spark.connect.StreamingForeachFunctionR\x0c\x66oreachBatch\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\t\n\x07triggerB\x12\n\x10sink_destination"\xb3\x01\n\x18StreamingForeachFunction\x12\x43\n\x0fpython_function\x18\x01 \x01(\x0b\x32\x18.spark.connect.PythonUDFH\x00R\x0epythonFunction\x12\x46\n\x0escala_function\x18\x02 \x01(\x0b\x32\x1d.spark.connect.ScalarScalaUDFH\x00R\rscalaFunctionB\n\n\x08\x66unction"y\n\x1fWriteStreamOperationStartResult\x12\x42\n\x08query_id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x07queryId\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name"A\n\x18StreamingQueryInstanceId\x12\x0e\n\x02id\x18\x01 \x01(\tR\x02id\x12\x15\n\x06run_id\x18\x02 \x01(\tR\x05runId"\xf8\x04\n\x15StreamingQueryCommand\x12\x42\n\x08query_id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x07queryId\x12\x18\n\x06status\x18\x02 \x01(\x08H\x00R\x06status\x12%\n\rlast_progress\x18\x03 \x01(\x08H\x00R\x0clastProgress\x12)\n\x0frecent_progress\x18\x04 \x01(\x08H\x00R\x0erecentProgress\x12\x14\n\x04stop\x18\x05 \x01(\x08H\x00R\x04stop\x12\x34\n\x15process_all_available\x18\x06 \x01(\x08H\x00R\x13processAllAvailable\x12O\n\x07\x65xplain\x18\x07 \x01(\x0b\x32\x33.spark.connect.StreamingQueryCommand.ExplainCommandH\x00R\x07\x65xplain\x12\x1e\n\texception\x18\x08 \x01(\x08H\x00R\texception\x12k\n\x11\x61wait_termination\x18\t \x01(\x0b\x32<.spark.connect.StreamingQueryCommand.AwaitTerminationCommandH\x00R\x10\x61waitTermination\x1a,\n\x0e\x45xplainCommand\x12\x1a\n\x08\x65xtended\x18\x01 \x01(\x08R\x08\x65xtended\x1aL\n\x17\x41waitTerminationCommand\x12"\n\ntimeout_ms\x18\x02 \x01(\x03H\x00R\ttimeoutMs\x88\x01\x01\x42\r\n\x0b_timeout_msB\t\n\x07\x63ommand"\xf5\x08\n\x1bStreamingQueryCommandResult\x12\x42\n\x08query_id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x07queryId\x12Q\n\x06status\x18\x02 \x01(\x0b\x32\x37.spark.connect.StreamingQueryCommandResult.StatusResultH\x00R\x06status\x12j\n\x0frecent_progress\x18\x03 \x01(\x0b\x32?.spark.connect.StreamingQueryCommandResult.RecentProgressResultH\x00R\x0erecentProgress\x12T\n\x07\x65xplain\x18\x04 \x01(\x0b\x32\x38.spark.connect.StreamingQueryCommandResult.ExplainResultH\x00R\x07\x65xplain\x12Z\n\texception\x18\x05 \x01(\x0b\x32:.spark.connect.StreamingQueryCommandResult.ExceptionResultH\x00R\texception\x12p\n\x11\x61wait_termination\x18\x06 \x01(\x0b\x32\x41.spark.connect.StreamingQueryCommandResult.AwaitTerminationResultH\x00R\x10\x61waitTermination\x1a\xaa\x01\n\x0cStatusResult\x12%\n\x0estatus_message\x18\x01 \x01(\tR\rstatusMessage\x12*\n\x11is_data_available\x18\x02 \x01(\x08R\x0fisDataAvailable\x12*\n\x11is_trigger_active\x18\x03 \x01(\x08R\x0fisTriggerActive\x12\x1b\n\tis_active\x18\x04 \x01(\x08R\x08isActive\x1aH\n\x14RecentProgressResult\x12\x30\n\x14recent_progress_json\x18\x05 \x03(\tR\x12recentProgressJson\x1a\'\n\rExplainResult\x12\x16\n\x06result\x18\x01 \x01(\tR\x06result\x1a\xc5\x01\n\x0f\x45xceptionResult\x12\x30\n\x11\x65xception_message\x18\x01 \x01(\tH\x00R\x10\x65xceptionMessage\x88\x01\x01\x12$\n\x0b\x65rror_class\x18\x02 \x01(\tH\x01R\nerrorClass\x88\x01\x01\x12$\n\x0bstack_trace\x18\x03 \x01(\tH\x02R\nstackTrace\x88\x01\x01\x42\x14\n\x12_exception_messageB\x0e\n\x0c_error_classB\x0e\n\x0c_stack_trace\x1a\x38\n\x16\x41waitTerminationResult\x12\x1e\n\nterminated\x18\x01 \x01(\x08R\nterminatedB\r\n\x0bresult_type"\xbd\x06\n\x1cStreamingQueryManagerCommand\x12\x18\n\x06\x61\x63tive\x18\x01 \x01(\x08H\x00R\x06\x61\x63tive\x12\x1d\n\tget_query\x18\x02 \x01(\tH\x00R\x08getQuery\x12|\n\x15\x61wait_any_termination\x18\x03 \x01(\x0b\x32\x46.spark.connect.StreamingQueryManagerCommand.AwaitAnyTerminationCommandH\x00R\x13\x61waitAnyTermination\x12+\n\x10reset_terminated\x18\x04 \x01(\x08H\x00R\x0fresetTerminated\x12n\n\x0c\x61\x64\x64_listener\x18\x05 \x01(\x0b\x32I.spark.connect.StreamingQueryManagerCommand.StreamingQueryListenerCommandH\x00R\x0b\x61\x64\x64Listener\x12t\n\x0fremove_listener\x18\x06 \x01(\x0b\x32I.spark.connect.StreamingQueryManagerCommand.StreamingQueryListenerCommandH\x00R\x0eremoveListener\x12\'\n\x0elist_listeners\x18\x07 \x01(\x08H\x00R\rlistListeners\x1aO\n\x1a\x41waitAnyTerminationCommand\x12"\n\ntimeout_ms\x18\x01 \x01(\x03H\x00R\ttimeoutMs\x88\x01\x01\x42\r\n\x0b_timeout_ms\x1a\xcd\x01\n\x1dStreamingQueryListenerCommand\x12)\n\x10listener_payload\x18\x01 \x01(\x0cR\x0flistenerPayload\x12U\n\x17python_listener_payload\x18\x02 \x01(\x0b\x32\x18.spark.connect.PythonUDFH\x00R\x15pythonListenerPayload\x88\x01\x01\x12\x0e\n\x02id\x18\x03 \x01(\tR\x02idB\x1a\n\x18_python_listener_payloadB\t\n\x07\x63ommand"\xb4\x08\n"StreamingQueryManagerCommandResult\x12X\n\x06\x61\x63tive\x18\x01 \x01(\x0b\x32>.spark.connect.StreamingQueryManagerCommandResult.ActiveResultH\x00R\x06\x61\x63tive\x12`\n\x05query\x18\x02 \x01(\x0b\x32H.spark.connect.StreamingQueryManagerCommandResult.StreamingQueryInstanceH\x00R\x05query\x12\x81\x01\n\x15\x61wait_any_termination\x18\x03 \x01(\x0b\x32K.spark.connect.StreamingQueryManagerCommandResult.AwaitAnyTerminationResultH\x00R\x13\x61waitAnyTermination\x12+\n\x10reset_terminated\x18\x04 \x01(\x08H\x00R\x0fresetTerminated\x12#\n\x0c\x61\x64\x64_listener\x18\x05 \x01(\x08H\x00R\x0b\x61\x64\x64Listener\x12)\n\x0fremove_listener\x18\x06 \x01(\x08H\x00R\x0eremoveListener\x12{\n\x0elist_listeners\x18\x07 \x01(\x0b\x32R.spark.connect.StreamingQueryManagerCommandResult.ListStreamingQueryListenerResultH\x00R\rlistListeners\x1a\x7f\n\x0c\x41\x63tiveResult\x12o\n\x0e\x61\x63tive_queries\x18\x01 \x03(\x0b\x32H.spark.connect.StreamingQueryManagerCommandResult.StreamingQueryInstanceR\ractiveQueries\x1as\n\x16StreamingQueryInstance\x12\x37\n\x02id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x02id\x12\x17\n\x04name\x18\x02 \x01(\tH\x00R\x04name\x88\x01\x01\x42\x07\n\x05_name\x1a;\n\x19\x41waitAnyTerminationResult\x12\x1e\n\nterminated\x18\x01 \x01(\x08R\nterminated\x1aK\n\x1eStreamingQueryListenerInstance\x12)\n\x10listener_payload\x18\x01 \x01(\x0cR\x0flistenerPayload\x1a\x45\n ListStreamingQueryListenerResult\x12!\n\x0clistener_ids\x18\x01 \x03(\tR\x0blistenerIdsB\r\n\x0bresult_type"\x15\n\x13GetResourcesCommand"\xd4\x01\n\x19GetResourcesCommandResult\x12U\n\tresources\x18\x01 \x03(\x0b\x32\x37.spark.connect.GetResourcesCommandResult.ResourcesEntryR\tresources\x1a`\n\x0eResourcesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x38\n\x05value\x18\x02 \x01(\x0b\x32".spark.connect.ResourceInformationR\x05value:\x02\x38\x01\x42\x36\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' ) _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, globals()) @@ -74,75 +74,75 @@ _CREATEDATAFRAMEVIEWCOMMAND._serialized_start = 1681 _CREATEDATAFRAMEVIEWCOMMAND._serialized_end = 1831 _WRITEOPERATION._serialized_start = 1834 - _WRITEOPERATION._serialized_end = 2885 - _WRITEOPERATION_OPTIONSENTRY._serialized_start = 2309 - _WRITEOPERATION_OPTIONSENTRY._serialized_end = 2367 - _WRITEOPERATION_SAVETABLE._serialized_start = 2370 - _WRITEOPERATION_SAVETABLE._serialized_end = 2628 - _WRITEOPERATION_SAVETABLE_TABLESAVEMETHOD._serialized_start = 2504 - _WRITEOPERATION_SAVETABLE_TABLESAVEMETHOD._serialized_end = 2628 - _WRITEOPERATION_BUCKETBY._serialized_start = 2630 - _WRITEOPERATION_BUCKETBY._serialized_end = 2721 - _WRITEOPERATION_SAVEMODE._serialized_start = 2724 - _WRITEOPERATION_SAVEMODE._serialized_end = 2861 - _WRITEOPERATIONV2._serialized_start = 2888 - _WRITEOPERATIONV2._serialized_end = 3701 - _WRITEOPERATIONV2_OPTIONSENTRY._serialized_start = 2309 - _WRITEOPERATIONV2_OPTIONSENTRY._serialized_end = 2367 - _WRITEOPERATIONV2_TABLEPROPERTIESENTRY._serialized_start = 3460 - _WRITEOPERATIONV2_TABLEPROPERTIESENTRY._serialized_end = 3526 - _WRITEOPERATIONV2_MODE._serialized_start = 3529 - _WRITEOPERATIONV2_MODE._serialized_end = 3688 - _WRITESTREAMOPERATIONSTART._serialized_start = 3704 - _WRITESTREAMOPERATIONSTART._serialized_end = 4504 - _WRITESTREAMOPERATIONSTART_OPTIONSENTRY._serialized_start = 2309 - _WRITESTREAMOPERATIONSTART_OPTIONSENTRY._serialized_end = 2367 - _STREAMINGFOREACHFUNCTION._serialized_start = 4507 - _STREAMINGFOREACHFUNCTION._serialized_end = 4686 - _WRITESTREAMOPERATIONSTARTRESULT._serialized_start = 4688 - _WRITESTREAMOPERATIONSTARTRESULT._serialized_end = 4809 - _STREAMINGQUERYINSTANCEID._serialized_start = 4811 - _STREAMINGQUERYINSTANCEID._serialized_end = 4876 - _STREAMINGQUERYCOMMAND._serialized_start = 4879 - _STREAMINGQUERYCOMMAND._serialized_end = 5511 - _STREAMINGQUERYCOMMAND_EXPLAINCOMMAND._serialized_start = 5378 - _STREAMINGQUERYCOMMAND_EXPLAINCOMMAND._serialized_end = 5422 - _STREAMINGQUERYCOMMAND_AWAITTERMINATIONCOMMAND._serialized_start = 5424 - _STREAMINGQUERYCOMMAND_AWAITTERMINATIONCOMMAND._serialized_end = 5500 - _STREAMINGQUERYCOMMANDRESULT._serialized_start = 5514 - _STREAMINGQUERYCOMMANDRESULT._serialized_end = 6655 - _STREAMINGQUERYCOMMANDRESULT_STATUSRESULT._serialized_start = 6097 - _STREAMINGQUERYCOMMANDRESULT_STATUSRESULT._serialized_end = 6267 - _STREAMINGQUERYCOMMANDRESULT_RECENTPROGRESSRESULT._serialized_start = 6269 - _STREAMINGQUERYCOMMANDRESULT_RECENTPROGRESSRESULT._serialized_end = 6341 - _STREAMINGQUERYCOMMANDRESULT_EXPLAINRESULT._serialized_start = 6343 - _STREAMINGQUERYCOMMANDRESULT_EXPLAINRESULT._serialized_end = 6382 - _STREAMINGQUERYCOMMANDRESULT_EXCEPTIONRESULT._serialized_start = 6385 - _STREAMINGQUERYCOMMANDRESULT_EXCEPTIONRESULT._serialized_end = 6582 - _STREAMINGQUERYCOMMANDRESULT_AWAITTERMINATIONRESULT._serialized_start = 6584 - _STREAMINGQUERYCOMMANDRESULT_AWAITTERMINATIONRESULT._serialized_end = 6640 - _STREAMINGQUERYMANAGERCOMMAND._serialized_start = 6658 - _STREAMINGQUERYMANAGERCOMMAND._serialized_end = 7487 - _STREAMINGQUERYMANAGERCOMMAND_AWAITANYTERMINATIONCOMMAND._serialized_start = 7189 - _STREAMINGQUERYMANAGERCOMMAND_AWAITANYTERMINATIONCOMMAND._serialized_end = 7268 - _STREAMINGQUERYMANAGERCOMMAND_STREAMINGQUERYLISTENERCOMMAND._serialized_start = 7271 - _STREAMINGQUERYMANAGERCOMMAND_STREAMINGQUERYLISTENERCOMMAND._serialized_end = 7476 - _STREAMINGQUERYMANAGERCOMMANDRESULT._serialized_start = 7490 - _STREAMINGQUERYMANAGERCOMMANDRESULT._serialized_end = 8566 - _STREAMINGQUERYMANAGERCOMMANDRESULT_ACTIVERESULT._serialized_start = 8098 - _STREAMINGQUERYMANAGERCOMMANDRESULT_ACTIVERESULT._serialized_end = 8225 - _STREAMINGQUERYMANAGERCOMMANDRESULT_STREAMINGQUERYINSTANCE._serialized_start = 8227 - _STREAMINGQUERYMANAGERCOMMANDRESULT_STREAMINGQUERYINSTANCE._serialized_end = 8342 - _STREAMINGQUERYMANAGERCOMMANDRESULT_AWAITANYTERMINATIONRESULT._serialized_start = 8344 - _STREAMINGQUERYMANAGERCOMMANDRESULT_AWAITANYTERMINATIONRESULT._serialized_end = 8403 - _STREAMINGQUERYMANAGERCOMMANDRESULT_STREAMINGQUERYLISTENERINSTANCE._serialized_start = 8405 - _STREAMINGQUERYMANAGERCOMMANDRESULT_STREAMINGQUERYLISTENERINSTANCE._serialized_end = 8480 - _STREAMINGQUERYMANAGERCOMMANDRESULT_LISTSTREAMINGQUERYLISTENERRESULT._serialized_start = 8482 - _STREAMINGQUERYMANAGERCOMMANDRESULT_LISTSTREAMINGQUERYLISTENERRESULT._serialized_end = 8551 - _GETRESOURCESCOMMAND._serialized_start = 8568 - _GETRESOURCESCOMMAND._serialized_end = 8589 - _GETRESOURCESCOMMANDRESULT._serialized_start = 8592 - _GETRESOURCESCOMMANDRESULT._serialized_end = 8804 - _GETRESOURCESCOMMANDRESULT_RESOURCESENTRY._serialized_start = 8708 - _GETRESOURCESCOMMANDRESULT_RESOURCESENTRY._serialized_end = 8804 + _WRITEOPERATION._serialized_end = 2932 + _WRITEOPERATION_OPTIONSENTRY._serialized_start = 2356 + _WRITEOPERATION_OPTIONSENTRY._serialized_end = 2414 + _WRITEOPERATION_SAVETABLE._serialized_start = 2417 + _WRITEOPERATION_SAVETABLE._serialized_end = 2675 + _WRITEOPERATION_SAVETABLE_TABLESAVEMETHOD._serialized_start = 2551 + _WRITEOPERATION_SAVETABLE_TABLESAVEMETHOD._serialized_end = 2675 + _WRITEOPERATION_BUCKETBY._serialized_start = 2677 + _WRITEOPERATION_BUCKETBY._serialized_end = 2768 + _WRITEOPERATION_SAVEMODE._serialized_start = 2771 + _WRITEOPERATION_SAVEMODE._serialized_end = 2908 + _WRITEOPERATIONV2._serialized_start = 2935 + _WRITEOPERATIONV2._serialized_end = 3795 + _WRITEOPERATIONV2_OPTIONSENTRY._serialized_start = 2356 + _WRITEOPERATIONV2_OPTIONSENTRY._serialized_end = 2414 + _WRITEOPERATIONV2_TABLEPROPERTIESENTRY._serialized_start = 3554 + _WRITEOPERATIONV2_TABLEPROPERTIESENTRY._serialized_end = 3620 + _WRITEOPERATIONV2_MODE._serialized_start = 3623 + _WRITEOPERATIONV2_MODE._serialized_end = 3782 + _WRITESTREAMOPERATIONSTART._serialized_start = 3798 + _WRITESTREAMOPERATIONSTART._serialized_end = 4598 + _WRITESTREAMOPERATIONSTART_OPTIONSENTRY._serialized_start = 2356 + _WRITESTREAMOPERATIONSTART_OPTIONSENTRY._serialized_end = 2414 + _STREAMINGFOREACHFUNCTION._serialized_start = 4601 + _STREAMINGFOREACHFUNCTION._serialized_end = 4780 + _WRITESTREAMOPERATIONSTARTRESULT._serialized_start = 4782 + _WRITESTREAMOPERATIONSTARTRESULT._serialized_end = 4903 + _STREAMINGQUERYINSTANCEID._serialized_start = 4905 + _STREAMINGQUERYINSTANCEID._serialized_end = 4970 + _STREAMINGQUERYCOMMAND._serialized_start = 4973 + _STREAMINGQUERYCOMMAND._serialized_end = 5605 + _STREAMINGQUERYCOMMAND_EXPLAINCOMMAND._serialized_start = 5472 + _STREAMINGQUERYCOMMAND_EXPLAINCOMMAND._serialized_end = 5516 + _STREAMINGQUERYCOMMAND_AWAITTERMINATIONCOMMAND._serialized_start = 5518 + _STREAMINGQUERYCOMMAND_AWAITTERMINATIONCOMMAND._serialized_end = 5594 + _STREAMINGQUERYCOMMANDRESULT._serialized_start = 5608 + _STREAMINGQUERYCOMMANDRESULT._serialized_end = 6749 + _STREAMINGQUERYCOMMANDRESULT_STATUSRESULT._serialized_start = 6191 + _STREAMINGQUERYCOMMANDRESULT_STATUSRESULT._serialized_end = 6361 + _STREAMINGQUERYCOMMANDRESULT_RECENTPROGRESSRESULT._serialized_start = 6363 + _STREAMINGQUERYCOMMANDRESULT_RECENTPROGRESSRESULT._serialized_end = 6435 + _STREAMINGQUERYCOMMANDRESULT_EXPLAINRESULT._serialized_start = 6437 + _STREAMINGQUERYCOMMANDRESULT_EXPLAINRESULT._serialized_end = 6476 + _STREAMINGQUERYCOMMANDRESULT_EXCEPTIONRESULT._serialized_start = 6479 + _STREAMINGQUERYCOMMANDRESULT_EXCEPTIONRESULT._serialized_end = 6676 + _STREAMINGQUERYCOMMANDRESULT_AWAITTERMINATIONRESULT._serialized_start = 6678 + _STREAMINGQUERYCOMMANDRESULT_AWAITTERMINATIONRESULT._serialized_end = 6734 + _STREAMINGQUERYMANAGERCOMMAND._serialized_start = 6752 + _STREAMINGQUERYMANAGERCOMMAND._serialized_end = 7581 + _STREAMINGQUERYMANAGERCOMMAND_AWAITANYTERMINATIONCOMMAND._serialized_start = 7283 + _STREAMINGQUERYMANAGERCOMMAND_AWAITANYTERMINATIONCOMMAND._serialized_end = 7362 + _STREAMINGQUERYMANAGERCOMMAND_STREAMINGQUERYLISTENERCOMMAND._serialized_start = 7365 + _STREAMINGQUERYMANAGERCOMMAND_STREAMINGQUERYLISTENERCOMMAND._serialized_end = 7570 + _STREAMINGQUERYMANAGERCOMMANDRESULT._serialized_start = 7584 + _STREAMINGQUERYMANAGERCOMMANDRESULT._serialized_end = 8660 + _STREAMINGQUERYMANAGERCOMMANDRESULT_ACTIVERESULT._serialized_start = 8192 + _STREAMINGQUERYMANAGERCOMMANDRESULT_ACTIVERESULT._serialized_end = 8319 + _STREAMINGQUERYMANAGERCOMMANDRESULT_STREAMINGQUERYINSTANCE._serialized_start = 8321 + _STREAMINGQUERYMANAGERCOMMANDRESULT_STREAMINGQUERYINSTANCE._serialized_end = 8436 + _STREAMINGQUERYMANAGERCOMMANDRESULT_AWAITANYTERMINATIONRESULT._serialized_start = 8438 + _STREAMINGQUERYMANAGERCOMMANDRESULT_AWAITANYTERMINATIONRESULT._serialized_end = 8497 + _STREAMINGQUERYMANAGERCOMMANDRESULT_STREAMINGQUERYLISTENERINSTANCE._serialized_start = 8499 + _STREAMINGQUERYMANAGERCOMMANDRESULT_STREAMINGQUERYLISTENERINSTANCE._serialized_end = 8574 + _STREAMINGQUERYMANAGERCOMMANDRESULT_LISTSTREAMINGQUERYLISTENERRESULT._serialized_start = 8576 + _STREAMINGQUERYMANAGERCOMMANDRESULT_LISTSTREAMINGQUERYLISTENERRESULT._serialized_end = 8645 + _GETRESOURCESCOMMAND._serialized_start = 8662 + _GETRESOURCESCOMMAND._serialized_end = 8683 + _GETRESOURCESCOMMANDRESULT._serialized_start = 8686 + _GETRESOURCESCOMMANDRESULT._serialized_end = 8898 + _GETRESOURCESCOMMANDRESULT_RESOURCESENTRY._serialized_start = 8802 + _GETRESOURCESCOMMANDRESULT_RESOURCESENTRY._serialized_end = 8898 # @@protoc_insertion_point(module_scope) diff --git a/python/pyspark/sql/connect/proto/commands_pb2.pyi b/python/pyspark/sql/connect/proto/commands_pb2.pyi index 7855f403e9c92..ce7f3e5abfdbb 100644 --- a/python/pyspark/sql/connect/proto/commands_pb2.pyi +++ b/python/pyspark/sql/connect/proto/commands_pb2.pyi @@ -489,6 +489,7 @@ class WriteOperation(google.protobuf.message.Message): PARTITIONING_COLUMNS_FIELD_NUMBER: builtins.int BUCKET_BY_FIELD_NUMBER: builtins.int OPTIONS_FIELD_NUMBER: builtins.int + CLUSTERING_COLUMNS_FIELD_NUMBER: builtins.int @property def input(self) -> pyspark.sql.connect.proto.relations_pb2.Relation: """(Required) The output of the `input` relation will be persisted according to the options.""" @@ -517,6 +518,11 @@ class WriteOperation(google.protobuf.message.Message): @property def options(self) -> google.protobuf.internal.containers.ScalarMap[builtins.str, builtins.str]: """(Optional) A list of configuration options.""" + @property + def clustering_columns( + self, + ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.str]: + """(Optional) Columns used for clustering the table.""" def __init__( self, *, @@ -529,6 +535,7 @@ class WriteOperation(google.protobuf.message.Message): partitioning_columns: collections.abc.Iterable[builtins.str] | None = ..., bucket_by: global___WriteOperation.BucketBy | None = ..., options: collections.abc.Mapping[builtins.str, builtins.str] | None = ..., + clustering_columns: collections.abc.Iterable[builtins.str] | None = ..., ) -> None: ... def HasField( self, @@ -556,6 +563,8 @@ class WriteOperation(google.protobuf.message.Message): b"_source", "bucket_by", b"bucket_by", + "clustering_columns", + b"clustering_columns", "input", b"input", "mode", @@ -662,6 +671,7 @@ class WriteOperationV2(google.protobuf.message.Message): TABLE_PROPERTIES_FIELD_NUMBER: builtins.int MODE_FIELD_NUMBER: builtins.int OVERWRITE_CONDITION_FIELD_NUMBER: builtins.int + CLUSTERING_COLUMNS_FIELD_NUMBER: builtins.int @property def input(self) -> pyspark.sql.connect.proto.relations_pb2.Relation: """(Required) The output of the `input` relation will be persisted according to the options.""" @@ -693,6 +703,11 @@ class WriteOperationV2(google.protobuf.message.Message): @property def overwrite_condition(self) -> pyspark.sql.connect.proto.expressions_pb2.Expression: """(Optional) A condition for overwrite saving mode""" + @property + def clustering_columns( + self, + ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.str]: + """(Optional) Columns used for clustering the table.""" def __init__( self, *, @@ -707,6 +722,7 @@ class WriteOperationV2(google.protobuf.message.Message): table_properties: collections.abc.Mapping[builtins.str, builtins.str] | None = ..., mode: global___WriteOperationV2.Mode.ValueType = ..., overwrite_condition: pyspark.sql.connect.proto.expressions_pb2.Expression | None = ..., + clustering_columns: collections.abc.Iterable[builtins.str] | None = ..., ) -> None: ... def HasField( self, @@ -726,6 +742,8 @@ class WriteOperationV2(google.protobuf.message.Message): field_name: typing_extensions.Literal[ "_provider", b"_provider", + "clustering_columns", + b"clustering_columns", "input", b"input", "mode", diff --git a/python/pyspark/sql/tests/test_types.py b/python/pyspark/sql/tests/test_types.py index c6e70da1f8dcb..b038cf6ce5baa 100644 --- a/python/pyspark/sql/tests/test_types.py +++ b/python/pyspark/sql/tests/test_types.py @@ -1579,6 +1579,13 @@ def test_row_without_field_sorting(self): self.assertEqual(r, expected) self.assertEqual(repr(r), "Row(b=1, a=2)") + def test_struct_field_from_json(self): + # SPARK-40820: fromJson with only name and type + json = {"name": "c1", "type": "string"} + struct_field = StructField.fromJson(json) + + self.assertEqual(repr(struct_field), "StructField('c1', StringType(), True)") + class TypesTests(TypesTestsMixin, ReusedSQLTestCase): pass diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index d6862d7178abf..698a54999d451 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -845,8 +845,8 @@ def fromJson(cls, json: Dict[str, Any]) -> "StructField": return StructField( json["name"], _parse_datatype_json_value(json["type"]), - json["nullable"], - json["metadata"], + json.get("nullable", True), + json.get("metadata"), ) def needConversion(self) -> bool: diff --git a/sql/api/src/main/java/org/apache/spark/sql/streaming/Trigger.java b/sql/api/src/main/java/org/apache/spark/sql/streaming/Trigger.java index 328c5290e7789..6e3a93ba94545 100644 --- a/sql/api/src/main/java/org/apache/spark/sql/streaming/Trigger.java +++ b/sql/api/src/main/java/org/apache/spark/sql/streaming/Trigger.java @@ -98,7 +98,7 @@ public static Trigger ProcessingTime(String interval) { * processing of watermark advancement including no-data batch. * See the NOTES in {@link #AvailableNow()} for details. */ - @Deprecated + @Deprecated(since = "3.4.0") public static Trigger Once() { return OneTimeTrigger$.MODULE$; } 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 2bd88d5975634..5f563e3b7a8f1 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 @@ -258,6 +258,11 @@ object DataType { ("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)) => + StructField(name, parseDataType(dataType)) case other => throw new IllegalArgumentException( s"Failed to convert the JSON string '${compact(render(other))}' to a field.") diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionDescription.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionDescription.java index e9ccf2b56f82e..833d68bdf897a 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionDescription.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionDescription.java @@ -102,7 +102,7 @@ * {@link #examples}, {@link #note}, {@link #since} and {@link #deprecated} instead * to document the extended usage. */ - @Deprecated + @Deprecated(since = "3.0.0") String extended() default ""; String arguments() default ""; String examples() default ""; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionInfo.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionInfo.java index 84ee30440b77e..e7af5a68b4663 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionInfo.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionInfo.java @@ -192,7 +192,7 @@ public ExpressionInfo(String className, String db, String name) { * @deprecated This constructor is deprecated as of Spark 3.0. Use other constructors to fully * specify each argument for extended usage. */ - @Deprecated + @Deprecated(since = "3.0.0") public ExpressionInfo(String className, String db, String name, String usage, String extended) { // `arguments` and `examples` are concatenated for the extended description. So, here // simply pass the `extended` as `arguments` and an empty string for `examples`. diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/StagingTableCatalog.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/StagingTableCatalog.java index 4337a7c615208..e69fc3324c691 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/StagingTableCatalog.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/StagingTableCatalog.java @@ -57,10 +57,10 @@ public interface StagingTableCatalog extends TableCatalog { /** * Stage the creation of a table, preparing it to be committed into the metastore. *

- * This is deprecated. Please override + * @deprecated This is deprecated. Please override * {@link #stageCreate(Identifier, Column[], Transform[], Map)} instead. */ - @Deprecated + @Deprecated(since = "3.4.0") StagedTable stageCreate( Identifier ident, StructType schema, diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Table.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Table.java index b9753a08aba96..d5eb03dcf94d4 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Table.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Table.java @@ -52,9 +52,9 @@ public interface Table { * Returns the schema of this table. If the table is not readable and doesn't have a schema, an * empty schema can be returned here. *

- * This is deprecated. Please override {@link #columns} instead. + * @deprecated This is deprecated. Please override {@link #columns} instead. */ - @Deprecated + @Deprecated(since = "3.4.0") StructType schema(); /** diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java index d99e7e14b0117..6642adc33548d 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java @@ -170,10 +170,10 @@ default boolean tableExists(Identifier ident) { /** * Create a table in the catalog. *

- * This is deprecated. Please override + * @deprecated This is deprecated. Please override * {@link #createTable(Identifier, Column[], Transform[], Map)} instead. */ - @Deprecated + @Deprecated(since = "3.4.0") Table createTable( Identifier ident, StructType schema, diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/WriteBuilder.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/WriteBuilder.java index 51ca167a4c0df..c8b20ec7e92fd 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/WriteBuilder.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/WriteBuilder.java @@ -56,7 +56,7 @@ public StreamingWrite toStreaming() { * * @deprecated use {@link #build()} instead. */ - @Deprecated + @Deprecated(since = "3.2.0") default BatchWrite buildForBatch() { throw new UnsupportedOperationException(getClass().getName() + " does not support batch write"); @@ -67,7 +67,7 @@ default BatchWrite buildForBatch() { * * @deprecated use {@link #build()} instead. */ - @Deprecated + @Deprecated(since = "3.2.0") default StreamingWrite buildForStreaming() { throw new UnsupportedOperationException(getClass().getName() + " does not support streaming write"); diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 06d949ece2626..65338f9917bc4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -1122,7 +1122,8 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor case r @ RelationTimeTravel(u: UnresolvedRelation, timestamp, version) if timestamp.forall(ts => ts.resolved && !SubqueryExpression.hasSubquery(ts)) => - resolveRelation(u, TimeTravelSpec.create(timestamp, version, conf)).getOrElse(r) + val timeTravelSpec = TimeTravelSpec.create(timestamp, version, conf.sessionLocalTimeZone) + resolveRelation(u, timeTravelSpec).getOrElse(r) case u @ UnresolvedTable(identifier, cmd, suggestAlternative) => lookupTableOrView(identifier).map { @@ -1255,17 +1256,27 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor private def resolveRelation( u: UnresolvedRelation, timeTravelSpec: Option[TimeTravelSpec] = None): Option[LogicalPlan] = { - resolveTempView(u.multipartIdentifier, u.isStreaming, timeTravelSpec.isDefined).orElse { + val timeTravelSpecFromOptions = TimeTravelSpec.fromOptions( + u.options, + conf.getConf(SQLConf.TIME_TRAVEL_TIMESTAMP_KEY), + conf.getConf(SQLConf.TIME_TRAVEL_VERSION_KEY), + conf.sessionLocalTimeZone + ) + if (timeTravelSpec.nonEmpty && timeTravelSpecFromOptions.nonEmpty) { + throw new AnalysisException("MULTIPLE_TIME_TRAVEL_SPEC", Map.empty[String, String]) + } + val finalTimeTravelSpec = timeTravelSpec.orElse(timeTravelSpecFromOptions) + resolveTempView(u.multipartIdentifier, u.isStreaming, finalTimeTravelSpec.isDefined).orElse { expandIdentifier(u.multipartIdentifier) match { case CatalogAndIdentifier(catalog, ident) => - val key = ((catalog.name +: ident.namespace :+ ident.name).toSeq, timeTravelSpec) + val key = ((catalog.name +: ident.namespace :+ ident.name).toSeq, finalTimeTravelSpec) AnalysisContext.get.relationCache.get(key).map(_.transform { case multi: MultiInstanceRelation => val newRelation = multi.newInstance() newRelation.copyTagsFrom(multi) newRelation }).orElse { - val table = CatalogV2Util.loadTable(catalog, ident, timeTravelSpec) + val table = CatalogV2Util.loadTable(catalog, ident, finalTimeTravelSpec) val loaded = createRelation(catalog, ident, table, u.options, u.isStreaming) loaded.foreach(AnalysisContext.get.relationCache.update(key, _)) loaded diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TimeTravelSpec.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TimeTravelSpec.scala index 26856d9a5e089..8bfcd955497b9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TimeTravelSpec.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TimeTravelSpec.scala @@ -17,10 +17,11 @@ package org.apache.spark.sql.catalyst.analysis -import org.apache.spark.sql.catalyst.expressions.{Cast, Expression, RuntimeReplaceable, SubqueryExpression, Unevaluable} +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.expressions.{Cast, Expression, Literal, RuntimeReplaceable, SubqueryExpression, Unevaluable} import org.apache.spark.sql.errors.QueryCompilationErrors -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.TimestampType +import org.apache.spark.sql.util.CaseInsensitiveStringMap sealed trait TimeTravelSpec @@ -31,7 +32,7 @@ object TimeTravelSpec { def create( timestamp: Option[Expression], version: Option[String], - conf: SQLConf) : Option[TimeTravelSpec] = { + sessionLocalTimeZone: String) : Option[TimeTravelSpec] = { if (timestamp.nonEmpty && version.nonEmpty) { throw QueryCompilationErrors.invalidTimeTravelSpecError() } else if (timestamp.nonEmpty) { @@ -50,7 +51,7 @@ object TimeTravelSpec { throw QueryCompilationErrors.invalidTimestampExprForTimeTravel( "INVALID_TIME_TRAVEL_TIMESTAMP_EXPR.NON_DETERMINISTIC", ts) } - val tz = Some(conf.sessionLocalTimeZone) + val tz = Some(sessionLocalTimeZone) // Set `ansiEnabled` to false, so that it can return null for invalid input and we can provide // better error message. val value = Cast(tsToEval, TimestampType, tz, ansiEnabled = false).eval() @@ -65,4 +66,35 @@ object TimeTravelSpec { None } } + + def fromOptions( + options: CaseInsensitiveStringMap, + timestampKey: String, + versionKey: String, + sessionLocalTimeZone: String): Option[TimeTravelSpec] = { + (Option(options.get(timestampKey)), Option(options.get(versionKey))) match { + case (Some(_), Some(_)) => + throw QueryCompilationErrors.invalidTimeTravelSpecError() + + case (Some(timestampStr), None) => + val timestampValue = Cast( + Literal(timestampStr), + TimestampType, + Some(sessionLocalTimeZone), + ansiEnabled = false + ).eval() + if (timestampValue == null) { + throw new AnalysisException( + "INVALID_TIME_TRAVEL_TIMESTAMP_EXPR.OPTION", + Map("expr" -> s"'$timestampStr'") + ) + } + Some(AsOfTimestamp(timestampValue.asInstanceOf[Long])) + + case (None, Some(versionStr)) => + Some(AsOfVersion(versionStr)) + + case _ => None + } + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala index 759000bc5f52a..0a080423b10fe 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala @@ -1768,7 +1768,7 @@ case class Slice(x: Expression, start: Expression, length: Expression) startInt - 1 } if (lengthInt < 0) { - throw QueryExecutionErrors.unexpectedValueForLengthInFunctionError(prettyName) + throw QueryExecutionErrors.unexpectedValueForLengthInFunctionError(prettyName, lengthInt) } // startIndex can be negative if start is negative and its absolute value is greater than the // number of elements in the array @@ -1796,7 +1796,8 @@ case class Slice(x: Expression, start: Expression, length: Expression) | $startIdx = $start - 1; |} |if ($length < 0) { - | throw QueryExecutionErrors.unexpectedValueForLengthInFunctionError("$prettyName"); + | throw QueryExecutionErrors.unexpectedValueForLengthInFunctionError( + | "$prettyName", $length); |} else if ($length > $x.numElements() - $startIdx) { | $resLength = $x.numElements() - $startIdx; |} else { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index 6d1aa076eb081..ae5ebd6a97489 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -3343,7 +3343,7 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat def invalidTimeTravelSpecError(): Throwable = { new AnalysisException( - errorClass = "_LEGACY_ERROR_TEMP_1334", + errorClass = "INVALID_TIME_TRAVEL_SPEC", messageParameters = Map.empty) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala index a11b929919dd9..afc244509c41d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala @@ -1368,25 +1368,27 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase with ExecutionE } def unsupportedOperandTypeForSizeFunctionError( - dataType: DataType): SparkUnsupportedOperationException = { - new SparkUnsupportedOperationException( - errorClass = "_LEGACY_ERROR_TEMP_2156", - messageParameters = Map( - "dataType" -> dataType.getClass.getCanonicalName)) + dataType: DataType): Throwable = { + SparkException.internalError( + s"The size function doesn't support the operand type ${toSQLType(dataType)}") } def unexpectedValueForStartInFunctionError(prettyName: String): SparkRuntimeException = { new SparkRuntimeException( - errorClass = "_LEGACY_ERROR_TEMP_2157", + errorClass = "INVALID_PARAMETER_VALUE.START", messageParameters = Map( - "prettyName" -> prettyName)) + "parameter" -> toSQLId("start"), + "functionName" -> toSQLId(prettyName))) } - def unexpectedValueForLengthInFunctionError(prettyName: String): SparkRuntimeException = { + def unexpectedValueForLengthInFunctionError( + prettyName: String, length: Int): SparkRuntimeException = { new SparkRuntimeException( - errorClass = "_LEGACY_ERROR_TEMP_2158", + errorClass = "INVALID_PARAMETER_VALUE.LENGTH", messageParameters = Map( - "prettyName" -> prettyName)) + "parameter" -> toSQLId("length"), + "length" -> length.toString, + "functionName" -> toSQLId(prettyName))) } def invalidIndexOfZeroError(context: SQLQueryContext): RuntimeException = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 1e759b6266c61..43dc541fbb9fc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -4474,6 +4474,20 @@ object SQLConf { .booleanConf .createWithDefault(true) + val TIME_TRAVEL_TIMESTAMP_KEY = + buildConf("spark.sql.timeTravelTimestampKey") + .doc("The option name to specify the time travel timestamp when reading a table.") + .version("4.0.0") + .stringConf + .createWithDefault("timestampAsOf") + + val TIME_TRAVEL_VERSION_KEY = + buildConf("spark.sql.timeTravelVersionKey") + .doc("The option name to specify the time travel table version when reading a table.") + .version("4.0.0") + .stringConf + .createWithDefault("versionAsOf") + val LEGACY_PERCENTILE_DISC_CALCULATION = buildConf("spark.sql.legacy.percentileDiscCalculation") .internal() .doc("If true, the old bogus percentile_disc calculation is used. The old calculation " + diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala index ff393857c31ce..b3a21bd9565f7 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala @@ -32,12 +32,14 @@ import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.DataTypeMismatch import org.apache.spark.sql.catalyst.util.{DateTimeTestUtils, DateTimeUtils} import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.{outstandingZoneIds, LA, UTC} import org.apache.spark.sql.catalyst.util.IntervalUtils._ +import org.apache.spark.sql.errors.DataTypeErrorsBase import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.array.ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH import org.apache.spark.unsafe.types.UTF8String -class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { +class CollectionExpressionsSuite + extends SparkFunSuite with ExpressionEvalHelper with DataTypeErrorsBase { implicit def stringToUTF8Str(str: String): UTF8String = UTF8String.fromString(str) @@ -86,6 +88,19 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper } } + test("Unsupported data type for size()") { + val exception = intercept[org.apache.spark.SparkException] { + Size(Literal.create("str", StringType)).eval(EmptyRow) + } + checkError( + exception = exception, + errorClass = "INTERNAL_ERROR", + parameters = Map( + "message" -> ("The size function doesn't support the operand type " + + toSQLType(StringType)) + )) + } + test("MapKeys/MapValues") { val m0 = Literal.create(Map("a" -> "1", "b" -> "2"), MapType(StringType, StringType)) val m1 = Literal.create(Map[String, String](), MapType(StringType, StringType)) @@ -600,10 +615,21 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(Slice(a0, Literal(-3), Literal(2)), Seq(4, 5)) checkEvaluation(Slice(a0, Literal(4), Literal(10)), Seq(4, 5, 6)) checkEvaluation(Slice(a0, Literal(-1), Literal(2)), Seq(6)) - checkExceptionInExpression[RuntimeException](Slice(a0, Literal(1), Literal(-1)), - "Unexpected value for length") - checkExceptionInExpression[RuntimeException](Slice(a0, Literal(0), Literal(1)), - "Unexpected value for start") + checkErrorInExpression[SparkRuntimeException]( + expression = Slice(a0, Literal(1), Literal(-1)), + errorClass = "INVALID_PARAMETER_VALUE.LENGTH", + parameters = Map( + "parameter" -> toSQLId("length"), + "length" -> (-1).toString, + "functionName" -> toSQLId("slice") + )) + checkErrorInExpression[SparkRuntimeException]( + expression = Slice(a0, Literal(0), Literal(1)), + errorClass = "INVALID_PARAMETER_VALUE.START", + parameters = Map( + "parameter" -> toSQLId("start"), + "functionName" -> toSQLId("slice") + )) checkEvaluation(Slice(a0, Literal(-20), Literal(1)), Seq.empty[Int]) checkEvaluation(Slice(a1, Literal(-20), Literal(1)), Seq.empty[String]) checkEvaluation(Slice(a0, Literal.create(null, IntegerType), Literal(2)), null) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratorExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratorExpressionSuite.scala index 49c6625851dd4..dd512b0d83e5c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratorExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratorExpressionSuite.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.types._ class GeneratorExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { private def checkTuple(actual: Expression, expected: Seq[InternalRow]): Unit = { - assert(actual.eval(null).asInstanceOf[IterableOnce[InternalRow]].toSeq === expected) + assert(actual.eval(null).asInstanceOf[IterableOnce[InternalRow]].iterator.to(Seq) === expected) } private final val empty_array = CreateArray(Seq.empty) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala index 94e40b9806520..9b651ef8762e8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala @@ -269,7 +269,7 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with Nil private def checkJsonTuple(jt: JsonTuple, expected: InternalRow): Unit = { - assert(jt.eval(null).toSeq.head === expected) + assert(jt.eval(null).iterator.to(Seq).head === expected) } test("json_tuple escaping") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala index 0e78f875ad7ce..6daeda693002a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala @@ -317,6 +317,29 @@ class DataTypeSuite extends SparkFunSuite { assert(message.contains("Unrecognized token 'abcd'")) } + // SPARK-40820: fromJson with only name and type + test("Deserialized and serialized schema without nullable or metadata in") { + val schema = + """ + |{ + | "type": "struct", + | "fields": [ + | { + | "name": "c1", + | "type": "string" + | } + | ] + |} + |""".stripMargin + val dt = DataType.fromJson(schema) + + dt.simpleString equals "struct" + dt.json equals + """ + |{"type":"struct","fields":[{"name":"c1","type":"string","nullable":false,"metadata":{}}]} + |""".stripMargin + } + def checkDefaultSize(dataType: DataType, expectedDefaultSize: Int): Unit = { test(s"Check the default size of $dataType") { assert(dataType.defaultSize === expectedDefaultSize) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetCompressionCodec.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetCompressionCodec.java new file mode 100644 index 0000000000000..1a37c7a33f20c --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetCompressionCodec.java @@ -0,0 +1,62 @@ +/* + * 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.execution.datasources.parquet; + +import java.util.Arrays; +import java.util.List; +import java.util.Locale; + +import org.apache.parquet.hadoop.metadata.CompressionCodecName; + +/** + * A mapper class from Spark supported parquet compression codecs to parquet compression codecs. + */ +public enum ParquetCompressionCodec { + NONE(CompressionCodecName.UNCOMPRESSED), + UNCOMPRESSED(CompressionCodecName.UNCOMPRESSED), + SNAPPY(CompressionCodecName.SNAPPY), + GZIP(CompressionCodecName.GZIP), + LZO(CompressionCodecName.LZO), + BROTLI(CompressionCodecName.BROTLI), + LZ4(CompressionCodecName.LZ4), + LZ4_RAW(CompressionCodecName.LZ4_RAW), + ZSTD(CompressionCodecName.ZSTD); + + private final CompressionCodecName compressionCodec; + + ParquetCompressionCodec(CompressionCodecName compressionCodec) { + this.compressionCodec = compressionCodec; + } + + public CompressionCodecName getCompressionCodec() { + return this.compressionCodec; + } + + public static ParquetCompressionCodec fromString(String s) { + return ParquetCompressionCodec.valueOf(s.toUpperCase(Locale.ROOT)); + } + + public static final List availableCodecs = + Arrays.asList( + ParquetCompressionCodec.UNCOMPRESSED, + ParquetCompressionCodec.SNAPPY, + ParquetCompressionCodec.GZIP, + ParquetCompressionCodec.ZSTD, + ParquetCompressionCodec.LZ4, + ParquetCompressionCodec.LZ4_RAW); +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/expressions/javalang/typed.java b/sql/core/src/main/java/org/apache/spark/sql/expressions/javalang/typed.java index 259328d319c8e..e1e4ba4c8e0dc 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/expressions/javalang/typed.java +++ b/sql/core/src/main/java/org/apache/spark/sql/expressions/javalang/typed.java @@ -32,7 +32,7 @@ * @since 2.0.0 * @deprecated As of release 3.0.0, please use the untyped builtin aggregate functions. */ -@Deprecated +@Deprecated(since = "3.0.0") public class typed { // Note: make sure to keep in sync with typed.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Observation.scala b/sql/core/src/main/scala/org/apache/spark/sql/Observation.scala index 14c4983794bbc..cb6fbfbb2ae3e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Observation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Observation.scala @@ -73,7 +73,10 @@ class Observation(val name: String) { */ private[spark] def on[T](ds: Dataset[T], expr: Column, exprs: Column*): Dataset[T] = { if (ds.isStreaming) { - throw new IllegalArgumentException("Observation does not support streaming Datasets") + throw new IllegalArgumentException("Observation does not support streaming Datasets." + + "This is because there will be multiple observed metrics as microbatches are constructed" + + ". Please register a StreamingQueryListener and get the metric for each microbatch in " + + "QueryProgressEvent.progress, or use query.lastProgress or query.recentProgress.") } register(ds) ds.observe(name, expr, exprs: _*) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala index 987c0668d94a8..49546935da932 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala @@ -97,7 +97,7 @@ case class GenerateExec( // we should always set the left (required child output) joinedRow.withLeft(pruneChildForResult(row)) val outputRows = boundGenerator.eval(row) - if (outer && outputRows.isEmpty) { + if (outer && outputRows.iterator.isEmpty) { joinedRow.withRight(generatorNullRow) :: Nil } else { outputRows.iterator.map(joinedRow.withRight) @@ -110,7 +110,7 @@ case class GenerateExec( } else { iter.flatMap { row => val outputRows = boundGenerator.eval(row) - if (outer && outputRows.isEmpty) { + if (outer && outputRows.iterator.isEmpty) { Seq(generatorNullRow) } else { outputRows diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index bbd74a1fe7407..d93a83dec44ad 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -536,13 +536,13 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ while (buf.length < n && i < res.length) { val rows = decodeUnsafeRows(res(i)._2) if (n - buf.length >= res(i)._1) { - buf.prepend(rows.toArray[InternalRow]: _*) + buf.prependAll(rows.toArray[InternalRow]) } else { val dropUntil = res(i)._1 - (n - buf.length) // Same as Iterator.drop but this only takes a long. var j: Long = 0L while (j < dropUntil) { rows.next(); j += 1L} - buf.prepend(rows.toArray[InternalRow]: _*) + buf.prependAll(rows.toArray[InternalRow]) } i += 1 } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala index a0dd939dda232..d6bf1e29edddd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala @@ -315,7 +315,7 @@ private[sql] object ArrowConverters extends Logging { val reader = new ArrowStreamReader(new ByteArrayInputStream(arrowBatchIter.next()), allocator) val root = if (reader.loadNextBatch()) reader.getVectorSchemaRoot else null - resources.append(reader, root) + resources.appendAll(Seq(reader, root)) if (root == null) { (Iterator.empty, null) } else { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala index 559a994319d3e..ae110fdd0d3a3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala @@ -20,7 +20,6 @@ package org.apache.spark.sql.execution.datasources.parquet import java.util.Locale import org.apache.parquet.hadoop.ParquetOutputFormat -import org.apache.parquet.hadoop.metadata.CompressionCodecName import org.apache.spark.sql.catalyst.{DataSourceOptions, FileSourceOptions} import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap @@ -88,16 +87,10 @@ class ParquetOptions( object ParquetOptions extends DataSourceOptions { // The parquet compression short names - private val shortParquetCompressionCodecNames = Map( - "none" -> CompressionCodecName.UNCOMPRESSED, - "uncompressed" -> CompressionCodecName.UNCOMPRESSED, - "snappy" -> CompressionCodecName.SNAPPY, - "gzip" -> CompressionCodecName.GZIP, - "lzo" -> CompressionCodecName.LZO, - "brotli" -> CompressionCodecName.BROTLI, - "lz4" -> CompressionCodecName.LZ4, - "lz4_raw" -> CompressionCodecName.LZ4_RAW, - "zstd" -> CompressionCodecName.ZSTD) + private val shortParquetCompressionCodecNames = + ParquetCompressionCodec.values().map { + codec => codec.name().toLowerCase(Locale.ROOT) -> codec.getCompressionCodec + }.toMap def getParquetCompressionCodecName(name: String): String = { shortParquetCompressionCodecNames(name).name() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExec.scala index 094a7b20808ab..afcc762e636a3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExec.scala @@ -100,7 +100,7 @@ case class BatchScanExec( "partition values that are not present in the original partitioning.") } - groupPartitions(newPartitions).get.groupedParts.map(_.parts) + groupPartitions(newPartitions).map(_.groupedParts.map(_.parts)).getOrElse(Seq.empty) case _ => // no validation is needed as the data source did not report any specific partitioning diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Utils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Utils.scala index e485300615780..c4e7bf23cace7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Utils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Utils.scala @@ -133,7 +133,8 @@ private[sql] object DataSourceV2Utils extends Logging { } else { None } - val timeTravel = TimeTravelSpec.create(timeTravelTimestamp, timeTravelVersion, conf) + val timeTravel = TimeTravelSpec.create( + timeTravelTimestamp, timeTravelVersion, conf.sessionLocalTimeZone) (CatalogV2Util.getTable(catalog, ident, timeTravel), Some(catalog), Some(ident)) case _ => // TODO: Non-catalog paths for DSV2 are currently not well defined. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala index 31bfa5aff351f..2fefd8f70cd5c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala @@ -421,7 +421,7 @@ case class MapGroupsExec( val result = func( getKey(key), rowIter.map(getValue)) - result.map(outputObject) + result.iterator.map(outputObject) } } } @@ -653,7 +653,7 @@ case class CoGroupExec( getKey(key), leftResult.map(getLeft), rightResult.map(getRight)) - result.map(outputObject) + result.iterator.map(outputObject) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTextSocketSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTextSocketSource.scala index d1b346b4174bb..3f5f81bee4e85 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTextSocketSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTextSocketSource.scala @@ -144,7 +144,7 @@ class TextSocketContinuousStream( " for partition " + partition + ". Max valid offset: " + max) } val n = offset - startOffset.offsets(partition) - buckets(partition).trimStart(n) + buckets(partition).dropInPlace(n) } startOffset = endOffset recordEndpoint.setStartOffsets(startOffset.offsets) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala index f81fac8e892b2..826543fd56525 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala @@ -189,7 +189,7 @@ case class MemoryStream[A : Encoder]( protected var lastOffsetCommitted : LongOffset = new LongOffset(-1) def addData(data: IterableOnce[A]): Offset = { - val objects = data.toSeq + val objects = data.iterator.to(Seq) val rows = objects.iterator.map(d => toRow(d).copy().asInstanceOf[UnsafeRow]).toArray logDebug(s"Adding: $objects") this.synchronized { @@ -279,7 +279,7 @@ case class MemoryStream[A : Encoder]( s"Offsets committed out of order: $lastOffsetCommitted followed by $end") } - batches.trimStart(offsetDiff) + batches.dropInPlace(offsetDiff) lastOffsetCommitted = newOffset } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ContinuousMemoryStream.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ContinuousMemoryStream.scala index 028884e8f34ca..dc97386d8fcf1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ContinuousMemoryStream.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ContinuousMemoryStream.scala @@ -58,7 +58,7 @@ class ContinuousMemoryStream[A : Encoder](id: Int, sqlContext: SQLContext, numPa def addData(data: IterableOnce[A]): Offset = synchronized { // Distribute data evenly among partition lists. - data.toSeq.zipWithIndex.map { + data.iterator.to(Seq).zipWithIndex.map { case (item, index) => records(index % numPartitions) += toRow(item).copy().asInstanceOf[UnsafeRow] } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketMicroBatchStream.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketMicroBatchStream.scala index 580f7066e448e..a01f40bead893 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketMicroBatchStream.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketMicroBatchStream.scala @@ -159,7 +159,7 @@ class TextSocketMicroBatchStream(host: String, port: Int, numPartitions: Int) s"Offsets committed out of order: $lastOffsetCommitted followed by $end") } - batches.trimStart(offsetDiff) + batches.dropInPlace(offsetDiff) lastOffsetCommitted = newOffset } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowFunctionFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowFunctionFrame.scala index a849c3894f0d6..d49e5ed56626e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowFunctionFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowFunctionFrame.scala @@ -86,7 +86,8 @@ abstract class OffsetWindowFunctionFrameBase( expressions: Array[OffsetWindowFunction], inputSchema: Seq[Attribute], newMutableProjection: (Seq[Expression], Seq[Attribute]) => MutableProjection, - offset: Int) + offset: Int, + ignoreNulls: Boolean) extends WindowFunctionFrame { /** Rows of the partition currently being processed. */ @@ -140,6 +141,8 @@ abstract class OffsetWindowFunctionFrameBase( // is not null. protected var skippedNonNullCount = 0 + protected val absOffset = Math.abs(offset) + // Reset the states by the data of the new partition. protected def resetStates(rows: ExternalAppendOnlyUnsafeRowArray): Unit = { input = rows @@ -175,6 +178,31 @@ abstract class OffsetWindowFunctionFrameBase( } } + override def prepare(rows: ExternalAppendOnlyUnsafeRowArray): Unit = { + if (absOffset > rows.length) { + fillDefaultValue(EmptyRow) + } else { + resetStates(rows) + if (ignoreNulls) { + prepareForIgnoreNulls() + } else { + prepareForRespectNulls() + } + } + } + + protected def prepareForIgnoreNulls(): Unit = findNextRowWithNonNullInput() + + protected def prepareForRespectNulls(): Unit + + override def write(index: Int, current: InternalRow): Unit = { + if (input != null) { + doWrite(index, current) + } + } + + protected def doWrite(index: Int, current: InternalRow): Unit + override def currentLowerBound(): Int = throw new UnsupportedOperationException() override def currentUpperBound(): Int = throw new UnsupportedOperationException() @@ -196,24 +224,15 @@ class FrameLessOffsetWindowFunctionFrame( offset: Int, ignoreNulls: Boolean = false) extends OffsetWindowFunctionFrameBase( - target, ordinal, expressions, inputSchema, newMutableProjection, offset) { + target, ordinal, expressions, inputSchema, newMutableProjection, offset, ignoreNulls) { - override def prepare(rows: ExternalAppendOnlyUnsafeRowArray): Unit = { - resetStates(rows) - if (ignoreNulls) { - if (Math.abs(offset) > rows.length) { - fillDefaultValue(EmptyRow) - } else { - findNextRowWithNonNullInput() - } - } else { - // drain the first few rows if offset is larger than zero - while (inputIndex < offset) { - if (inputIterator.hasNext) inputIterator.next() - inputIndex += 1 - } - inputIndex = offset + override def prepareForRespectNulls(): Unit = { + // drain the first few rows if offset is larger than zero + while (inputIndex < offset) { + if (inputIterator.hasNext) inputIterator.next() + inputIndex += 1 } + inputIndex = offset } private val doWrite = if (ignoreNulls && offset > 0) { @@ -260,7 +279,6 @@ class FrameLessOffsetWindowFunctionFrame( // 7. current row -> z, next selected row -> y, output: y; // 8. current row -> v, next selected row -> z, output: z; // 9. current row -> null, next selected row -> v, output: v; - val absOffset = Math.abs(offset) (current: InternalRow) => if (skippedNonNullCount == absOffset) { nextSelectedRow = EmptyRow @@ -294,7 +312,7 @@ class FrameLessOffsetWindowFunctionFrame( inputIndex += 1 } - override def write(index: Int, current: InternalRow): Unit = { + protected def doWrite(index: Int, current: InternalRow): Unit = { doWrite(current) } } @@ -317,35 +335,30 @@ class UnboundedOffsetWindowFunctionFrame( offset: Int, ignoreNulls: Boolean = false) extends OffsetWindowFunctionFrameBase( - target, ordinal, expressions, inputSchema, newMutableProjection, offset) { + target, ordinal, expressions, inputSchema, newMutableProjection, offset, ignoreNulls) { assert(offset > 0) - override def prepare(rows: ExternalAppendOnlyUnsafeRowArray): Unit = { - if (offset > rows.length) { + override def prepareForIgnoreNulls(): Unit = { + findNextRowWithNonNullInput() + if (nextSelectedRow == EmptyRow) { + // Use default values since the offset row whose input value is not null does not exist. fillDefaultValue(EmptyRow) } else { - resetStates(rows) - if (ignoreNulls) { - findNextRowWithNonNullInput() - if (nextSelectedRow == EmptyRow) { - // Use default values since the offset row whose input value is not null does not exist. - fillDefaultValue(EmptyRow) - } else { - projection(nextSelectedRow) - } - } else { - var selectedRow: UnsafeRow = null - // drain the first few rows if offset is larger than one - while (inputIndex < offset) { - selectedRow = WindowFunctionFrame.getNextOrNull(inputIterator) - inputIndex += 1 - } - projection(selectedRow) - } + projection(nextSelectedRow) } } - override def write(index: Int, current: InternalRow): Unit = { + override def prepareForRespectNulls(): Unit = { + var selectedRow: UnsafeRow = null + // drain the first few rows if offset is larger than one + while (inputIndex < offset) { + selectedRow = WindowFunctionFrame.getNextOrNull(inputIterator) + inputIndex += 1 + } + projection(selectedRow) + } + + protected def doWrite(index: Int, current: InternalRow): Unit = { // The results are the same for each row in the partition, and have been evaluated in prepare. // Don't need to recalculate here. } @@ -370,27 +383,18 @@ class UnboundedPrecedingOffsetWindowFunctionFrame( offset: Int, ignoreNulls: Boolean = false) extends OffsetWindowFunctionFrameBase( - target, ordinal, expressions, inputSchema, newMutableProjection, offset) { + target, ordinal, expressions, inputSchema, newMutableProjection, offset, ignoreNulls) { assert(offset > 0) - override def prepare(rows: ExternalAppendOnlyUnsafeRowArray): Unit = { - if (offset > rows.length) { - fillDefaultValue(EmptyRow) - } else { - resetStates(rows) - if (ignoreNulls) { - findNextRowWithNonNullInput() - } else { - // drain the first few rows if offset is larger than one - while (inputIndex < offset) { - nextSelectedRow = WindowFunctionFrame.getNextOrNull(inputIterator) - inputIndex += 1 - } - } + override def prepareForRespectNulls(): Unit = { + // drain the first few rows if offset is larger than one + while (inputIndex < offset) { + nextSelectedRow = WindowFunctionFrame.getNextOrNull(inputIterator) + inputIndex += 1 } } - override def write(index: Int, current: InternalRow): Unit = { + protected def doWrite(index: Int, current: InternalRow): Unit = { if (index >= inputIndex - 1 && nextSelectedRow != null) { projection(nextSelectedRow) } else { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala index 45b80d81e7360..8c293bab64108 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -756,10 +756,10 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan } val expected = new ListBuffer[Row]() - expected.append( + expected.appendAll(Seq( Row(1, "1", 1, 1), Row(1, "1", 1, 2), Row(2, "2", 2, 1), Row(2, "2", 2, 2), - Row(3, "3", 3, 1), Row(3, "3", 3, 2) + Row(3, "3", 3, 1), Row(3, "3", 3, 2)) ) for (i <- 4 to 100) { expected.append(Row(i, i.toString, null, null)) @@ -830,10 +830,10 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan } val expected = new ListBuffer[Row]() - expected.append( + expected.appendAll(Seq( Row(1, "1", 1, 1), Row(1, "1", 1, 2), Row(2, "2", 2, 1), Row(2, "2", 2, 2), - Row(3, "3", 3, 1), Row(3, "3", 3, 2) + Row(3, "3", 3, 1), Row(3, "3", 3, 2)) ) for (i <- 4 to 100) { expected.append(Row(i, i.toString, null, null)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 3cbbc3786fcda..c2e759efe4026 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -2902,10 +2902,14 @@ class DataSourceV2SQLSuiteV1Filter sql(s"INSERT INTO $t2 VALUES (3)") sql(s"INSERT INTO $t2 VALUES (4)") - assert(sql("SELECT * FROM t VERSION AS OF 'Snapshot123456789'").collect() - === Array(Row(1), Row(2))) - assert(sql("SELECT * FROM t VERSION AS OF 2345678910").collect() - === Array(Row(3), Row(4))) + val res1_sql = sql("SELECT * FROM t VERSION AS OF 'Snapshot123456789'").collect() + assert(res1_sql === Array(Row(1), Row(2))) + val res1_df = spark.read.option("versionAsOf", "Snapshot123456789").table("t").collect() + assert(res1_df === Array(Row(1), Row(2))) + val res2_sql = sql("SELECT * FROM t VERSION AS OF 2345678910").collect() + assert(res2_sql === Array(Row(3), Row(4))) + val res2_df = spark.read.option("versionAsOf", "2345678910").table("t").collect() + assert(res2_df === Array(Row(3), Row(4))) } val ts1 = DateTimeUtils.stringToTimestampAnsi( @@ -2928,29 +2932,35 @@ class DataSourceV2SQLSuiteV1Filter sql(s"INSERT INTO $t4 VALUES (7)") sql(s"INSERT INTO $t4 VALUES (8)") - assert(sql("SELECT * FROM t TIMESTAMP AS OF '2019-01-29 00:37:58'").collect() - === Array(Row(5), Row(6))) - assert(sql("SELECT * FROM t TIMESTAMP AS OF '2021-01-29 00:00:00'").collect() - === Array(Row(7), Row(8))) - assert(sql(s"SELECT * FROM t TIMESTAMP AS OF $ts1InSeconds").collect() - === Array(Row(5), Row(6))) - assert(sql(s"SELECT * FROM t TIMESTAMP AS OF $ts2InSeconds").collect() - === Array(Row(7), Row(8))) - assert(sql(s"SELECT * FROM t FOR SYSTEM_TIME AS OF $ts1InSeconds").collect() - === Array(Row(5), Row(6))) - assert(sql(s"SELECT * FROM t FOR SYSTEM_TIME AS OF $ts2InSeconds").collect() - === Array(Row(7), Row(8))) - assert(sql("SELECT * FROM t TIMESTAMP AS OF make_date(2021, 1, 29)").collect() - === Array(Row(7), Row(8))) - assert(sql("SELECT * FROM t TIMESTAMP AS OF to_timestamp('2021-01-29 00:00:00')").collect() - === Array(Row(7), Row(8))) + val res1_sql = sql("SELECT * FROM t TIMESTAMP AS OF '2019-01-29 00:37:58'").collect() + assert(res1_sql === Array(Row(5), Row(6))) + val res1_df = spark.read.option("timestampAsOf", "2019-01-29 00:37:58").table("t").collect() + assert(res1_df === Array(Row(5), Row(6))) + val res2_sql = sql("SELECT * FROM t TIMESTAMP AS OF '2021-01-29 00:00:00'").collect() + assert(res2_sql === Array(Row(7), Row(8))) + val res2_df = spark.read.option("timestampAsOf", "2021-01-29 00:00:00").table("t").collect() + assert(res2_df === Array(Row(7), Row(8))) + + val res3 = sql(s"SELECT * FROM t TIMESTAMP AS OF $ts1InSeconds").collect() + assert(res3 === Array(Row(5), Row(6))) + val res4 = sql(s"SELECT * FROM t TIMESTAMP AS OF $ts2InSeconds").collect() + assert(res4 === Array(Row(7), Row(8))) + val res5 = sql(s"SELECT * FROM t FOR SYSTEM_TIME AS OF $ts1InSeconds").collect() + assert(res5 === Array(Row(5), Row(6))) + val res6 = sql(s"SELECT * FROM t FOR SYSTEM_TIME AS OF $ts2InSeconds").collect() + assert(res6 === Array(Row(7), Row(8))) + val res7 = sql("SELECT * FROM t TIMESTAMP AS OF make_date(2021, 1, 29)").collect() + assert(res7 === Array(Row(7), Row(8))) + val res8 = sql("SELECT * FROM t TIMESTAMP AS OF to_timestamp('2021-01-29 00:00:00')") + .collect() + assert(res8 === Array(Row(7), Row(8))) // Scalar subquery is also supported. - assert(sql("SELECT * FROM t TIMESTAMP AS OF (SELECT make_date(2021, 1, 29))").collect() - === Array(Row(7), Row(8))) + val res9 = sql("SELECT * FROM t TIMESTAMP AS OF (SELECT make_date(2021, 1, 29))").collect() + assert(res9 === Array(Row(7), Row(8))) // Nested subquery also works - assert( - sql("SELECT * FROM t TIMESTAMP AS OF (SELECT (SELECT make_date(2021, 1, 29)))").collect() - === Array(Row(7), Row(8))) + val res10 = sql("SELECT * FROM t TIMESTAMP AS OF (SELECT (SELECT make_date(2021, 1, 29)))") + .collect() + assert(res10 === Array(Row(7), Row(8))) checkError( exception = intercept[AnalysisException] { @@ -2967,6 +2977,23 @@ class DataSourceV2SQLSuiteV1Filter errorClass = "INVALID_TIME_TRAVEL_TIMESTAMP_EXPR.INPUT", parameters = Map("expr" -> "\"abc\"")) + checkError( + exception = intercept[AnalysisException] { + spark.read.option("timestampAsOf", "abc").table("t").collect() + }, + errorClass = "INVALID_TIME_TRAVEL_TIMESTAMP_EXPR.OPTION", + parameters = Map("expr" -> "'abc'")) + + checkError( + exception = intercept[AnalysisException] { + spark.read + .option("timestampAsOf", "abc") + .option("versionAsOf", "1") + .table("t") + .collect() + }, + errorClass = "INVALID_TIME_TRAVEL_SPEC") + checkError( exception = intercept[AnalysisException] { sql("SELECT * FROM t TIMESTAMP AS OF current_user()").collect() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala index 4cb5457b66bae..e6448d4d80fda 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala @@ -1477,4 +1477,46 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { } } } + + test("SPARK-45652: SPJ should handle empty partition after dynamic filtering") { + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false", + SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", + SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", + SQLConf.DYNAMIC_PARTITION_PRUNING_FALLBACK_FILTER_RATIO.key -> "10") { + val items_partitions = Array(identity("id")) + createTable(items, items_schema, items_partitions) + sql(s"INSERT INTO testcat.ns.$items VALUES " + + s"(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + + s"(1, 'aa', 41.0, cast('2020-01-15' as timestamp)), " + + s"(2, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + + s"(2, 'bb', 10.5, cast('2020-01-01' as timestamp)), " + + s"(3, 'cc', 15.5, cast('2020-02-01' as timestamp))") + + val purchases_partitions = Array(identity("item_id")) + createTable(purchases, purchases_schema, purchases_partitions) + sql(s"INSERT INTO testcat.ns.$purchases VALUES " + + s"(1, 42.0, cast('2020-01-01' as timestamp)), " + + s"(1, 44.0, cast('2020-01-15' as timestamp)), " + + s"(1, 45.0, cast('2020-01-15' as timestamp)), " + + s"(2, 11.0, cast('2020-01-01' as timestamp)), " + + s"(3, 19.5, cast('2020-02-01' as timestamp))") + + Seq(true, false).foreach { pushDownValues => + Seq(true, false).foreach { partiallyClustered => { + withSQLConf( + SQLConf.V2_BUCKETING_PARTIALLY_CLUSTERED_DISTRIBUTION_ENABLED.key -> + partiallyClustered.toString, + SQLConf.V2_BUCKETING_PUSH_PART_VALUES_ENABLED.key -> pushDownValues.toString) { + // The dynamic filtering effectively filtered out all the partitions + val df = sql(s"SELECT p.price from testcat.ns.$items i, testcat.ns.$purchases p " + + "WHERE i.id = p.item_id AND i.price > 50.0") + checkAnswer(df, Seq.empty) + } + } + } + } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala index fb1d05f2a9a05..945dd782da0e0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala @@ -37,6 +37,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext import org.apache.spark.sql.catalyst.expressions.objects.InitializeJavaBean import org.apache.spark.sql.catalyst.rules.RuleIdCollection import org.apache.spark.sql.catalyst.util.BadRecordException +import org.apache.spark.sql.errors.DataTypeErrorsBase import org.apache.spark.sql.execution.datasources.jdbc.{DriverRegistry, JDBCOptions} import org.apache.spark.sql.execution.datasources.jdbc.connection.ConnectionProvider import org.apache.spark.sql.execution.datasources.orc.OrcTest @@ -57,7 +58,8 @@ class QueryExecutionErrorsSuite extends QueryTest with ParquetTest with OrcTest - with SharedSparkSession { + with SharedSparkSession + with DataTypeErrorsBase { import testImplicits._ @@ -1064,6 +1066,33 @@ class QueryExecutionErrorsSuite ) } } + + test("Unexpected `start` for slice()") { + checkError( + exception = intercept[SparkRuntimeException] { + sql("select slice(array(1,2,3), 0, 1)").collect() + }, + errorClass = "INVALID_PARAMETER_VALUE.START", + parameters = Map( + "parameter" -> toSQLId("start"), + "functionName" -> toSQLId("slice") + ) + ) + } + + test("Unexpected `length` for slice()") { + checkError( + exception = intercept[SparkRuntimeException] { + sql("select slice(array(1,2,3), 1, -1)").collect() + }, + errorClass = "INVALID_PARAMETER_VALUE.LENGTH", + parameters = Map( + "parameter" -> toSQLId("length"), + "length" -> (-1).toString, + "functionName" -> toSQLId("slice") + ) + ) + } } class FakeFileSystemSetPermission extends LocalFileSystem { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BuiltInDataSourceWriteBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BuiltInDataSourceWriteBenchmark.scala index 4752787c501bf..ba3228878ecee 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BuiltInDataSourceWriteBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BuiltInDataSourceWriteBenchmark.scala @@ -16,9 +16,12 @@ */ package org.apache.spark.sql.execution.benchmark +import java.util.Locale + import org.apache.parquet.column.ParquetProperties import org.apache.parquet.hadoop.ParquetOutputFormat +import org.apache.spark.sql.execution.datasources.parquet.ParquetCompressionCodec import org.apache.spark.sql.internal.SQLConf /** @@ -51,7 +54,8 @@ object BuiltInDataSourceWriteBenchmark extends DataSourceWriteBenchmark { mainArgs } - spark.conf.set(SQLConf.PARQUET_COMPRESSION.key, "snappy") + spark.conf.set(SQLConf.PARQUET_COMPRESSION.key, + ParquetCompressionCodec.SNAPPY.name().toLowerCase(Locale.ROOT)) spark.conf.set(SQLConf.ORC_COMPRESSION.key, "snappy") formats.foreach { format => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DataSourceReadBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DataSourceReadBenchmark.scala index 771f944f1f6c5..a8736c041517f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DataSourceReadBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DataSourceReadBenchmark.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution.benchmark import java.io.File +import java.util.Locale import scala.jdk.CollectionConverters._ import scala.util.Random @@ -28,7 +29,7 @@ import org.apache.spark.{SparkConf, TestUtils} import org.apache.spark.benchmark.Benchmark import org.apache.spark.sql.{DataFrame, DataFrameWriter, Row, SparkSession} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.execution.datasources.parquet.VectorizedParquetRecordReader +import org.apache.spark.sql.execution.datasources.parquet.{ParquetCompressionCodec, VectorizedParquetRecordReader} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.sql.vectorized.ColumnVector @@ -99,15 +100,17 @@ object DataSourceReadBenchmark extends SqlBasedBenchmark { spark.read.json(dir).createOrReplaceTempView("jsonTable") } + val parquetCodec = ParquetCompressionCodec.SNAPPY.name().toLowerCase(Locale.ROOT) + private def saveAsParquetV1Table(df: DataFrameWriter[Row], dir: String): Unit = { - df.mode("overwrite").option("compression", "snappy").parquet(dir) + df.mode("overwrite").option("compression", parquetCodec).parquet(dir) spark.read.parquet(dir).createOrReplaceTempView("parquetV1Table") } private def saveAsParquetV2Table(df: DataFrameWriter[Row], dir: String): Unit = { withSQLConf(ParquetOutputFormat.WRITER_VERSION -> ParquetProperties.WriterVersion.PARQUET_2_0.toString) { - df.mode("overwrite").option("compression", "snappy").parquet(dir) + df.mode("overwrite").option("compression", parquetCodec).parquet(dir) spark.read.parquet(dir).createOrReplaceTempView("parquetV2Table") } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala index 4862571b9c1be..10781ec90fa00 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala @@ -18,12 +18,14 @@ package org.apache.spark.sql.execution.benchmark import java.io.File +import java.util.Locale import scala.util.Random import org.apache.spark.SparkConf import org.apache.spark.benchmark.Benchmark import org.apache.spark.sql.{DataFrame, SparkSession} +import org.apache.spark.sql.execution.datasources.parquet.ParquetCompressionCodec import org.apache.spark.sql.functions.{monotonically_increasing_id, timestamp_seconds} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.ParquetOutputTimestampType @@ -50,7 +52,8 @@ object FilterPushdownBenchmark extends SqlBasedBenchmark { .setIfMissing("spark.driver.memory", "3g") .setIfMissing("spark.executor.memory", "3g") .setIfMissing("orc.compression", "snappy") - .setIfMissing("spark.sql.parquet.compression.codec", "snappy") + .setIfMissing("spark.sql.parquet.compression.codec", + ParquetCompressionCodec.SNAPPY.name().toLowerCase(Locale.ROOT)) SparkSession.builder().config(conf).getOrCreate() } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQueryBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQueryBenchmark.scala index c26272d1dcd63..f01cfea62a958 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQueryBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQueryBenchmark.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.execution.benchmark +import java.util.Locale + import scala.util.Try import org.apache.spark.SparkConf @@ -29,6 +31,7 @@ import org.apache.spark.sql.catalyst.plans.logical.SubqueryAlias import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.catalyst.util.DateTimeConstants.NANOS_PER_SECOND import org.apache.spark.sql.execution.datasources.LogicalRelation +import org.apache.spark.sql.execution.datasources.parquet.ParquetCompressionCodec import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.StructType @@ -51,7 +54,8 @@ object TPCDSQueryBenchmark extends SqlBasedBenchmark with Logging { val conf = new SparkConf() .setMaster(System.getProperty("spark.sql.test.master", "local[1]")) .setAppName("test-sql-context") - .set("spark.sql.parquet.compression.codec", "snappy") + .set("spark.sql.parquet.compression.codec", + ParquetCompressionCodec.SNAPPY.name().toLowerCase(Locale.ROOT)) .set("spark.sql.shuffle.partitions", System.getProperty("spark.sql.shuffle.partitions", "4")) .set("spark.driver.memory", "3g") .set("spark.executor.memory", "3g") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceCodecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceCodecSuite.scala index 11e9f4665a9cf..1f1805a02d765 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceCodecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceCodecSuite.scala @@ -17,7 +17,12 @@ package org.apache.spark.sql.execution.datasources +import java.util.Locale + +import scala.jdk.CollectionConverters._ + import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.execution.datasources.parquet.ParquetCompressionCodec import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} @@ -58,9 +63,10 @@ class ParquetCodecSuite extends FileSourceCodecSuite { // Exclude "lzo" because it is GPL-licenced so not included in Hadoop. // Exclude "brotli" because the com.github.rdblue:brotli-codec dependency is not available // on Maven Central. - override protected def availableCodecs: Seq[String] = { - Seq("none", "uncompressed", "snappy", "gzip", "zstd", "lz4", "lz4_raw") - } + override protected def availableCodecs: Seq[String] = + (ParquetCompressionCodec.NONE +: + ParquetCompressionCodec.availableCodecs.asScala.iterator.to(Seq)) + .map(_.name().toLowerCase(Locale.ROOT)).iterator.to(Seq) } class OrcCodecSuite extends FileSourceCodecSuite { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetCompressionCodecPrecedenceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetCompressionCodecPrecedenceSuite.scala index 1a387b7d2de63..28ea430635a2b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetCompressionCodecPrecedenceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetCompressionCodecPrecedenceSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution.datasources.parquet import java.io.File +import java.util.Locale import scala.jdk.CollectionConverters._ @@ -29,18 +30,9 @@ import org.apache.spark.sql.test.SharedSparkSession class ParquetCompressionCodecPrecedenceSuite extends ParquetTest with SharedSparkSession { test("Test `spark.sql.parquet.compression.codec` config") { - Seq( - "NONE", - "UNCOMPRESSED", - "SNAPPY", - "GZIP", - "LZO", - "LZ4", - "BROTLI", - "ZSTD", - "LZ4_RAW").foreach { c => - withSQLConf(SQLConf.PARQUET_COMPRESSION.key -> c) { - val expected = if (c == "NONE") "UNCOMPRESSED" else c + ParquetCompressionCodec.values().foreach { codec => + withSQLConf(SQLConf.PARQUET_COMPRESSION.key -> codec.name()) { + val expected = codec.getCompressionCodec.name() val option = new ParquetOptions(Map.empty[String, String], spark.sessionState.conf) assert(option.compressionCodecClassName == expected) } @@ -49,25 +41,32 @@ class ParquetCompressionCodecPrecedenceSuite extends ParquetTest with SharedSpar test("[SPARK-21786] Test Acquiring 'compressionCodecClassName' for parquet in right order.") { // When "compression" is configured, it should be the first choice. - withSQLConf(SQLConf.PARQUET_COMPRESSION.key -> "snappy") { - val props = Map("compression" -> "uncompressed", ParquetOutputFormat.COMPRESSION -> "gzip") + withSQLConf(SQLConf.PARQUET_COMPRESSION.key -> + ParquetCompressionCodec.SNAPPY.name.toLowerCase(Locale.ROOT)) { + val props = Map( + "compression" -> ParquetCompressionCodec.UNCOMPRESSED.name.toLowerCase(Locale.ROOT), + ParquetOutputFormat.COMPRESSION -> + ParquetCompressionCodec.GZIP.name.toLowerCase(Locale.ROOT)) val option = new ParquetOptions(props, spark.sessionState.conf) - assert(option.compressionCodecClassName == "UNCOMPRESSED") + assert(option.compressionCodecClassName == ParquetCompressionCodec.UNCOMPRESSED.name) } // When "compression" is not configured, "parquet.compression" should be the preferred choice. - withSQLConf(SQLConf.PARQUET_COMPRESSION.key -> "snappy") { - val props = Map(ParquetOutputFormat.COMPRESSION -> "gzip") + withSQLConf(SQLConf.PARQUET_COMPRESSION.key -> + ParquetCompressionCodec.SNAPPY.name.toLowerCase(Locale.ROOT)) { + val props = Map(ParquetOutputFormat.COMPRESSION -> + ParquetCompressionCodec.GZIP.name.toLowerCase(Locale.ROOT)) val option = new ParquetOptions(props, spark.sessionState.conf) - assert(option.compressionCodecClassName == "GZIP") + assert(option.compressionCodecClassName == ParquetCompressionCodec.GZIP.name) } // When both "compression" and "parquet.compression" are not configured, // spark.sql.parquet.compression.codec should be the right choice. - withSQLConf(SQLConf.PARQUET_COMPRESSION.key -> "snappy") { + withSQLConf(SQLConf.PARQUET_COMPRESSION.key -> + ParquetCompressionCodec.SNAPPY.name.toLowerCase(Locale.ROOT)) { val props = Map.empty[String, String] val option = new ParquetOptions(props, spark.sessionState.conf) - assert(option.compressionCodecClassName == "SNAPPY") + assert(option.compressionCodecClassName == ParquetCompressionCodec.SNAPPY.name) } } @@ -113,8 +112,8 @@ class ParquetCompressionCodecPrecedenceSuite extends ParquetTest with SharedSpar } test("Create parquet table with compression") { + val codecs = ParquetCompressionCodec.availableCodecs.asScala.map(_.name()) Seq(true, false).foreach { isPartitioned => - val codecs = Seq("UNCOMPRESSED", "SNAPPY", "GZIP", "ZSTD", "LZ4", "LZ4_RAW") codecs.foreach { compressionCodec => checkCompressionCodec(compressionCodec, isPartitioned) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala index 95a45e52bfb49..a5d5f8ce30f0c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala @@ -34,8 +34,6 @@ import org.apache.parquet.example.data.Group import org.apache.parquet.example.data.simple.{SimpleGroup, SimpleGroupFactory} import org.apache.parquet.hadoop._ import org.apache.parquet.hadoop.example.ExampleParquetWriter -import org.apache.parquet.hadoop.metadata.CompressionCodecName -import org.apache.parquet.hadoop.metadata.CompressionCodecName.GZIP import org.apache.parquet.io.api.Binary import org.apache.parquet.schema.{MessageType, MessageTypeParser} @@ -845,7 +843,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession val data = (0 until 10).map(i => (i, i.toString)) - def checkCompressionCodec(codec: CompressionCodecName): Unit = { + def checkCompressionCodec(codec: ParquetCompressionCodec): Unit = { withSQLConf(SQLConf.PARQUET_COMPRESSION.key -> codec.name()) { withParquetFile(data) { path => assertResult(spark.conf.get(SQLConf.PARQUET_COMPRESSION).toUpperCase(Locale.ROOT)) { @@ -857,12 +855,9 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession // Checks default compression codec checkCompressionCodec( - CompressionCodecName.fromConf(spark.conf.get(SQLConf.PARQUET_COMPRESSION))) + ParquetCompressionCodec.fromString(spark.conf.get(SQLConf.PARQUET_COMPRESSION))) - checkCompressionCodec(CompressionCodecName.UNCOMPRESSED) - checkCompressionCodec(CompressionCodecName.GZIP) - checkCompressionCodec(CompressionCodecName.SNAPPY) - checkCompressionCodec(CompressionCodecName.ZSTD) + ParquetCompressionCodec.availableCodecs.asScala.foreach(checkCompressionCodec(_)) } private def createParquetWriter( @@ -878,7 +873,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession .withDictionaryEncoding(dictionaryEnabled) .withType(schema) .withWriterVersion(PARQUET_1_0) - .withCompressionCodec(GZIP) + .withCompressionCodec(ParquetCompressionCodec.GZIP.getCompressionCodec) .withRowGroupSize(1024 * 1024) .withPageSize(pageSize) .withDictionaryPageSize(dictionaryPageSize) @@ -1507,9 +1502,12 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession } test("SPARK-18433: Improve DataSource option keys to be more case-insensitive") { - withSQLConf(SQLConf.PARQUET_COMPRESSION.key -> "snappy") { - val option = new ParquetOptions(Map("Compression" -> "uncompressed"), spark.sessionState.conf) - assert(option.compressionCodecClassName == "UNCOMPRESSED") + withSQLConf(SQLConf.PARQUET_COMPRESSION.key -> + ParquetCompressionCodec.SNAPPY.name.toLowerCase(Locale.ROOT)) { + val option = new ParquetOptions( + Map("Compression" -> ParquetCompressionCodec.UNCOMPRESSED.name.toLowerCase(Locale.ROOT)), + spark.sessionState.conf) + assert(option.compressionCodecClassName == ParquetCompressionCodec.UNCOMPRESSED.name) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/expressions/ExpressionInfoSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/expressions/ExpressionInfoSuite.scala index f8dde124b3120..0f1a45319af78 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/expressions/ExpressionInfoSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/expressions/ExpressionInfoSuite.scala @@ -122,7 +122,7 @@ class ExpressionInfoSuite extends SparkFunSuite with SharedSparkSession { withClue(s"Expression class '$className'") { val exprExamples = info.getOriginalExamples if (!exprExamples.isEmpty && !ignoreSet.contains(className)) { - assert(exampleRe.findAllIn(exprExamples).toIterable + assert(exampleRe.findAllIn(exprExamples).iterator.to(Iterable) .filter(setStmtRe.findFirstIn(_).isEmpty) // Ignore SET commands .forall(_.contains("_FUNC_"))) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CompressionCodecSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CompressionCodecSuite.scala index a5d11f6e0e14d..df28e7b4485a6 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CompressionCodecSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CompressionCodecSuite.scala @@ -29,7 +29,7 @@ import org.apache.parquet.hadoop.ParquetOutputFormat import org.scalatest.BeforeAndAfterAll import org.apache.spark.sql.execution.datasources.orc.OrcOptions -import org.apache.spark.sql.execution.datasources.parquet.{ParquetOptions, ParquetTest} +import org.apache.spark.sql.execution.datasources.parquet.{ParquetCompressionCodec, ParquetOptions, ParquetTest} import org.apache.spark.sql.hive.orc.OrcFileOperator import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf @@ -289,8 +289,14 @@ class CompressionCodecSuite extends TestHiveSingleton with ParquetTest with Befo test("both table-level and session-level compression are set") { checkForTableWithCompressProp("parquet", - tableCompressCodecs = List("UNCOMPRESSED", "SNAPPY", "GZIP"), - sessionCompressCodecs = List("SNAPPY", "GZIP", "SNAPPY")) + tableCompressCodecs = List( + ParquetCompressionCodec.UNCOMPRESSED.name, + ParquetCompressionCodec.SNAPPY.name, + ParquetCompressionCodec.GZIP.name), + sessionCompressCodecs = List( + ParquetCompressionCodec.SNAPPY.name, + ParquetCompressionCodec.GZIP.name, + ParquetCompressionCodec.SNAPPY.name)) checkForTableWithCompressProp("orc", tableCompressCodecs = List(CompressionKind.NONE.name, CompressionKind.SNAPPY.name, CompressionKind.ZLIB.name), @@ -301,7 +307,10 @@ class CompressionCodecSuite extends TestHiveSingleton with ParquetTest with Befo test("table-level compression is not set but session-level compressions is set ") { checkForTableWithCompressProp("parquet", tableCompressCodecs = List.empty, - sessionCompressCodecs = List("UNCOMPRESSED", "SNAPPY", "GZIP")) + sessionCompressCodecs = List( + ParquetCompressionCodec.UNCOMPRESSED.name, + ParquetCompressionCodec.SNAPPY.name, + ParquetCompressionCodec.GZIP.name)) checkForTableWithCompressProp("orc", tableCompressCodecs = List.empty, sessionCompressCodecs = @@ -339,7 +348,11 @@ class CompressionCodecSuite extends TestHiveSingleton with ParquetTest with Befo } test("test table containing mixed compression codec") { - checkTableWriteWithCompressionCodecs("parquet", List("UNCOMPRESSED", "SNAPPY", "GZIP")) + checkTableWriteWithCompressionCodecs("parquet", + List( + ParquetCompressionCodec.UNCOMPRESSED.name, + ParquetCompressionCodec.SNAPPY.name, + ParquetCompressionCodec.GZIP.name)) checkTableWriteWithCompressionCodecs( "orc", List(CompressionKind.NONE.name, CompressionKind.SNAPPY.name, CompressionKind.ZLIB.name)) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala index 2a3c77a56e6db..45dd8da6e0200 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala @@ -19,9 +19,10 @@ package org.apache.spark.sql.hive import java.time.{Duration, Period} import java.time.temporal.ChronoUnit +import java.util.Locale import org.apache.spark.sql.{AnalysisException, QueryTest, Row} -import org.apache.spark.sql.execution.datasources.parquet.ParquetTest +import org.apache.spark.sql.execution.datasources.parquet.{ParquetCompressionCodec, ParquetTest} import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf @@ -157,7 +158,8 @@ class HiveParquetSuite extends QueryTest test("SPARK-37098: Alter table properties should invalidate cache") { // specify the compression in case we change it in future - withSQLConf(SQLConf.PARQUET_COMPRESSION.key -> "snappy") { + withSQLConf(SQLConf.PARQUET_COMPRESSION.key -> + ParquetCompressionCodec.SNAPPY.name().toLowerCase(Locale.ROOT)) { withTempPath { dir => withTable("t") { sql(s"CREATE TABLE t (c int) STORED AS PARQUET LOCATION '${dir.getCanonicalPath}'") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index 05d2ca1e210f3..78365d25c8984 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -35,7 +35,7 @@ import org.apache.spark.sql.connector.catalog.CatalogManager import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME import org.apache.spark.sql.connector.catalog.SupportsNamespaces.PROP_OWNER import org.apache.spark.sql.execution.command.{DDLSuite, DDLUtils} -import org.apache.spark.sql.execution.datasources.parquet.ParquetFooterReader +import org.apache.spark.sql.execution.datasources.parquet.{ParquetCompressionCodec, ParquetFooterReader} import org.apache.spark.sql.functions._ import org.apache.spark.sql.hive.{HiveExternalCatalog, HiveUtils} import org.apache.spark.sql.hive.HiveUtils.{CONVERT_METASTORE_ORC, CONVERT_METASTORE_PARQUET} @@ -2709,7 +2709,9 @@ class HiveDDLSuite assert(compression === actualCompression) } - Seq(("orc", "ZLIB"), ("parquet", "GZIP")).foreach { case (fileFormat, compression) => + Seq( + ("orc", "ZLIB"), + ("parquet", ParquetCompressionCodec.GZIP.name)).foreach { case (fileFormat, compression) => test(s"SPARK-22158 convertMetastore should not ignore table property - $fileFormat") { withSQLConf(CONVERT_METASTORE_ORC.key -> "true", CONVERT_METASTORE_PARQUET.key -> "true") { withTable("t") { @@ -2804,14 +2806,14 @@ class HiveDDLSuite assert(DDLUtils.isHiveTable(table)) assert(table.storage.serde.get.contains("parquet")) val properties = table.properties - assert(properties.get("parquet.compression") == Some("GZIP")) + assert(properties.get("parquet.compression") == Some(ParquetCompressionCodec.GZIP.name)) assert(spark.table("t").collect().isEmpty) sql("INSERT INTO t SELECT 1") checkAnswer(spark.table("t"), Row(1)) val maybeFile = path.listFiles().find(_.getName.startsWith("part")) - assertCompression(maybeFile, "parquet", "GZIP") + assertCompression(maybeFile, "parquet", ParquetCompressionCodec.GZIP.name) } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/ParquetHadoopFsRelationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/ParquetHadoopFsRelationSuite.scala index 18e8401ee3d2b..84ee19e62bca2 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/ParquetHadoopFsRelationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/ParquetHadoopFsRelationSuite.scala @@ -26,6 +26,7 @@ import org.apache.parquet.hadoop.ParquetOutputFormat import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.catalog.CatalogUtils import org.apache.spark.sql.execution.datasources.SQLHadoopMapReduceCommitProtocol +import org.apache.spark.sql.execution.datasources.parquet.ParquetCompressionCodec import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -199,7 +200,7 @@ class ParquetHadoopFsRelationSuite extends HadoopFsRelationTest { } test("SPARK-13543: Support for specifying compression codec for Parquet via option()") { - withSQLConf(SQLConf.PARQUET_COMPRESSION.key -> "UNCOMPRESSED") { + withSQLConf(SQLConf.PARQUET_COMPRESSION.key -> ParquetCompressionCodec.UNCOMPRESSED.name) { withTempPath { dir => val path = s"${dir.getCanonicalPath}/table1" val df = (1 to 5).map(i => (i, (i % 2).toString)).toDF("a", "b") diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapWithStateDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapWithStateDStream.scala index 3368382a55297..eae5f1a27b3b7 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapWithStateDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapWithStateDStream.scala @@ -74,7 +74,7 @@ private[streaming] class MapWithStateDStreamImpl[ /** Return a pair DStream where each RDD is the snapshot of the state of all the keys. */ def stateSnapshots(): DStream[(KeyType, StateType)] = { internalStream.flatMap { - _.stateMap.getAll().map { case (k, s, _) => (k, s) }.toTraversable } + _.stateMap.getAll().map { case (k, s, _) => (k, s) }.iterator.to(Iterable) } } def keyClass: Class[_] = implicitly[ClassTag[KeyType]].runtimeClass