Skip to content

Commit

Permalink
Propagate primitive length when setting repetition (#742)
Browse files Browse the repository at this point in the history
* Propagate primitive length when setting repetition

* Set scale to arbitrary BigDecimal

* Consistent builder method ordering
  • Loading branch information
RustedBones committed Sep 12, 2023
1 parent 2d8516e commit 69e2024
Show file tree
Hide file tree
Showing 3 changed files with 30 additions and 25 deletions.
16 changes: 12 additions & 4 deletions parquet/src/main/scala/magnolify/parquet/Schema.scala
Original file line number Diff line number Diff line change
Expand Up @@ -38,8 +38,8 @@ private object Schema {
val p = schema.asPrimitiveType()
Types
.primitive(p.getPrimitiveTypeName, schema.getRepetition)
.length(p.getTypeLength)
.as(schema.getLogicalTypeAnnotation)
.length(p.getTypeLength)
.named(name)
} else {
schema
Expand All @@ -55,9 +55,11 @@ private object Schema {
def setRepetition(schema: Type, repetition: Repetition): Type = {
require(schema.isRepetition(Repetition.REQUIRED))
if (schema.isPrimitive) {
val p = schema.asPrimitiveType()
Types
.primitive(schema.asPrimitiveType().getPrimitiveTypeName, repetition)
.primitive(p.getPrimitiveTypeName, repetition)
.as(schema.getLogicalTypeAnnotation)
.length(p.getTypeLength)
.named(schema.getName)
} else {
schema
Expand All @@ -72,14 +74,20 @@ private object Schema {

def setLogicalType(schema: Type, lta: LogicalTypeAnnotation): Type = {
require(schema.isPrimitive)
val p = schema.asPrimitiveType()
Types
.primitive(schema.asPrimitiveType().getPrimitiveTypeName, schema.getRepetition)
.primitive(p.getPrimitiveTypeName, schema.getRepetition)
.as(lta)
.length(p.getTypeLength)
.named(schema.getName)
}

def primitive(ptn: PrimitiveTypeName, lta: LogicalTypeAnnotation = null, length: Int = 0): Type =
Types.required(ptn).as(lta).length(length).named(ptn.name())
Types
.required(ptn)
.as(lta)
.length(length)
.named(ptn.name())

def message(schema: Type): MessageType = {
val builder = Types.buildMessage()
Expand Down
12 changes: 7 additions & 5 deletions parquet/src/test/scala/magnolify/parquet/AvroParquetSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,6 @@ import magnolify.scalacheck.auto._
import magnolify.scalacheck.TestArbitrary._
import magnolify.test._
import magnolify.test.Simple._

import org.apache.avro.generic.GenericRecord
import org.apache.hadoop.conf.Configuration
import org.apache.parquet.avro.{
Expand Down Expand Up @@ -161,11 +160,14 @@ class AvroParquetSuite extends MagnolifySuite {
{
import magnolify.avro.logical.bigquery._
// Precision = number of digits, so 5 means -99999 to 99999
val nines = math.pow(10, 38).toLong - 1
val precision = 38
val scale = 9
val max = BigInt(10).pow(precision) - 1
implicit val arbBigDecimal: Arbitrary[BigDecimal] =
Arbitrary(Gen.choose(-nines, nines).map(BigDecimal(_)))
implicit val pfBigDecimal: ParquetField[BigDecimal] = ParquetField.decimalBinary(38, 9)
test[DecimalBinary]()
Arbitrary(Gen.choose(-max, max).map(BigDecimal(_, scale)))
implicit val pfBigDecimal: ParquetField[BigDecimal] =
ParquetField.decimalBinary(precision, scale)
test[Decimal]()
}

test[AvroParquetLogical]()
Expand Down
27 changes: 11 additions & 16 deletions parquet/src/test/scala/magnolify/parquet/ParquetTypeSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -123,50 +123,50 @@ class ParquetTypeSuite extends MagnolifySuite {

// Precision = number of digits, so 5 means -99999 to 99999
private def decimal(precision: Int): Arbitrary[BigDecimal] = {
val nines = math.pow(10, precision.toDouble).toLong - 1
Arbitrary(Gen.choose(-nines, nines).map(BigDecimal(_)))
val max = BigInt(10).pow(precision) - 1
Arbitrary(Gen.choose(-max, max).map(BigDecimal.apply))
}

{
implicit val arbBigDecimal: Arbitrary[BigDecimal] = decimal(9)
implicit val pfBigDecimal: ParquetField[BigDecimal] = ParquetField.decimal32(9, 0)
test[Decimal32]
test[Decimal]
}

{
implicit val arbBigDecimal: Arbitrary[BigDecimal] = decimal(18)
implicit val pfBigDecimal: ParquetField[BigDecimal] = ParquetField.decimal64(18, 0)
test[Decimal64]
test[Decimal]
}

{
implicit val arbBigDecimal: Arbitrary[BigDecimal] = decimal(18)
// math.floor(math.log10(math.pow(2, 8*8-1) - 1)) = 18 digits
implicit val pfBigDecimal: ParquetField[BigDecimal] = ParquetField.decimalFixed(8, 18, 0)
test[DecimalFixed]
test[Decimal]
}

{
implicit val arbBigDecimal: Arbitrary[BigDecimal] = decimal(20)
implicit val pfBigDecimal: ParquetField[BigDecimal] = ParquetField.decimalBinary(20, 0)
test[DecimalBinary]
test[Decimal]
}

test[Logical]

{
import magnolify.parquet.logical.millis._
test[TimeMillis]
test[Time]
}

{
import magnolify.parquet.logical.micros._
test[TimeMicros]
test[Time]
}

{
import magnolify.parquet.logical.nanos._
test[TimeNanos]
test[Time]
}

{
Expand All @@ -185,14 +185,9 @@ class ParquetTypeSuite extends MagnolifySuite {

case class Unsafe(c: Char)
case class ParquetTypes(b: Byte, s: Short, ba: Array[Byte])
case class Decimal32(bd: BigDecimal)
case class Decimal64(bd: BigDecimal)
case class DecimalFixed(bd: BigDecimal)
case class DecimalBinary(bd: BigDecimal)
case class Decimal(bd: BigDecimal, bdo: Option[BigDecimal])
case class Logical(u: UUID, d: LocalDate)
case class TimeMillis(i: Instant, dt: LocalDateTime, ot: OffsetTime, t: LocalTime)
case class TimeMicros(i: Instant, dt: LocalDateTime, ot: OffsetTime, t: LocalTime)
case class TimeNanos(i: Instant, dt: LocalDateTime, ot: OffsetTime, t: LocalTime)
case class Time(i: Instant, dt: LocalDateTime, ot: OffsetTime, t: LocalTime)
@doc("Parquet with doc")
case class ParquetDoc(@doc("string") s: String, @doc("integers") i: Integers)

Expand Down

0 comments on commit 69e2024

Please sign in to comment.