Skip to content

Commit a03c4cb

Browse files
LuciferYangdongjoon-hyun
authored andcommitted
[SPARK-50973][SQL][AVRO] Cleanup deprecated api usage related to avro.Schema#toString(boolean)
### What changes were proposed in this pull request? This pr aims to use `SchemaFormatter.format(string, Schema)` instead of `avro.Schema#toString(boolean)`, the changes refer to https://github.com/apache/avro/blob/8c27801dc8d42ccc00997f25c0b8f45f8d4a233e/lang/java/avro/src/main/java/org/apache/avro/Schema.java#L412-L422 ![image](https://github.com/user-attachments/assets/ff4477af-dd49-40e6-a2ea-e6636e1e5ece) ### Why are the changes needed? Cleanup deprecated api usage related to `avro.Schema#toString(boolean)` ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Pass GitHub Actions ### Was this patch authored or co-authored using generative AI tooling? No Closes apache#49640 from LuciferYang/avro-schema-toString. Authored-by: yangjie01 <[email protected]> Signed-off-by: Dongjoon Hyun <[email protected]>
1 parent 7e3de64 commit a03c4cb

File tree

4 files changed

+16
-9
lines changed

4 files changed

+16
-9
lines changed

connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala

Lines changed: 4 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -25,7 +25,7 @@ import java.util.UUID
2525

2626
import scala.jdk.CollectionConverters._
2727

28-
import org.apache.avro.{AvroTypeException, Schema, SchemaBuilder}
28+
import org.apache.avro.{AvroTypeException, Schema, SchemaBuilder, SchemaFormatter}
2929
import org.apache.avro.Schema.{Field, Type}
3030
import org.apache.avro.Schema.Type._
3131
import org.apache.avro.file.{DataFileReader, DataFileWriter}
@@ -86,7 +86,7 @@ abstract class AvroSuite
8686
}
8787

8888
def getAvroSchemaStringFromFiles(filePath: String): String = {
89-
new DataFileReader({
89+
val schema = new DataFileReader({
9090
val file = new File(filePath)
9191
if (file.isFile) {
9292
file
@@ -96,7 +96,8 @@ abstract class AvroSuite
9696
.filter(_.getName.endsWith("avro"))
9797
.head
9898
}
99-
}, new GenericDatumReader[Any]()).getSchema.toString(false)
99+
}, new GenericDatumReader[Any]()).getSchema
100+
SchemaFormatter.format(AvroUtils.JSON_INLINE_FORMAT, schema)
100101
}
101102

102103
// Check whether an Avro schema of union type is converted to SQL in an expected way, when the

sql/core/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala

Lines changed: 6 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -21,7 +21,7 @@ import java.util.Locale
2121

2222
import scala.jdk.CollectionConverters._
2323

24-
import org.apache.avro.Schema
24+
import org.apache.avro.{Schema, SchemaFormatter}
2525
import org.apache.avro.file.{DataFileReader, FileReader}
2626
import org.apache.avro.generic.{GenericDatumReader, GenericRecord}
2727
import org.apache.avro.mapred.{AvroOutputFormat, FsInput}
@@ -44,6 +44,10 @@ import org.apache.spark.sql.types._
4444
import org.apache.spark.util.Utils
4545

4646
private[sql] object AvroUtils extends Logging {
47+
48+
val JSON_INLINE_FORMAT: String = "json/inline"
49+
val JSON_PRETTY_FORMAT: String = "json/pretty"
50+
4751
def inferSchema(
4852
spark: SparkSession,
4953
options: Map[String, String],
@@ -71,7 +75,7 @@ private[sql] object AvroUtils extends Logging {
7175
case _ => throw new RuntimeException(
7276
s"""Avro schema cannot be converted to a Spark SQL StructType:
7377
|
74-
|${avroSchema.toString(true)}
78+
|${SchemaFormatter.format(JSON_PRETTY_FORMAT, avroSchema)}
7579
|""".stripMargin)
7680
}
7781
}

sql/core/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -22,7 +22,7 @@ import java.util.Locale
2222
import scala.collection.mutable
2323
import scala.jdk.CollectionConverters._
2424

25-
import org.apache.avro.{LogicalTypes, Schema, SchemaBuilder}
25+
import org.apache.avro.{LogicalTypes, Schema, SchemaBuilder, SchemaFormatter}
2626
import org.apache.avro.LogicalTypes.{Decimal, _}
2727
import org.apache.avro.Schema.Type._
2828
import org.apache.avro.SchemaBuilder.FieldAssembler
@@ -148,9 +148,10 @@ object SchemaConverters extends Logging {
148148
case RECORD =>
149149
val recursiveDepth: Int = existingRecordNames.getOrElse(avroSchema.getFullName, 0)
150150
if (recursiveDepth > 0 && recursiveFieldMaxDepth <= 0) {
151+
val formattedAvroSchema = SchemaFormatter.format(AvroUtils.JSON_PRETTY_FORMAT, avroSchema)
151152
throw new IncompatibleSchemaException(s"""
152153
|Found recursive reference in Avro schema, which can not be processed by Spark by
153-
| default: ${avroSchema.toString(true)}. Try setting the option `recursiveFieldMaxDepth`
154+
| default: $formattedAvroSchema. Try setting the option `recursiveFieldMaxDepth`
154155
| to 1 - $RECURSIVE_FIELD_MAX_DEPTH_LIMIT.
155156
""".stripMargin)
156157
} else if (recursiveDepth > 0 && recursiveDepth >= recursiveFieldMaxDepth) {

sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetAvroCompatibilitySuite.scala

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -23,13 +23,14 @@ import java.util.{List => JList, Map => JMap}
2323

2424
import scala.jdk.CollectionConverters._
2525

26-
import org.apache.avro.Schema
26+
import org.apache.avro.{Schema, SchemaFormatter}
2727
import org.apache.avro.generic.IndexedRecord
2828
import org.apache.hadoop.fs.Path
2929
import org.apache.parquet.avro.AvroParquetWriter
3030
import org.apache.parquet.hadoop.ParquetWriter
3131

3232
import org.apache.spark.sql.Row
33+
import org.apache.spark.sql.avro.AvroUtils
3334
import org.apache.spark.sql.execution.datasources.parquet.test.avro._
3435
import org.apache.spark.sql.test.SharedSparkSession
3536

@@ -40,7 +41,7 @@ class ParquetAvroCompatibilitySuite extends ParquetCompatibilityTest with Shared
4041
logInfo(
4142
s"""Writing Avro records with the following Avro schema into Parquet file:
4243
|
43-
|${schema.toString(true)}
44+
|${SchemaFormatter.format(AvroUtils.JSON_PRETTY_FORMAT, schema)}
4445
""".stripMargin)
4546

4647
val writer = AvroParquetWriter.builder[T](new Path(path)).withSchema(schema).build()

0 commit comments

Comments
 (0)