Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

(fix #766) Deprecate AvroCompat, replace automatic schema detection on read + Configurable write #996

Merged
merged 19 commits into from
Feb 12, 2025
Merged
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
1 change: 1 addition & 0 deletions build.sbt
Original file line number Diff line number Diff line change
@@ -671,6 +671,7 @@ lazy val tools = project
"com.google.apis" % "google-api-services-bigquery" % bigqueryVersion,
"org.apache.avro" % "avro" % avroVersion % Provided,
"org.apache.parquet" % "parquet-hadoop" % parquetVersion,
"org.apache.hadoop" % "hadoop-common" % hadoopVersion,
Copy link
Contributor Author

@clairemcginty clairemcginty Sep 12, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm on the fence about relying so heavily on the hadoop Configuration class, since it pulls in hadoop-common artifact and links us more tightly with Hadoop. Parquet is trying to move away from Configuration and onto their own ParquetConfiguration class, which we could use instead. However, it might be confusing for Scio users since Scio is heavily dependent on Configuration and we don't have immediate plans to offboard from it

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Actually I might pull this out into a separate PR. will update shortly

"org.typelevel" %% "paiges-core" % paigesVersion
)
)
115 changes: 91 additions & 24 deletions docs/parquet.md
Original file line number Diff line number Diff line change
@@ -55,50 +55,116 @@ val pfDecimalBinary = ParquetField.decimalBinary(20, 0)

For a full specification of Date/Time mappings in Parquet, see @ref:[Type Mappings](mapping.md).

## Avro Compatibility
## Parquet-Avro Compatibility

