ahmedabu98 commented on code in PR #35567: URL: https://github.com/apache/beam/pull/35567#discussion_r2208744842
########## sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java: ########## @@ -1317,6 +1364,79 @@ <T> long insertAll( } } + /** + * Formats a {@link TableRow} for logging, comparing the provided row against a BigQuery schema. + * The formatted string shows the type of each field in the row, and indicates any mismatches + * between the row's types and the schema's expected types. + * + * <p>For example, a {@link TableRow} with a "name" (String) and "age" (Integer) field, where + * the schema expects a String and Integer respectively, would be formatted as: + * + * <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 field". If a + * field's type doesn't match the schema, it's shown as "Provided: X, Expected: Y". + * + * @param row The {@link TableRow} to format. + * @param bqSchemaMap A map of field names to their expected BigQuery types (e.g., "STRING", + * "INTEGER"). + * @return A string representation of the row, indicating the type of each field and any schema + * mismatches. + */ + private String formatRowWithSchema(TableRow row, Map<String, String> bqSchemaMap) { Review Comment: This method essentially compares the row's object (Java) name with the BQ schema field (SQL) type name [1]. I don't think this is a very robust approach, the Java class name may not always overlap with the SQL type name (e.g. a BQ "TIMESTAMP" corresponds to a joda DateTime object). [1] https://cloud.google.com/bigquery/docs/reference/standard-sql/data-types#data_type_list ########## sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java: ########## @@ -1149,22 +1157,61 @@ <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) { + if (retryPolicy.shouldRetry(new InsertRetryPolicy.Context(error))) { + // Obtain table schema + TableSchema tableSchema = null; + try { + String tableSpec = BigQueryHelpers.toTableSpec(ref); + if (tableSchemaCache.containsKey(tableSpec)) { + tableSchema = tableSchemaCache.get(tableSpec); + } else { + Table table = getTable(ref); + if (table != null) { + table.getSchema(); + tableSchema = + TableRowToStorageApiProto.schemaToProtoTableSchema(table.getSchema()); + tableSchemaCache.put(tableSpec, tableSchema); + } + } + } catch (Exception e) { + LOG.warn("Could not fetch table schema for {}.", ref, e); + } + + // Create BigQuery schema map to use for formatting + String rowDetails; + try { + if (tableSchema != null) { + // Creates bqSchemaMap containing field name, field type, and + // possibly field mode if available. + Map<String, String> bqSchemaMap = + tableSchema.getFieldsList().stream() + .collect(Collectors.toMap(f -> f.getName(), f -> f.getType().name())); + rowDetails = formatRowWithSchema(row, bqSchemaMap); + } else { + rowDetails = formatRowWithoutSchema(row); + } + } catch (Exception e) { + rowDetails = row.keySet().toString(); + } + if (rowDetails.length() > 1024) { + rowDetails = rowDetails.substring(0, 1024) + "...}"; + } + throw new RuntimeException( 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. This is probably due to a schema " + + "mismatch. Problematic row schema " Review Comment: > This is probably due to a schema mismatch Is this actually true? I would think the usual case would be that the user actually wrote a very large row. I don't see how mismatch schemas is a factor here, but maybe I'm missing something ########## sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java: ########## @@ -1155,16 +1158,36 @@ <T> long insertAll( .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) { + if (retryPolicy.shouldRetry(new InsertRetryPolicy.Context(error))) { + // Create row details composed of key value pairs. + String rowDetails; Review Comment: Alternatively, we can log those details as debug level, and reference it in the error message. ########## sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java: ########## @@ -1149,22 +1157,61 @@ <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) { + if (retryPolicy.shouldRetry(new InsertRetryPolicy.Context(error))) { + // Obtain table schema + TableSchema tableSchema = null; + try { + String tableSpec = BigQueryHelpers.toTableSpec(ref); + if (tableSchemaCache.containsKey(tableSpec)) { + tableSchema = tableSchemaCache.get(tableSpec); + } else { + Table table = getTable(ref); + if (table != null) { + table.getSchema(); + tableSchema = + TableRowToStorageApiProto.schemaToProtoTableSchema(table.getSchema()); + tableSchemaCache.put(tableSpec, tableSchema); + } + } + } catch (Exception e) { + LOG.warn("Could not fetch table schema for {}.", ref, e); + } + + // Create BigQuery schema map to use for formatting + String rowDetails; + try { + if (tableSchema != null) { + // Creates bqSchemaMap containing field name, field type, and + // possibly field mode if available. + Map<String, String> bqSchemaMap = + tableSchema.getFieldsList().stream() + .collect(Collectors.toMap(f -> f.getName(), f -> f.getType().name())); + rowDetails = formatRowWithSchema(row, bqSchemaMap); Review Comment: Left some comments. I think if possible, we can leave out the comparisons between expected and actual types. I'm not sure if it's an essential part of this fix or not. Unknown/extra fields in the row could be valuable information though. ########## sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java: ########## @@ -1149,22 +1157,61 @@ <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) { + if (retryPolicy.shouldRetry(new InsertRetryPolicy.Context(error))) { + // Obtain table schema + TableSchema tableSchema = null; + try { + String tableSpec = BigQueryHelpers.toTableSpec(ref); + if (tableSchemaCache.containsKey(tableSpec)) { + tableSchema = tableSchemaCache.get(tableSpec); + } else { + Table table = getTable(ref); + if (table != null) { + table.getSchema(); + tableSchema = + TableRowToStorageApiProto.schemaToProtoTableSchema(table.getSchema()); + tableSchemaCache.put(tableSpec, tableSchema); + } + } + } catch (Exception e) { + LOG.warn("Could not fetch table schema for {}.", ref, e); + } + + // Create BigQuery schema map to use for formatting + String rowDetails; + try { + if (tableSchema != null) { + // Creates bqSchemaMap containing field name, field type, and + // possibly field mode if available. + Map<String, String> bqSchemaMap = + tableSchema.getFieldsList().stream() + .collect(Collectors.toMap(f -> f.getName(), f -> f.getType().name())); + rowDetails = formatRowWithSchema(row, bqSchemaMap); + } else { + rowDetails = formatRowWithoutSchema(row); + } + } catch (Exception e) { + rowDetails = row.keySet().toString(); + } + if (rowDetails.length() > 1024) { + rowDetails = rowDetails.substring(0, 1024) + "...}"; + } + throw new RuntimeException( 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. This is probably due to a schema " + + "mismatch. Problematic row schema " Review Comment: Perhaps we should tailor the error message based on whether rowDetails is empty or not. -- 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: github-unsubscr...@beam.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org