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
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
1 change: 1 addition & 0 deletions build.sbt
Original file line number Diff line number Diff line change
Expand Up @@ -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
)
)
Expand Down
115 changes: 91 additions & 24 deletions docs/parquet.md
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand All @@ -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
Expand All @@ -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()
Expand All @@ -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"
}
Loading
Loading