-
Notifications
You must be signed in to change notification settings - Fork 4.6k
[Test] fix(bigquery): handle field named "f" in tableRowFromMessage #36397
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
Closed
Closed
Changes from 4 commits
Commits
Show all changes
10 commits
Select commit
Hold shift + click to select a range
7be50bf
fix(bigquery): handle field named "f" in TableRow conversion
liferoad 5eddc3c
Merge branch 'master' into fix-tableRowFromMessage-setF
liferoad 224294c
test(bigquery): add integration test for nested 'f' field handling
liferoad 310ad4e
test(BigQueryNestedFFieldIT): add validation for failed BigQuery inserts
liferoad cb69c74
spotless
liferoad d1bef95
Merge branch 'master' into fix-tableRowFromMessage-setF
liferoad 5535a9f
fixed the test
liferoad 6cf8394
refactor(bigquery): simplify table row conversion logic in Storage API
liferoad a694ea6
feat(bigquery): add enhanced table row conversion for storage api
liferoad a922d3c
fixed tests
liferoad File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
211 changes: 211 additions & 0 deletions
211
...ud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryNestedFFieldIT.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,211 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one | ||
| * or more contributor license agreements. See the NOTICE file | ||
| * distributed with this work for additional information | ||
| * regarding copyright ownership. The ASF licenses this file | ||
| * to you 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 org.apache.beam.sdk.io.gcp.bigquery; | ||
|
|
||
| import static org.junit.Assert.assertEquals; | ||
|
|
||
| import com.google.api.services.bigquery.model.QueryResponse; | ||
| import com.google.api.services.bigquery.model.TableFieldSchema; | ||
| import com.google.api.services.bigquery.model.TableRow; | ||
| import com.google.api.services.bigquery.model.TableSchema; | ||
| import com.google.common.collect.ImmutableList; | ||
| import com.google.common.collect.ImmutableMap; | ||
| import java.io.IOException; | ||
| import java.security.SecureRandom; | ||
| import java.util.TreeMap; | ||
| import org.apache.beam.sdk.Pipeline; | ||
| import org.apache.beam.sdk.PipelineResult; | ||
| import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; | ||
| import org.apache.beam.sdk.io.gcp.testing.BigqueryClient; | ||
| import org.apache.beam.sdk.testing.PAssert; | ||
| import org.apache.beam.sdk.testing.TestPipeline; | ||
| import org.apache.beam.sdk.transforms.Create; | ||
| import org.apache.beam.sdk.transforms.DoFn; | ||
| import org.apache.beam.sdk.transforms.MapElements; | ||
| import org.apache.beam.sdk.transforms.ParDo; | ||
| import org.apache.beam.sdk.transforms.SimpleFunction; | ||
| import org.apache.beam.sdk.values.PCollection; | ||
| import org.junit.AfterClass; | ||
| import org.junit.BeforeClass; | ||
| import org.junit.Test; | ||
| import org.junit.runner.RunWith; | ||
| import org.junit.runners.JUnit4; | ||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
||
| /** | ||
| * Integration test for BigQuery Storage API write with nested structures containing 'f' field. This | ||
| * test verifies the fix for IllegalArgumentException when setting a List field to Double in nested | ||
| * TableRow structures, based on the scenario from BigQuerySetFPipeline.java. | ||
| */ | ||
| @RunWith(JUnit4.class) | ||
| public class BigQueryNestedFFieldIT { | ||
|
|
||
| private static final Logger LOG = LoggerFactory.getLogger(BigQueryNestedFFieldIT.class); | ||
| private static String project; | ||
| private static final String DATASET_ID = | ||
| "nested_f_field_it_" + System.currentTimeMillis() + "_" + new SecureRandom().nextInt(32); | ||
| private static final String TABLE_NAME = "nested_f_field_test"; | ||
|
|
||
| private static TestBigQueryOptions bqOptions; | ||
| private static final BigqueryClient BQ_CLIENT = new BigqueryClient("BigQueryNestedFFieldIT"); | ||
|
|
||
| @BeforeClass | ||
| public static void setup() throws Exception { | ||
| bqOptions = TestPipeline.testingPipelineOptions().as(TestBigQueryOptions.class); | ||
| project = bqOptions.as(GcpOptions.class).getProject(); | ||
| // Create one BQ dataset for all test cases. | ||
| BQ_CLIENT.createNewDataset(project, DATASET_ID, null, bqOptions.getBigQueryLocation()); | ||
| } | ||
|
|
||
| @AfterClass | ||
| public static void cleanup() { | ||
| BQ_CLIENT.deleteDataset(project, DATASET_ID); | ||
| } | ||
|
|
||
| /** | ||
| * Test case that reproduces the scenario from BigQuerySetFPipeline.java where a nested structure | ||
| * contains an 'f' field with a float value. This tests the fix for the IllegalArgumentException | ||
| * that occurred when TableRowToStorageApiProto tried to set a List field to a Double value. | ||
| */ | ||
| @Test | ||
| public void testNestedFFieldWithFloat() throws IOException, InterruptedException { | ||
| // Define the table schema with nested structure containing 'f' field | ||
| TableSchema schema = | ||
| new TableSchema() | ||
| .setFields( | ||
| ImmutableList.of( | ||
| new TableFieldSchema().setName("bytes").setType("BYTES"), | ||
| new TableFieldSchema() | ||
| .setName("sub") | ||
| .setType("RECORD") | ||
| .setFields( | ||
| ImmutableList.of( | ||
| new TableFieldSchema().setName("a").setType("STRING"), | ||
| new TableFieldSchema().setName("c").setType("INTEGER"), | ||
| new TableFieldSchema().setName("f").setType("FLOAT"))))); | ||
|
|
||
| String tableSpec = String.format("%s:%s.%s", project, DATASET_ID, TABLE_NAME); | ||
|
|
||
| // Set up pipeline options for Storage API | ||
| bqOptions.setUseStorageWriteApi(true); | ||
| bqOptions.setUseStorageWriteApiAtLeastOnce(true); | ||
|
|
||
| Pipeline pipeline = Pipeline.create(bqOptions); | ||
|
|
||
| // Create test data similar to BigQuerySetFPipeline.java | ||
| WriteResult result = | ||
| pipeline | ||
| .apply("CreateInput", Create.of("test")) | ||
| .apply("GenerateTestData", ParDo.of(new GenerateTestDataFn())) | ||
| .apply("CreateTableRows", MapElements.via(new CreateTableRowFn())) | ||
| .apply( | ||
| "WriteToBigQuery", | ||
| BigQueryIO.writeTableRows() | ||
| .to(tableSpec) | ||
| .withSchema(schema) | ||
| .withMethod(BigQueryIO.Write.Method.STORAGE_API_AT_LEAST_ONCE) | ||
| .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED) | ||
| .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_APPEND)); | ||
|
|
||
| // Validate failed inserts using PAssert | ||
| PCollection<BigQueryStorageApiInsertError> failedInserts = result.getFailedStorageApiInserts(); | ||
|
|
||
| // Assert that we expect exactly 3 failed inserts (entire batch fails when one row exceeds size | ||
| // limit) | ||
| // The test intentionally creates a batch with one row that exceeds BigQuery's size limit | ||
| PAssert.that(failedInserts) | ||
| .satisfies( | ||
| (Iterable<BigQueryStorageApiInsertError> errors) -> { | ||
| int count = 0; | ||
| for (BigQueryStorageApiInsertError error : errors) { | ||
| count++; | ||
| if (!error.getErrorMessage().contains("Row payload too large")) { | ||
| throw new AssertionError( | ||
| "Expected 'Row payload too large' error, got: " + error.getErrorMessage()); | ||
| } | ||
| } | ||
| if (count != 3) { | ||
|
liferoad marked this conversation as resolved.
Outdated
|
||
| throw new AssertionError("Expected exactly 3 failed inserts, got: " + count); | ||
| } | ||
| return null; | ||
| }); | ||
|
|
||
| // Run the pipeline | ||
| PipelineResult pipelineResult = pipeline.run(); | ||
| pipelineResult.waitUntilFinish(); | ||
|
|
||
| // Check if the BigQuery table exists and has any rows | ||
| String testQuery = | ||
| String.format("SELECT sub.a, sub.c, sub.f FROM [%s.%s];", DATASET_ID, TABLE_NAME); | ||
|
|
||
| try { | ||
| QueryResponse response = BQ_CLIENT.queryWithRetries(testQuery, project); | ||
|
|
||
| if (response.getRows() != null && response.getRows().size() > 0) { | ||
| LOG.info("Found {} successful inserts in BigQuery table", response.getRows().size()); | ||
|
|
||
| // Verify the nested 'f' field value for all rows | ||
| for (int i = 0; i < response.getRows().size(); i++) { | ||
| TableRow resultRow = response.getRows().get(i); | ||
| assertEquals("hello", resultRow.getF().get(0).getV()); // sub.a | ||
| assertEquals("3", resultRow.getF().get(1).getV()); // sub.c | ||
| assertEquals("1.2", resultRow.getF().get(2).getV()); // sub.f | ||
| } | ||
|
|
||
| LOG.info( | ||
| "Successfully wrote and verified nested structure with 'f' field containing float value. " | ||
| + "Verified {} rows", | ||
| response.getRows().size()); | ||
| } else { | ||
| LOG.info("No successful inserts found in BigQuery table - all rows may have failed"); | ||
| } | ||
| } catch (Exception e) { | ||
| LOG.info("BigQuery table query failed (table may not exist)", e); | ||
| LOG.info("This suggests all inserts failed or the pipeline encountered an error"); | ||
| } | ||
| } | ||
|
|
||
| private static class GenerateTestDataFn extends DoFn<String, Integer> { | ||
| @ProcessElement | ||
| public void processElement(ProcessContext c) { | ||
| c.output(1_000); // Small byte array size for test | ||
| c.output(1_000_000); // Medium byte array size for test | ||
| c.output(10_000_000); // Large byte array size for test - this row will not be added | ||
| } | ||
| } | ||
|
|
||
| /** Static SimpleFunction for creating TableRows to avoid serialization issues. */ | ||
| private static class CreateTableRowFn extends SimpleFunction<Integer, TableRow> { | ||
| @Override | ||
| public TableRow apply(Integer bytesSize) { | ||
| // Create nested structure with 'f' field containing float value | ||
| // This reproduces the exact scenario from BigQuerySetFPipeline.java | ||
| ImmutableMap<String, Object> data = | ||
| ImmutableMap.of( | ||
| "bytes", | ||
| new byte[bytesSize], | ||
| "sub", | ||
| new TreeMap<>(ImmutableMap.of("a", "hello", "c", 3, "f", 1.2f))); | ||
|
|
||
| TableRow row = new TableRow(); | ||
| row.putAll(new TreeMap<>(data)); | ||
| return row; | ||
| } | ||
| } | ||
| } | ||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.