Skip to content

Commit

Permalink
Add test
Browse files Browse the repository at this point in the history
  • Loading branch information
clairemcginty committed Jul 9, 2024
1 parent c24b4bf commit deeb62d
Showing 1 changed file with 108 additions and 0 deletions.
108 changes: 108 additions & 0 deletions parquet/src/test/scala/magnolify/parquet/ParquetTypeSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -27,8 +27,13 @@ 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.hadoop.ParquetWriter
import org.apache.parquet.io._
import org.apache.parquet.hadoop.api.InitContext
import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName
import org.apache.parquet.schema.Type.Repetition
import org.apache.parquet.schema.Types
import org.scalacheck._

import java.io.ByteArrayInputStream
Expand Down Expand Up @@ -192,6 +197,107 @@ class ParquetTypeSuite extends MagnolifySuite {
assertEquals(inner.getFields.asScala.map(_.getName).toSeq, Seq("INNERFIRST"))
}
}

test(s"AvroCompat") {
val pt = ParquetType[WithList]
// Assert that by default, Magnolify doesn't wrap repeated fields in group types
assert(!Schema.hasGroupedArray(pt.schema))
assertEquals(
"""|message magnolify.parquet.WithList {
| required binary s (STRING);
| repeated binary l (STRING);
|}
|""".stripMargin,
pt.schema.toString
)

// Construct Magnolify read support for a Parquet file written using parquet-avro
val readSupport = pt.readSupport.init(
new InitContext(
new Configuration(),
Map(ParquetWriter.OBJECT_MODEL_NAME_PROP -> Set("avro").asJava).asJava,
// Build expected writer schema, which uses Avro array grouping
Types
.buildMessage()
.addField(Types.primitive(PrimitiveTypeName.BINARY, Repetition.REQUIRED).named("s"))
.addField(
Types
.buildGroup(Repetition.REQUIRED)
.addField(
Types
.primitive(PrimitiveTypeName.BINARY, Repetition.REPEATED)
.named("array")
)
.named("list")
)
.named("WithList")
)
)

// Assert that Magnolify is using a compatible array schema to read
assert(Schema.hasGroupedArray(readSupport.getRequestedSchema))
assertEquals(
"""|message magnolify.parquet.WithList {
| required binary s (STRING);
| required group l (LIST) {
| repeated binary array (STRING);
| }
|}
|""".stripMargin,
readSupport.getRequestedSchema.toString
)
}

test(s"AvroCompat") {
val pt = ParquetType[WithList]

// Assert that by default, Magnolify doesn't wrap repeated fields in group types
assert(!Schema.hasGroupedArray(pt.schema))
assertEquals(
"""|message magnolify.parquet.WithList {
| required binary s (STRING);
| repeated binary l (STRING);
|}
|""".stripMargin,
pt.schema.toString
)

// Construct Magnolify read support for a Parquet file written using parquet-avro
val readSupport = pt.readSupport.init(
new InitContext(
new Configuration(),
Map(ParquetWriter.OBJECT_MODEL_NAME_PROP -> Set("avro").asJava).asJava,
// Build expected writer schema, which uses parquet-avro-style array grouping
Types
.buildMessage()
.addField(Types.primitive(PrimitiveTypeName.BINARY, Repetition.REQUIRED).named("s"))
.addField(
Types
.buildGroup(Repetition.REQUIRED)
.addField(
Types
.primitive(PrimitiveTypeName.BINARY, Repetition.REPEATED)
.named("array")
)
.named("list")
)
.named("WithList")
)
)

// Assert that Magnolify is using a compatible array schema to read
assert(Schema.hasGroupedArray(readSupport.getRequestedSchema))
assertEquals(
"""|message magnolify.parquet.WithList {
| required binary s (STRING);
| required group l (LIST) {
| repeated binary array (STRING);
| }
|}
|""".stripMargin,
readSupport.getRequestedSchema.toString
)
}
}

case class Unsafe(c: Char)
Expand Down Expand Up @@ -225,6 +331,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
Expand Down

0 comments on commit deeb62d

Please sign in to comment.