The official Parquet format specification supports the `REPEATED` modifier to denote array types. By default, magnolify-parquet conforms to this specification:
The official Parquet format specification supports [multiple valid schema representations of LIST types](https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#lists). Historically, magnolify-parquet has supported the simplest representation: simply marking the list element field as `REPEATED`, which per the spec defaults to a _required list field with required elements_. For example:

```scala mdoc
import magnolify.parquet._

case class MyRecord(listField: List[Int])
ParquetType[MyRecord].schema
case class RecordWithList(listField: List[Int])
ParquetType[RecordWithList].schema
```

However, the parquet-avro API encodes array types differently: as a nested array inside a required group.
Unfortunately, this schema isn't interoperable out-of-the-box with Parquet files produced by [parquet-avro](https://github.com/apache/parquet-java/tree/master/parquet-avro), which defaults to Parquet's 2-level list encoding (with a configurable option to use 3-level list encoding).

```scala mdoc
import org.apache.avro.Schema
val avroSchema = new Schema.Parser().parse("{\"type\":\"record\",\"name\":\"MyRecord\",\"fields\":[{\"name\": \"listField\", \"type\": {\"type\": \"array\", \"items\": \"string\"}}]}")

// Avro schema matches `RecordWithList`
val avroSchema = new Schema.Parser().parse(s"""{
"type": "record",
"name": "RecordWithList",
"fields": [
{"name": "listField", "type": {"type": "array", "items": "int"}}
]
}""")

// Used by parquet-avro to convert Avro to Parquet schemas
import org.apache.parquet.avro.AvroSchemaConverter
new AvroSchemaConverter().convert(avroSchema)

// 2-level list encoding -- compare to schema generated for `RecordWithList` above
val convertedParquetSchema = new AvroSchemaConverter().convert(avroSchema)
```

Due to this discrepancy, **by default, a Repeated type (i.e. a `List` or `Seq`) written by parquet-avro isn't readable by magnolify-parquet, and vice versa**.
Parquet-avro doesn't fully support the spec magnolify-parquet adheres to and can't interpret the Magnolify list schema. As a result, by default, if your schema contains a repeated type, **records produced by parquet-avro can't be consumed by magnolify-parquet, and vice versa**, unless you're using **Parquet-Avro Compatibility Mode**.

### Parquet-Avro Compatibility Mode

When Parquet-Avro Compatibility Mode is enabled, magnolify-parquet will interpret repeated fields using the same 2-level list structure that parquet-avro uses.
In addition, Parquet file writes will include an extra metadata key, `parquet.avro.schema`, to the file footer, containing the converted, String-serialized Avro schema.

#### Enabling Compatibility Mode on Magnolify < 0.8

To address this, magnolify-parquet supports an "Avro compatibility mode" that, when enabled, will:
You can enable this mode by importing `magnolify.parquet.ParquetArray.AvroCompat._` at the site where your `ParquetType[T]` is derived.
Note that you'll need to add this import for both writes (to produce 2-level encoded lists) _and_ reads (to consume 2-level encoded lists).

- Use the same Repeated schema format as parquet-avro
- Write an additional metadata key, `parquet.avro.schema`, to the Parquet file footer, containing the equivalent Avro schema.
```scala mdoc:fail
import magnolify.parquet.ParquetArray.AvroCompat._

case class RecordWithList(listField: List[String])

val pt = ParquetType[RecordWithList]
```

### Enabling Avro Compatibility Mode
#### Enabling Compatibility Mode on Magnolify >= 0.8

You can enable this mode by importing `magnolify.parquet.ParquetArray.AvroCompat._`:
The `magnolify.parquet.ParquetArray.AvroCompat._` import is **deprecated** in Magnolify 0.8 and is expected to be removed in future versions.

Instead, in Magnolify 0.8 and above, this mode should be enabled on the _writer_ by setting a Hadoop `Configuration` option, `magnolify.parquet.write-grouped-arrays`.

```scala mdoc:reset
import org.apache.hadoop.conf.Configuration
import magnolify.parquet._
import magnolify.parquet.ParquetArray.AvroCompat._

case class MyRecord(listField: List[Int])
// List schema matches parquet-avro spec
ParquetType[MyRecord].schema
case class RecordWithList(listField: List[String])

val conf = new Configuration()
conf.setBoolean(MagnolifyParquetProperties.WriteAvroCompatibleArrays, true) // sets `magnolify.parquet.write-grouped-arrays`

// This String value of this schema will be written to the Parquet metadata key `parquet.avro.schema`
ParquetType[MyRecord].avroSchema
// Instantiate ParquetType with configuration
val pt = ParquetType[RecordWithList](conf)

// Check that the converted Avro schema uses 2-level encoding
pt.schema
```

If you're a Scio user with `com.spotify:scio-parquet` on your classpath, you can instantiate a Configured `ParqueType` as a one-liner:

```scala mdoc:fail
import com.spotify.scio.parquet._
import magnolify.parquet._

case class RecordWithList(listField: List[String])

val pt = ParquetType[RecordWithList](
ParquetConfiguration.of(MagnolifyParquetProperties.WriteAvroCompatibleArrays -> true)
)
```

You can combine a Configuration with a CaseMapper:

```scala mdcoc:compile-only
import magnolify.shared._

// Can be combined with a CaseMapper
val cm: CaseMapper = ???
ParquetType[RecordWithList](cm, conf)
```

If you don't have Hadoop on your classpath, you can instantiate a `MagnolifyParquetProperties` instance directly:

```scala mdoc:compile-only
import magnolify.parquet._

ParquetType[RecordWithList](new MagnolifyParquetProperties {
override def WriteAvroCompatibleArrays: Boolean = true
}
)
```

On the _reader_ side, 2-level arrays will be detected automatically based on the input file schema, so **no imports or extra Configurations are needed**.

## Field Descriptions

The top level class and all fields (including nested class fields) can be annotated with `@doc` annotation. Note that nested classes annotations are ignored.
@@ -113,8 +179,7 @@ case class NestedClass(@doc("nested field annotation") i: Int)
case class TopLevelType(@doc("field annotation") pd: NestedClass)
```

Note that field descriptions are *not* natively supported by the Parquet format. Instead, the `@doc` annotation ensures
that in Avro compat mode, the generated Avro schema written to the metadata key `parquet.avro.schema` will contain the specified field description:
Note that field descriptions are *not* natively supported by the Parquet format. Instead, the `@doc` annotation ensures that the generated Avro schema written to the metadata key `parquet.avro.schema` will contain the specified field description:

```scala mdoc:reset:invisible
import org.apache.hadoop.conf.Configuration
@@ -128,13 +193,13 @@ val path = new Path(Files.createTempDirectory("parquet-tmp").toFile.getAbsoluteP

```scala mdoc
import magnolify.parquet._
// AvroCompat is required to write `parquet.avro.schema` key to file metadata
import magnolify.parquet.ParquetArray.AvroCompat._
import magnolify.shared._

@doc("Top level annotation")
case class MyRecord(@doc("field annotation") listField: List[Int])

// Note: If using Magnolify < 0.8, import magnolify.parquet.ParquetArray.AvroCompat._
// to ensure `parquet.avro.schema` metadata is written to file footer
val writer = ParquetType[MyRecord]
.writeBuilder(HadoopOutputFile.fromPath(path, new Configuration()))
.build()
@@ -145,4 +210,6 @@ writer.close()
ParquetFileReader.open(HadoopInputFile.fromPath(path, new Configuration())).getFileMetaData
```

**Therefore, [enabling Avro compatibility mode](#enabling-avro-compatibility-mode) via the `AvroCompat` import is required to use the `@doc` annotation with ParquetType.**
Note: On Magnolify < 0.8, you must enable [Avro compatibility mode](#parquet-avro-compatibility-mode) via the `AvroCompat` import if you're using the `@doc` annotation with ParquetType,
which triggers magnolify-parquet to write a translated Avro schema to the file footer metadata key `parquet.avro.schema`. Otherwise, your annotations will be essentially thrown out.
On Magnolify >= 0.8, this key is written by default.
Original file line number Diff line number Diff line change
@@ -0,0 +1,35 @@
/*
* Copyright 2025 Spotify AB
*
* Licensed 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 magnolify.parquet

trait MagnolifyParquetProperties extends Serializable {
def WriteAvroCompatibleArrays: Boolean = false
def writeAvroSchemaToMetadata: Boolean = true

private[parquet] final def schemaUniquenessKey: Int = WriteAvroCompatibleArrays.hashCode()
}

/**
* If set in your core-site.xml or an explicit Configruation object passed to ParquetType, will be
* parsed into MagnolifyParquetProperties
*/
object MagnolifyParquetProperties {
val Default: MagnolifyParquetProperties = new MagnolifyParquetProperties {}

val WriteAvroCompatibleArrays: String = "magnolify.parquet.write-grouped-arrays"
val WriteAvroSchemaToMetadata: String = "magnolify.parquet.write-avro-schema"
}
219 changes: 138 additions & 81 deletions parquet/src/main/scala/magnolify/parquet/ParquetField.scala

Large diffs are not rendered by default.

126 changes: 84 additions & 42 deletions parquet/src/main/scala/magnolify/parquet/ParquetType.scala
Original file line number Diff line number Diff line change
@@ -30,18 +30,27 @@
}
import org.apache.parquet.io.api._
import org.apache.parquet.io.{InputFile, OutputFile}
import org.apache.parquet.schema.MessageType
import org.apache.parquet.schema.{MessageType, Type}
import org.slf4j.LoggerFactory
import org.typelevel.scalaccompat.annotation.nowarn

sealed trait ParquetArray

/**
* Add `import magnolify.parquet.ParquetArray.AvroCompat._` to generate AVRO schema on write
* Add `import magnolify.parquet.ParquetArray.AvroCompat._` to generate generate Avro-compatible
* array schemas. This import is DEPRECATED. Instead, pass the following option to your Parquet
* Configuration:
*
* magnolify.parquet.write-grouped-arrays: true
*/
object ParquetArray {
implicit case object default extends ParquetArray

@deprecated(
message =
"AvroCompat import is deprecated; set Parquet Configuration option `magnolify.parquet.write-grouped-arrays: true` instead",
since = "0.8.0"
)
object AvroCompat {
implicit case object avroCompat extends ParquetArray
}
@@ -52,7 +61,6 @@

def schema: MessageType
def avroSchema: AvroSchema
def avroCompat: Boolean

def setupInput(job: Job): Unit = {
job.setInputFormatClass(classOf[ParquetInputFormat[T]])
@@ -72,33 +80,69 @@
def readBuilder(file: InputFile): ReadBuilder[T] = new ReadBuilder(file, readSupport)
def writeBuilder(file: OutputFile): WriteBuilder[T] = new WriteBuilder(file, writeSupport)

def write(c: RecordConsumer, v: T): Unit
def newConverter(): TypeConverter[T]
private[parquet] def properties: MagnolifyParquetProperties

private[parquet] def write(c: RecordConsumer, v: T): Unit = ()
private[parquet] def newConverter(writerSchema: Type): TypeConverter[T] = null

Check warning on line 86 in parquet/src/main/scala/magnolify/parquet/ParquetType.scala

Codecov / codecov/patch

parquet/src/main/scala/magnolify/parquet/ParquetType.scala#L85-L86

Added lines #L85 - L86 were not covered by tests
}

object ParquetType {
private val logger = LoggerFactory.getLogger(this.getClass)

implicit def apply[T](implicit f: ParquetField[T], pa: ParquetArray): ParquetType[T] =
ParquetType(CaseMapper.identity)
ParquetType(CaseMapper.identity, MagnolifyParquetProperties.Default)

def apply[T](
cm: CaseMapper
)(implicit f: ParquetField[T], pa: ParquetArray): ParquetType[T] =
ParquetType[T](cm, MagnolifyParquetProperties.Default)(f, pa)

def apply[T](
conf: Configuration
)(implicit f: ParquetField[T], pa: ParquetArray): ParquetType[T] =
ParquetType[T](
CaseMapper.identity, {
val WriteAvroCompatibleArraysOpt =
Option(conf.get(MagnolifyParquetProperties.WriteAvroCompatibleArrays)).map(_.toBoolean)
val writeMetadataOpt = Option(
conf.get(MagnolifyParquetProperties.WriteAvroSchemaToMetadata)
).map(_.toBoolean)

new MagnolifyParquetProperties {
override def WriteAvroCompatibleArrays: Boolean =
WriteAvroCompatibleArraysOpt.getOrElse(super.WriteAvroCompatibleArrays)
override def writeAvroSchemaToMetadata: Boolean =
writeMetadataOpt.getOrElse(super.writeAvroSchemaToMetadata)
}
}
)(f, pa)

def apply[T](
properties: MagnolifyParquetProperties
)(implicit f: ParquetField[T], pa: ParquetArray): ParquetType[T] =
ParquetType[T](CaseMapper.identity, properties)(f, pa)

Check warning on line 123 in parquet/src/main/scala/magnolify/parquet/ParquetType.scala

Codecov / codecov/patch

parquet/src/main/scala/magnolify/parquet/ParquetType.scala#L123

Added line #L123 was not covered by tests

def apply[T](
cm: CaseMapper,
props: MagnolifyParquetProperties
)(implicit f: ParquetField[T], pa: ParquetArray): ParquetType[T] = f match {
case r: ParquetField.Record[_] =>
new ParquetType[T] {
@transient override lazy val schema: MessageType = Schema.message(r.schema(cm))
@transient override lazy val avroSchema: AvroSchema = {
@transient override def schema: MessageType =
Schema.message(r.schema(cm, properties))

@transient override def avroSchema: AvroSchema = {
val s = new AvroSchemaConverter().convert(schema)
// add doc to avro schema
val fieldDocs = f.fieldDocs(cm)
SchemaUtil.deepCopy(s, f.typeDoc, fieldDocs.get)
}

override val avroCompat: Boolean =
pa == ParquetArray.AvroCompat.avroCompat || f.hasAvroArray
override def write(c: RecordConsumer, v: T): Unit = r.write(c, v)(cm)
override def newConverter(): TypeConverter[T] = r.newConverter()
override private[parquet] def properties: MagnolifyParquetProperties = props
override def write(c: RecordConsumer, v: T): Unit =
r.write(c, v)(cm, properties)
override private[parquet] def newConverter(writerSchema: Type): TypeConverter[T] =
r.newConverter(writerSchema)
}
case _ =>
throw new IllegalArgumentException(s"ParquetType can only be created from Record. Got $f")
@@ -120,7 +164,6 @@

// From AvroReadSupport
private val AVRO_SCHEMA_METADATA_KEY = "parquet.avro.schema"
private val OLD_AVRO_SCHEMA_METADATA_KEY = "avro.schema"

class ReadSupport[T](private var parquetType: ParquetType[T]) extends hadoop.ReadSupport[T] {
def this() = this(null)
@@ -133,25 +176,17 @@
parquetType = SerializationUtils.fromBase64[ParquetType[T]](readKeyType)
}

val metadata = context.getKeyValueMetadata
val model = metadata.get(ParquetWriter.OBJECT_MODEL_NAME_PROP)
val isAvroFile = (model != null && model.contains("avro")) ||
metadata.containsKey(AVRO_SCHEMA_METADATA_KEY) ||
metadata.containsKey(OLD_AVRO_SCHEMA_METADATA_KEY)
if (isAvroFile && !parquetType.avroCompat) {
logger.warn(
"Parquet file was written from Avro records, " +
"`import magnolify.parquet.ParquetArray.AvroCompat._` to read correctly"
)
}
if (!isAvroFile && parquetType.avroCompat) {
logger.warn(
"Parquet file was not written from Avro records, " +
"remove `import magnolify.parquet.ParquetArray.AvroCompat._` to read correctly"
)
val requestedSchema = {
val s = Schema.message(parquetType.schema)
// If reading Avro, roundtrip schema using parquet-avro converter to ensure array compatibility;
// magnolify-parquet does not automatically wrap repeated fields into a group like parquet-avro does
if (Schema.hasGroupedArray(context.getFileSchema)) {
val converter = new AvroSchemaConverter()
converter.convert(converter.convert(s))
} else {
s
}
}

val requestedSchema = Schema.message(parquetType.schema)
Schema.checkCompatibility(context.getFileSchema, requestedSchema)
new hadoop.ReadSupport.ReadContext(requestedSchema, java.util.Collections.emptyMap())
}
@@ -163,7 +198,7 @@
readContext: hadoop.ReadSupport.ReadContext
): RecordMaterializer[T] =
new RecordMaterializer[T] {
private val root = parquetType.newConverter()
private val root = parquetType.newConverter(fileSchema)
override def getCurrentRecord: T = root.get
override def getRootConverter: GroupConverter = root.asGroupConverter()
}
@@ -183,16 +218,23 @@

val schema = Schema.message(parquetType.schema)
val metadata = new java.util.HashMap[String, String]()
if (parquetType.avroCompat) {
// This overrides `WriteSupport#getName`
metadata.put(ParquetWriter.OBJECT_MODEL_NAME_PROP, "avro")
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I did drop the behavior of writing writer.model.name: avro if AvroCompat is enabled -- I don't think it makes sense, it should still be Magnolify. I can't think of any reason why this would impact downstream readers -- model name shouldn't matter at all when comparing schema compatibility across files. lf anyone can think of a good reason why this change is breaking let me know...

metadata.put(AVRO_SCHEMA_METADATA_KEY, parquetType.avroSchema.toString())
} else {
logger.warn(
"Parquet file is being written with no avro compatibility, this mode is not " +
"producing schema. Add `import magnolify.parquet.ParquetArray.AvroCompat._` to " +
"generate schema"
)

if (parquetType.properties.writeAvroSchemaToMetadata) {
try {
metadata.put(
AVRO_SCHEMA_METADATA_KEY,
parquetType.avroSchema.toString()
)
} catch {
// parquet-avro has greater schema restrictions than magnolify-parquet, e.g., parquet-avro does not
// support Maps with non-Binary key types
case e: IllegalArgumentException =>
logger.warn(
s"Writer schema `$schema` contains a type not supported by Avro schemas; will not write " +
s"key $AVRO_SCHEMA_METADATA_KEY to file metadata",
e
)
}
}

new hadoop.WriteSupport.WriteContext(schema, metadata)
9 changes: 7 additions & 2 deletions parquet/src/main/scala/magnolify/parquet/Predicate.scala
Original file line number Diff line number Diff line change
@@ -52,7 +52,10 @@ object Predicate {
)(filterFn: ScalaFieldT => Boolean)(implicit
pf: ParquetField.Primitive[ScalaFieldT]
): FilterPredicate = {
val fieldType = pf.schema(CaseMapper.identity).asPrimitiveType().getPrimitiveTypeName
val fieldType =
pf.schema(CaseMapper.identity, MagnolifyParquetProperties.Default)
.asPrimitiveType()
.getPrimitiveTypeName

val column = fieldType match {
case PrimitiveTypeName.INT32 => FilterApi.intColumn(fieldName)
@@ -65,7 +68,9 @@ object Predicate {
}

def wrap[T](addFn: (PrimitiveConverter, T) => Unit): T => ScalaFieldT = {
lazy val converter = pf.newConverter()
lazy val converter = pf.newConverter(
pf.schema(CaseMapper.identity, MagnolifyParquetProperties.Default)
)
value => {
addFn(converter.asPrimitiveConverter(), value)
converter.get
19 changes: 18 additions & 1 deletion parquet/src/main/scala/magnolify/parquet/Schema.scala
Original file line number Diff line number Diff line change
@@ -95,6 +95,21 @@ private object Schema {
builder.named(schema.getName)
}

// Check if writer schema encodes arrays as a single repeated field inside of an optional or required group
private[parquet] def hasGroupedArray(writer: Type): Boolean =
!writer.isPrimitive && writer.asGroupType().getFields.asScala.exists {
case f if isGroupedArrayType(f) => true
case f if !f.isPrimitive => f.asGroupType().getFields.asScala.exists(hasGroupedArray)
case _ => false
}

private def isGroupedArrayType(f: Type): Boolean =
!f.isPrimitive &&
f.getLogicalTypeAnnotation == LogicalTypeAnnotation.listType() && {
val fields = f.asGroupType().getFields.asScala
fields.size == 1 && fields.head.isRepetition(Repetition.REPEATED)
}

def checkCompatibility(writer: Type, reader: Type): Unit = {
def listFields(gt: GroupType) =
s"[${gt.getFields.asScala.map(f => s"${f.getName}: ${f.getRepetition}").mkString(",")}]"
@@ -109,7 +124,9 @@ private object Schema {
!isRepetitionBackwardCompatible(writer, reader) ||
writer.isPrimitive != reader.isPrimitive
) {
throw new InvalidRecordException(s"$writer found: expected $reader")
throw new InvalidRecordException(
s"Writer schema `$writer` incompatible with reader schema `$reader``"
)
}

writer match {
Original file line number Diff line number Diff line change
@@ -24,7 +24,6 @@ import magnolify.avro.AvroType
import magnolify.shared.{doc, CaseMapper}
import magnolify.avro.unsafe._
import magnolify.parquet.unsafe._
import magnolify.parquet.ParquetArray.AvroCompat._
import magnolify.parquet.util.AvroSchemaComparer
import magnolify.scalacheck.auto._
import magnolify.scalacheck.TestArbitrary._
@@ -41,10 +40,14 @@ import org.apache.parquet.avro.{
}
import org.scalacheck._

import scala.annotation.nowarn
import scala.reflect.ClassTag

@nowarn("cat=deprecation") // Suppress warnings from importing AvroCompat
class AvroParquetSuite extends MagnolifySuite {

import magnolify.parquet.ParquetArray.AvroCompat._

private def test[T: Arbitrary: ClassTag]()(implicit
at: AvroType[T],
tpe: ParquetType[T],
@@ -77,7 +80,8 @@ class AvroParquetSuite extends MagnolifySuite {
val r = at(t)

val out = new TestOutputFile
val writer = AvroParquetWriter.builder[GenericRecord](out).withSchema(at.schema).build()
val writer =
AvroParquetWriter.builder[GenericRecord](out).withSchema(at.schema).build()
writer.write(r)
writer.close()

60 changes: 59 additions & 1 deletion parquet/src/test/scala/magnolify/parquet/ParquetTypeSuite.scala
Original file line number Diff line number Diff line change
@@ -27,6 +27,10 @@ import magnolify.shared.doc
import magnolify.shared.TestEnumType._
import magnolify.test.Simple._
import magnolify.test._
import org.apache.hadoop.conf.Configuration
import org.apache.parquet.avro.AvroSchemaConverter
import org.apache.parquet.hadoop.ParquetWriter
import org.apache.parquet.hadoop.{api => hadoop}
import org.apache.parquet.io._
import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName
import org.scalacheck._
@@ -59,7 +63,6 @@ class ParquetTypeSuite extends MagnolifySuite {
val writer = tpe.writeBuilder(out).build()
writer.write(t)
writer.close()

val in = new TestInputFile(out.getBytes)
val reader = tpe.readBuilder(in).build()
val copy = reader.read()
@@ -218,6 +221,59 @@ class ParquetTypeSuite extends MagnolifySuite {
assertEquals(inner.getFields.asScala.map(_.getName).toSeq, Seq("INNERFIRST"))
}
}

test("AvroCompat") {
def conf(WriteAvroCompatibleArrays: Boolean): Configuration = {
val c = new Configuration()
c.setBoolean(MagnolifyParquetProperties.WriteAvroCompatibleArrays, WriteAvroCompatibleArrays)
c
}

val ptNonGroupedArrays = ParquetType[WithList](conf(WriteAvroCompatibleArrays = false))
// Assert that by default, Magnolify doesn't wrap repeated fields in group types
val nonAvroCompliantSchema = """|message magnolify.parquet.WithList {
| required binary s (STRING);
| repeated binary l (STRING);
|}
|""".stripMargin

assert(!Schema.hasGroupedArray(ptNonGroupedArrays.schema))
assertEquals(nonAvroCompliantSchema, ptNonGroupedArrays.schema.toString)

// Assert that ReadSupport converts non-grouped arrays to grouped arrays depending on writer schema
val asc = new AvroSchemaConverter()
val readSupport = ptNonGroupedArrays.readSupport.init(
new hadoop.InitContext(
new Configuration(),
Map(ParquetWriter.OBJECT_MODEL_NAME_PROP -> Set("avro").asJava).asJava,
asc.convert(
asc.convert(ptNonGroupedArrays.schema)
) // Use converted Avro-compliant schema, which groups lists
)
)

val avroCompliantSchema = """|message magnolify.parquet.WithList {
| required binary s (STRING);
| required group l (LIST) {
| repeated binary array (STRING);
| }
|}
|""".stripMargin

assert(Schema.hasGroupedArray(readSupport.getRequestedSchema))
assertEquals(avroCompliantSchema, readSupport.getRequestedSchema.toString)

// Assert that WriteSupport uses non-grouped schema otherwise
val wc1 = ptNonGroupedArrays.writeSupport.init(new Configuration())
assertEquals(nonAvroCompliantSchema, wc1.getSchema.toString)
assertEquals(ptNonGroupedArrays.schema, wc1.getSchema)

// Assert that WriteSupport uses grouped schema when explicitly configured
val ptGroupedArrays = ParquetType[WithList](conf(WriteAvroCompatibleArrays = true))
val wc2 = ptGroupedArrays.writeSupport.init(new Configuration())
assertEquals(avroCompliantSchema, wc2.getSchema.toString)
assertEquals(ptGroupedArrays.schema, wc2.getSchema)
}
}

case class Unsafe(c: Char)
@@ -251,6 +307,8 @@ case class ParquetNestedListDoc(
i: List[Integers]
)

case class WithList(s: String, l: List[String])

class TestInputFile(ba: Array[Byte]) extends InputFile {
private val bais = new ByteArrayInputStream(ba)
override def getLength: Long = ba.length.toLong
Original file line number Diff line number Diff line change
@@ -294,6 +294,7 @@ object SchemaEvolutionSuite {
}

@nowarn("msg=Unused import")
@nowarn("cat=deprecation") // Suppress warnings from importing AvroCompat
class SchemaEvolutionSuite extends MagnolifySuite {
import SchemaEvolutionSuite._

2 changes: 2 additions & 0 deletions tools/src/test/scala/magnolify/tools/ParquetParserSuite.scala
Original file line number Diff line number Diff line change
@@ -20,8 +20,10 @@ import magnolify.parquet._
import magnolify.test._

import java.time.{Instant, LocalDate, LocalDateTime, LocalTime, OffsetTime}
import scala.annotation.nowarn
import scala.reflect.ClassTag

@nowarn("cat=deprecation") // Suppress warnings from importing AvroCompat
class ParquetParserSuite extends MagnolifySuite {
import ParquetParserSuite._