ahmedabu98 commented on code in PR #35567:
URL: https://github.com/apache/beam/pull/35567#discussion_r2216589474
##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java:
##########
@@ -1149,22 +1157,54 @@ <T> long insertAll(
// If this row's encoding by itself is larger than the maximum row
payload, then it's
// impossible to insert into BigQuery, and so we send it out through
the dead-letter
// queue.
- if (nextRowSize >= MAX_BQ_ROW_PAYLOAD) {
+ if (nextRowSize >= MAX_BQ_ROW_PAYLOAD_BYTES) {
InsertErrors error =
new InsertErrors()
.setErrors(ImmutableList.of(new
ErrorProto().setReason("row-too-large")));
// We verify whether the retryPolicy parameter expects us to
retry. If it does, then
// it will return true. Otherwise it will return false.
- Boolean isRetry = retryPolicy.shouldRetry(new
InsertRetryPolicy.Context(error));
- if (isRetry) {
- throw new RuntimeException(
+ if (retryPolicy.shouldRetry(new InsertRetryPolicy.Context(error)))
{
+ // Obtain table schema
+ TableSchema tableSchema = null;
+ String tableSpec = BigQueryHelpers.toTableSpec(ref);
+ if (tableSchemaCache.containsKey(tableSpec)) {
+ tableSchema = tableSchemaCache.get(tableSpec);
+ } else {
+ Table table = getTable(ref);
+ if (table != null) {
+ tableSchema =
+
TableRowToStorageApiProto.schemaToProtoTableSchema(table.getSchema());
+ tableSchemaCache.put(tableSpec, tableSchema);
+ }
+ }
+
+ // Validate row schema
+ String rowDetails = "";
+ if (tableSchema != null) {
+ rowDetails = validateRowSchema(row, tableSchema);
+ }
+
+ // Shorten row details if too long for human readability
+ if (rowDetails.length() > 1024) {
+ rowDetails = rowDetails.substring(0, 1024) + "...}";
+ }
Review Comment:
nit: This logic should go inside `validateRowSchema`
##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java:
##########
@@ -1149,22 +1157,54 @@ <T> long insertAll(
// If this row's encoding by itself is larger than the maximum row
payload, then it's
// impossible to insert into BigQuery, and so we send it out through
the dead-letter
// queue.
- if (nextRowSize >= MAX_BQ_ROW_PAYLOAD) {
+ if (nextRowSize >= MAX_BQ_ROW_PAYLOAD_BYTES) {
InsertErrors error =
new InsertErrors()
.setErrors(ImmutableList.of(new
ErrorProto().setReason("row-too-large")));
// We verify whether the retryPolicy parameter expects us to
retry. If it does, then
// it will return true. Otherwise it will return false.
- Boolean isRetry = retryPolicy.shouldRetry(new
InsertRetryPolicy.Context(error));
- if (isRetry) {
- throw new RuntimeException(
+ if (retryPolicy.shouldRetry(new InsertRetryPolicy.Context(error)))
{
+ // Obtain table schema
+ TableSchema tableSchema = null;
+ String tableSpec = BigQueryHelpers.toTableSpec(ref);
+ if (tableSchemaCache.containsKey(tableSpec)) {
+ tableSchema = tableSchemaCache.get(tableSpec);
+ } else {
+ Table table = getTable(ref);
+ if (table != null) {
+ tableSchema =
+
TableRowToStorageApiProto.schemaToProtoTableSchema(table.getSchema());
+ tableSchemaCache.put(tableSpec, tableSchema);
+ }
+ }
+
+ // Validate row schema
+ String rowDetails = "";
+ if (tableSchema != null) {
+ rowDetails = validateRowSchema(row, tableSchema);
+ }
+
+ // Shorten row details if too long for human readability
+ if (rowDetails.length() > 1024) {
+ rowDetails = rowDetails.substring(0, 1024) + "...}";
+ }
+
+ // Basic log to return
+ String bqLimitLog =
String.format(
- "We have observed a row that is %s bytes in size and
exceeded BigQueryIO"
- + " limit of 9MB. While BigQuery supports request
sizes up to 10MB,"
- + " BigQueryIO sets the limit at 9MB to leave room
for request"
- + " overhead. You may change your retry strategy to
unblock this"
- + " pipeline, and the row will be output as a failed
insert.",
- nextRowSize));
+ "We have observed a row of size %s bytes exceeding the "
+ + "BigQueryIO limit of %s.",
+ nextRowSize, MAX_BQ_ROW_PAYLOAD_DESC);
+
+ // Add on row schema diff details if present
+ if (rowDetails.length() > 0) {
+ bqLimitLog +=
+ String.format(
+ " This is probably due to a schema "
+ + "mismatch. Problematic row schema: %s.",
Review Comment:
```suggestion
+ "mismatch. Problematic row had extra schema
fields: %s.",
```
##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java:
##########
@@ -1317,6 +1357,39 @@ <T> long insertAll(
}
}
+ /**
+ * Validates a {@link TableRow} for logging, comparing the provided row
against a BigQuery
+ * schema. The formatted string shows the field names in the row
indicating any mismatches
+ * unknown entries.
+ *
+ * <p>For example, a {@link TableRow} with a "names" and "age" fields,
where the schema expects
+ * "name" and "age" would return "name".
+ *
+ * <pre>{@code {'name': java.lang.String, 'age': java.lang.Integer}}</pre>
+ *
+ * <p>If a field exists in the row but not in the schema, it's marked as
"Unknown fields".
Review Comment:
cleanup?
##########
sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImplTest.java:
##########
@@ -1164,7 +1164,73 @@ public void
testInsertWithinRequestByteSizeLimitsErrorsOut() throws Exception {
false,
successfulRows));
- assertThat(e.getMessage(), containsString("exceeded BigQueryIO limit of
9MB."));
+ assertThat(e.getMessage(), containsString("exceeding the BigQueryIO
limit"));
+ }
+
+ /**
+ * Tests that {@link DatasetServiceImpl#insertAll} does not go over limit of
rows per request and
+ * schema difference check.
+ */
+ @SuppressWarnings("InlineMeInliner") // inline `Strings.repeat()` - Java 11+
API only
+ @Test
+ public void testInsertWithinRequestByteSizeLimitsWithBadSchemaErrorsOut()
throws Exception {
+ TableReference ref =
+ new
TableReference().setProjectId("project").setDatasetId("dataset").setTableId("tablersl");
+
+ TableSchema schema =
+ new TableSchema()
+ .setFields(ImmutableList.of(new
TableFieldSchema().setName("rows").setType("STRING")));
+ Table testTable = new Table().setTableReference(ref).setSchema(schema);
+
+ List<FailsafeValueInSingleWindow<TableRow, TableRow>> rows =
+ ImmutableList.of(
+ wrapValue(new TableRow().set("row", Strings.repeat("abcdefghi",
1024 * 1025))),
+ wrapValue(new TableRow().set("row", "a")),
+ wrapValue(new TableRow().set("row", "b")));
Review Comment:
Took me a while to understand what was going on because the actual schema
has field `rows`, but the TableRow elements all use field name `row`
Can we rename it so that they use the correct field name, and the first
(problematic) one has an extra unknown field? It would help make this more
readable
##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java:
##########
@@ -1149,22 +1157,54 @@ <T> long insertAll(
// If this row's encoding by itself is larger than the maximum row
payload, then it's
// impossible to insert into BigQuery, and so we send it out through
the dead-letter
// queue.
- if (nextRowSize >= MAX_BQ_ROW_PAYLOAD) {
+ if (nextRowSize >= MAX_BQ_ROW_PAYLOAD_BYTES) {
InsertErrors error =
new InsertErrors()
.setErrors(ImmutableList.of(new
ErrorProto().setReason("row-too-large")));
// We verify whether the retryPolicy parameter expects us to
retry. If it does, then
// it will return true. Otherwise it will return false.
- Boolean isRetry = retryPolicy.shouldRetry(new
InsertRetryPolicy.Context(error));
- if (isRetry) {
- throw new RuntimeException(
+ if (retryPolicy.shouldRetry(new InsertRetryPolicy.Context(error)))
{
+ // Obtain table schema
+ TableSchema tableSchema = null;
+ String tableSpec = BigQueryHelpers.toTableSpec(ref);
+ if (tableSchemaCache.containsKey(tableSpec)) {
+ tableSchema = tableSchemaCache.get(tableSpec);
+ } else {
+ Table table = getTable(ref);
+ if (table != null) {
+ tableSchema =
+
TableRowToStorageApiProto.schemaToProtoTableSchema(table.getSchema());
+ tableSchemaCache.put(tableSpec, tableSchema);
+ }
+ }
+
+ // Validate row schema
+ String rowDetails = "";
+ if (tableSchema != null) {
+ rowDetails = validateRowSchema(row, tableSchema);
+ }
+
+ // Shorten row details if too long for human readability
+ if (rowDetails.length() > 1024) {
+ rowDetails = rowDetails.substring(0, 1024) + "...}";
+ }
+
+ // Basic log to return
+ String bqLimitLog =
String.format(
- "We have observed a row that is %s bytes in size and
exceeded BigQueryIO"
- + " limit of 9MB. While BigQuery supports request
sizes up to 10MB,"
- + " BigQueryIO sets the limit at 9MB to leave room
for request"
- + " overhead. You may change your retry strategy to
unblock this"
- + " pipeline, and the row will be output as a failed
insert.",
- nextRowSize));
+ "We have observed a row of size %s bytes exceeding the "
+ + "BigQueryIO limit of %s.",
+ nextRowSize, MAX_BQ_ROW_PAYLOAD_DESC);
+
+ // Add on row schema diff details if present
+ if (rowDetails.length() > 0) {
Review Comment:
nit
```suggestion
if (!rowDetails.isEmpty()) {
```
##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java:
##########
@@ -1317,6 +1357,39 @@ <T> long insertAll(
}
}
+ /**
+ * Validates a {@link TableRow} for logging, comparing the provided row
against a BigQuery
+ * schema. The formatted string shows the field names in the row
indicating any mismatches
+ * unknown entries.
+ *
+ * <p>For example, a {@link TableRow} with a "names" and "age" fields,
where the schema expects
+ * "name" and "age" would return "name".
Review Comment:
did you mean to say `where the schema expects "age"`?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]