Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -3094,6 +3094,26 @@ abstract class AvroSuite
assert(AvroOptions.isValidOption("recursiveFieldMaxDepth"))
}

test("SPARK-53973: boolean Avro options reject non-boolean values") {
Seq(
AvroOptions.STABLE_ID_FOR_UNION_TYPE,
AvroOptions.POSITIONAL_FIELD_MATCHING,
AvroOptions.IGNORE_EXTENSION
).foreach { opt =>
val e = intercept[AnalysisException] {
AvroOptions(Map(opt -> "not_bool"))
}
checkError(
exception = e,
condition = "STDS_INVALID_OPTION_VALUE.WITH_MESSAGE",
parameters = Map(
"optionName" -> opt,
"message" -> "Cannot cast value 'not_bool' to Boolean."
)
)
}
}

test("SPARK-46633: read file with empty blocks") {
for (maxPartitionBytes <- Seq(100, 100000, 100000000)) {
withSQLConf(SQLConf.FILES_MAX_PARTITION_BYTES.key -> s"$maxPartitionBytes") {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,21 @@ private[sql] class AvroOptions(

import AvroOptions._

private def parseBoolean(optionName: String, value: String): Boolean = {
try {
value.toBoolean
} catch {
case _: IllegalArgumentException =>
throw QueryCompilationErrors.avroOptionsException(
optionName,
s"Cannot cast value '$value' to Boolean.")
}
}

private def getBoolean(optionName: String, defaultValue: => Boolean): Boolean = {
parameters.get(optionName).map(v => parseBoolean(optionName, v)).getOrElse(defaultValue)
}

def this(parameters: Map[String, String], conf: Configuration) = {
this(CaseInsensitiveMap(parameters), conf)
}
Expand Down Expand Up @@ -78,8 +93,7 @@ private[sql] class AvroOptions(
* name. This allows for a structurally equivalent Catalyst schema to be used with an Avro schema
* whose field names do not match. Defaults to false.
*/
val positionalFieldMatching: Boolean =
parameters.get(POSITIONAL_FIELD_MATCHING).exists(_.toBoolean)
val positionalFieldMatching: Boolean = getBoolean(POSITIONAL_FIELD_MATCHING, defaultValue = false)

/**
* Top level record name in write result, which is required in Avro spec.
Expand Down Expand Up @@ -107,10 +121,7 @@ private[sql] class AvroOptions(
AvroFileFormat.IgnoreFilesWithoutExtensionProperty,
ignoreFilesWithoutExtensionByDefault)

parameters
.get(IGNORE_EXTENSION)
.map(_.toBoolean)
.getOrElse(!ignoreFilesWithoutExtension)
getBoolean(IGNORE_EXTENSION, defaultValue = !ignoreFilesWithoutExtension)
}

/**
Expand All @@ -134,7 +145,7 @@ private[sql] class AvroOptions(
.getOrElse(SQLConf.get.getConf(SQLConf.AVRO_REBASE_MODE_IN_READ).toString)

val useStableIdForUnionType: Boolean =
parameters.get(STABLE_ID_FOR_UNION_TYPE).map(_.toBoolean).getOrElse(false)
getBoolean(STABLE_ID_FOR_UNION_TYPE, defaultValue = false)

val stableIdPrefixForUnionType: String = parameters
.getOrElse(STABLE_ID_PREFIX_FOR_UNION_TYPE, "member_")
Expand Down