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

[Bug]: Inconsistent written values for LocalTime types using BQ FileLoads vs StorageWrite API #34038

Open
2 of 17 tasks
clairemcginty opened this issue Feb 20, 2025 · 3 comments
Open
2 of 17 tasks

Comments

@clairemcginty
Copy link
Contributor

What happened?

On beam 2.63.0, when writing GenericRecords to BQ using time-millis logical types over the Java SDK, a different value is written depending on whether I use STORAGE_WRITE_API method or FILE_LOADS method.

Repro setup (sorry for Scio code, will work on reproducing in a Beam as well):

object BqWriteTest {

  val AvroSchema = new Schema.Parser().parse(
    s"""
       |{
       |  "type": "record",
       |  "namespace": "com.spotify.scio.extra.bigquery",
       |  "name": "AvroExampleWithLogicalType",
       |  "fields": [
       |    {"name": "timeMillisField", "type": { "type": "int", "logicalType": "time-millis"}},
       |    {"name": "timeMicrosField", "type": { "type": "long", "logicalType": "time-micros"}}
       |  ]
       |}
       |""".stripMargin
  )

  val TableSchema = new TableSchema().setFields(
    List(
      new TableFieldSchema().setName("timeMillisField").setType("TIME").setMode("REQUIRED"),
      new TableFieldSchema().setName("timeMicrosField").setType("TIME").setMode("REQUIRED")
    ).asJava
  )

  val TestRecord: GenericRecord = new GenericRecordBuilder(AvroSchema)
    .set(
      "timeMillisField",
      new TimeMillisConversion().toInt(java.time.LocalTime.of(6, 30, 10, 0), AvroSchema, LogicalTypes.timeMillis()))
    .set(
      "timeMicrosField",
      new TimeMicrosConversion().toLong(java.time.LocalTime.of(6, 30, 10, 0), AvroSchema, LogicalTypes.timeMicros()))
    .build()

  def main(cmdLineArgs: Array[String]): Unit = {
    val (sc, args) = ContextAndArgs(cmdLineArgs)

    implicit val coder: Coder[GenericRecord] = avroGenericRecordCoder(AvroSchema)

    val records = sc.parallelize(Seq(TestRecord))

    // Save to BQ using Storage Write
    records.saveAsCustomOutput(
        "Write to BQ using Storage Write API",
        org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO
          .write[GenericRecord]()
          .to(s"$project:$dataset.test_bq_file_loads_api")
          .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED)
          .useAvroLogicalTypes()
          .withSchema(TableSchema)
          .withAvroFormatFunction(Functions.serializableFn(_.getElement))
          .withAvroSchemaFactory(Functions.serializableFn(BigQueryUtils.toGenericAvroSchema(_, true)))
          .withMethod(org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.Method.FILE_LOADS)
      )

    // Save to BQ using File Loads
    records.saveAsCustomOutput(
        "Write to BQ using Storage Write API",
        org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO
          .write[GenericRecord]()
          .to(s"$project:$dataset.test_bq_storage_write_api")
          .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED)
          .useAvroLogicalTypes()
          .withSchema(TableSchema)
          .withAvroFormatFunction(Functions.serializableFn(_.getElement))
          .withAvroSchemaFactory(Functions.serializableFn(BigQueryUtils.toGenericAvroSchema(_, true)))
          .withMethod(org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.Method.STORAGE_WRITE_API)
      )

    sc.run()
  }
} 

The time-millis value for the table written using FILE_LOADS does not match the value written using STORAGE_WRITE_API:

storage_write_api data:
Image

file_loads data:
Image

Tbh, the only value that looks correct is the file-loads micros value 😬 I'm extremely concerned about this as potentially incorrect data is being written using BQ's Avro interface.

Issue Priority

Priority: 2 (default / most bugs should be filed as P2)

Issue Components

  • Component: Python SDK
  • Component: Java SDK
  • Component: Go SDK
  • Component: Typescript SDK
  • Component: IO connector
  • Component: Beam YAML
  • Component: Beam examples
  • Component: Beam playground
  • Component: Beam katas
  • Component: Website
  • Component: Infrastructure
  • Component: Spark Runner
  • Component: Flink Runner
  • Component: Samza Runner
  • Component: Twister2 Runner
  • Component: Hazelcast Jet Runner
  • Component: Google Cloud Dataflow Runner
@clairemcginty
Copy link
Contributor Author

clairemcginty commented Feb 20, 2025

cc @Abacn, I'm worried this might be related to recent changes to https://github.com/apache/beam/blob/master/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/AvroGenericRecordToStorageApiProto.java. Looking at the test these expected values don't look right -- shouldn't the milli values be rescaled 1/1000?

@Abacn
Copy link
Contributor

Abacn commented Feb 20, 2025

@clairemcginty
Copy link
Contributor Author

could this be related? https://github.com/apache/beam/pull/33422/files#r1891535677

hmm maybe, I notice that that branch doesn't do any if/else checks on the value of the micros flag like elsewhere in the method, so maybe we're just missing that multiplier

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

No branches or pull requests

2 participants