[ 
https://issues.apache.org/jira/browse/BEAM-4257?focusedWorklogId=112752&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-112752
 ]

ASF GitHub Bot logged work on BEAM-4257:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 18/Jun/18 13:35
            Start Date: 18/Jun/18 13:35
    Worklog Time Spent: 10m 
      Work Description: reuvenlax commented on a change in pull request #5341: 
[BEAM-4257] Increases BigQuery streaming error information
URL: https://github.com/apache/beam/pull/5341#discussion_r196075533
 
 

 ##########
 File path: 
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryInsertErrorCoder.java
 ##########
 @@ -0,0 +1,101 @@
+/*
+ * 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 com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.SerializationFeature;
+import com.google.api.services.bigquery.model.TableDataInsertAllResponse;
+import com.google.api.services.bigquery.model.TableReference;
+import com.google.api.services.bigquery.model.TableRow;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.values.TypeDescriptor;
+
+/** A {@link Coder} that encodes BigQuery {@link BigQueryInsertError} objects. 
*/
+public class BigQueryInsertErrorCoder extends AtomicCoder<BigQueryInsertError> 
{
+
+  public static BigQueryInsertErrorCoder of() {
+    return INSTANCE;
+  }
+
+  @Override
+  public void encode(BigQueryInsertError value, OutputStream outStream) throws 
IOException {
+    String errorStrValue = MAPPER.writeValueAsString(value.getError());
+    StringUtf8Coder.of().encode(errorStrValue, outStream);
+
+    TableRowJsonCoder.of().encode(value.getRow(), outStream);
+
+    String tableStrValue = MAPPER.writeValueAsString(value.getTable());
 
 Review comment:
   No need to encode this class. You should be able to store just the string 
tablespec instead.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


Issue Time Tracking
-------------------

    Worklog Id:     (was: 112752)
    Time Spent: 3h 10m  (was: 3h)

> Add error reason and table destination to BigQueryIO streaming failed inserts
> -----------------------------------------------------------------------------
>
>                 Key: BEAM-4257
>                 URL: https://issues.apache.org/jira/browse/BEAM-4257
>             Project: Beam
>          Issue Type: Improvement
>          Components: io-java-gcp
>            Reporter: Carlos Alonso
>            Assignee: Carlos Alonso
>            Priority: Minor
>          Time Spent: 3h 10m
>  Remaining Estimate: 0h
>
> When using `BigQueryIO.Write` and getting `WriteResult.getFailedInserts()` we 
> get a `PCollection<TableRow>` which is fine, but in order to properly work on 
> the errors downstream having extended information such as the `InsertError` 
> fields and the `TableReference` it was routed to would be really valuable.
>  
> My suggestion is to create a new object that contains all that information 
> and return a `PCollection` of those instead.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to