You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2021/09/09 00:40:21 UTC

[GitHub] [beam] amuletxheart opened a new pull request #15485: [BEAM-10655] Fix conversion of NanosInstant to BigQuery Timestamp

amuletxheart opened a new pull request #15485:
URL: https://github.com/apache/beam/pull/15485


   We are trying to achieve Protobuf schema-first development by defining protos as the single source of truth and have Beam's useBeamSchema() convert it to other formats.
   
   When trying to write to BigQuery using BigQueryIO.Write, we hit the issue where `google.protobuf.Timestamp` is successfully converted to `org.apache.beam.sdk.schemas.logicaltypes.NanosInstant`, but from there it cannot be converted to BigQuery's TIMESTAMP type.
   
   The solution I came up with is to convert NanosInstant to StandardSQLTypeName.TIMESTAMP, losing 3 digits of precision, i.e. nanoseconds get truncated to microseconds.
   
   When reading from BigQuery into beam, the existing code converts it to Schema.FieldType.DATETIME, losing 6 digits of precision, i.e. nanoseconds get truncated to milliseconds.
   
   
   ------------------------
   
   Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:
   
    - [ ] [**Choose reviewer(s)**](https://beam.apache.org/contribute/#make-your-change) and mention them in a comment (`R: @username`).
    - [ ] Format the pull request title like `[BEAM-XXX] Fixes bug in ApproximateQuantiles`, where you replace `BEAM-XXX` with the appropriate JIRA issue, if applicable. This will automatically link the pull request to the issue.
    - [ ] Update `CHANGES.md` with noteworthy changes.
    - [ ] If this contribution is large, please file an Apache [Individual Contributor License Agreement](https://www.apache.org/licenses/icla.pdf).
   
   See the [Contributor Guide](https://beam.apache.org/contribute) for more tips on [how to make review process smoother](https://beam.apache.org/contribute/#make-reviewers-job-easier).
   
   `ValidatesRunner` compliance status (on master branch)
   --------------------------------------------------------
   
   <table>
     <thead>
       <tr>
         <th>Lang</th>
         <th>ULR</th>
         <th>Dataflow</th>
         <th>Flink</th>
         <th>Samza</th>
         <th>Spark</th>
         <th>Twister2</th>
       </tr>
     </thead>
     <tbody>
       <tr>
         <td>Go</td>
         <td>---</td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Go/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Go/lastCompletedBuild/badge/icon">
           </a>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Go_VR_Flink/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Go_VR_Flink/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Go_VR_Samza/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Go_VR_Samza/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Go_VR_Spark/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Go_VR_Spark/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>---</td>
       </tr>
       <tr>
         <td>Java</td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_ULR/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_ULR/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow/lastCompletedBuild/badge/icon?subject=V1">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming/lastCompletedBuild/badge/icon?subject=V1+Streaming">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow_Java11/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow_Java11/lastCompletedBuild/badge/icon?subject=V1+Java+11">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java_VR_Dataflow_V2/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java_VR_Dataflow_V2/lastCompletedBuild/badge/icon?subject=V2">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java_VR_Dataflow_V2_Streaming/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java_VR_Dataflow_V2_Streaming/lastCompletedBuild/badge/icon?subject=V2+Streaming">
           </a><br>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink/lastCompletedBuild/badge/icon?subject=Java+8">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink_Java11/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink_Java11/lastCompletedBuild/badge/icon?subject=Java+11">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Flink_Batch/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Flink_Batch/lastCompletedBuild/badge/icon?subject=Portable">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Flink_Streaming/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Flink_Streaming/lastCompletedBuild/badge/icon?subject=Portable+Streaming">
           </a>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Samza/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Samza/lastCompletedBuild/badge/icon">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Samza/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Samza/lastCompletedBuild/badge/icon?subject=Portable">
           </a>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Spark/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Spark/lastCompletedBuild/badge/icon">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Spark_Batch/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Spark_Batch/lastCompletedBuild/badge/icon?subject=Portable">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming/lastCompletedBuild/badge/icon?subject=Structured+Streaming">
           </a>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Twister2/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Twister2/lastCompletedBuild/badge/icon">
           </a>
         </td>
       </tr>
       <tr>
         <td>Python</td>
         <td>---</td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Py_VR_Dataflow/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Py_VR_Dataflow/lastCompletedBuild/badge/icon?subject=V1">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Py_VR_Dataflow_V2/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Py_VR_Dataflow_V2/lastCompletedBuild/badge/icon?subject=V2">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Py_ValCont/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Py_ValCont/lastCompletedBuild/badge/icon?subject=ValCont">
           </a>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PreCommit_Python_PVR_Flink_Cron/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PreCommit_Python_PVR_Flink_Cron/lastCompletedBuild/badge/icon?subject=Portable">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Python_VR_Flink/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Python_VR_Flink/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Python_VR_Samza/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Python_VR_Samza/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Python_VR_Spark/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Python_VR_Spark/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>---</td>
       </tr>
       <tr>
         <td>XLang</td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_XVR_Direct/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_XVR_Direct/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_XVR_Dataflow/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_XVR_Dataflow/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_XVR_Flink/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_XVR_Flink/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_XVR_Samza/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_XVR_Samza/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_XVR_Spark/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_XVR_Spark/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>---</td>
       </tr>
     </tbody>
   </table>
   
   Examples testing status on various runners
   --------------------------------------------------------
   
   <table>
     <thead>
       <tr>
         <th>Lang</th>
         <th>ULR</th>
         <th>Dataflow</th>
         <th>Flink</th>
         <th>Samza</th>
         <th>Spark</th>
         <th>Twister2</th>
       </tr>
     </thead>
     <tbody>
       <tr>
         <td>Go</td>
         <td>---</td>
         <td>---</td>
         <td>---</td>
         <td>---</td>
         <td>---</td>
         <td>---</td>
         <td>---</td>
       </tr>
       <tr>
         <td>Java</td>
         <td>---</td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PreCommit_Java_Examples_Dataflow_Cron/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PreCommit_Java_Examples_Dataflow_Cron/lastCompletedBuild/badge/icon?subject=V1">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PreCommit_Java_Examples_Dataflow_Java11_Cron/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PreCommit_Java_Examples_Dataflow_Java11_Cron/lastCompletedBuild/badge/icon?subject=V1+Java11">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java_Examples_Dataflow_V2/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java_Examples_Dataflow_V2/lastCompletedBuild/badge/icon?subject=V2">
           </a><br>
         </td>
         <td>---</td>
         <td>---</td>
         <td>---</td>
         <td>---</td>
         <td>---</td>
       </tr>
       <tr>
         <td>Python</td>
         <td>---</td>
         <td>---</td>
         <td>---</td>
         <td>---</td>
         <td>---</td>
         <td>---</td>
         <td>---</td>
       </tr>
       <tr>
         <td>XLang</td>
         <td>---</td>
         <td>---</td>
         <td>---</td>
         <td>---</td>
         <td>---</td>
         <td>---</td>
         <td>---</td>
       </tr>
     </tbody>
   </table>
   
   Post-Commit SDK/Transform Integration Tests Status (on master branch)
   ------------------------------------------------------------------------------------------------
   
   <table>
     <thead>
       <tr>
         <th>Go</th>
         <th>Java</th>
         <th>Python</th>
       </tr>
     </thead>
     <tbody>
       <tr>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Go/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Go/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Python36/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Python36/lastCompletedBuild/badge/icon?subject=3.6">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Python37/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Python37/lastCompletedBuild/badge/icon?subject=3.7">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Python38/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Python38/lastCompletedBuild/badge/icon?subject=3.8">
           </a>
         </td>
       </tr>
     </tbody>
   </table>
   
   Pre-Commit Tests Status (on master branch)
   ------------------------------------------------------------------------------------------------
   
   <table>
     <thead>
       <tr>
         <th>---</th>
         <th>Java</th>
         <th>Python</th>
         <th>Go</th>
         <th>Website</th>
         <th>Whitespace</th>
         <th>Typescript</th>
       </tr>
     </thead>
     <tbody>
       <tr>
         <td>Non-portable</td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PreCommit_Java_Cron/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PreCommit_Java_Cron/lastCompletedBuild/badge/icon">
           </a><br>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PreCommit_Python_Cron/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PreCommit_Python_Cron/lastCompletedBuild/badge/icon?subject=Tests">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PreCommit_PythonLint_Cron/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PreCommit_PythonLint_Cron/lastCompletedBuild/badge/icon?subject=Lint">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PreCommit_PythonDocker_Cron/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PreCommit_PythonDocker_Cron/badge/icon?subject=Docker">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PreCommit_PythonDocs_Cron/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PreCommit_PythonDocs_Cron/badge/icon?subject=Docs">
           </a>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PreCommit_Go_Cron/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PreCommit_Go_Cron/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PreCommit_Website_Cron/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PreCommit_Website_Cron/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PreCommit_Whitespace_Cron/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PreCommit_Whitespace_Cron/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PreCommit_Typescript_Cron/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PreCommit_Typescript_Cron/lastCompletedBuild/badge/icon">
           </a>
         </td>
       </tr>
       <tr>
         <td>Portable</td>
         <td>---</td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PreCommit_Portable_Python_Cron/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PreCommit_Portable_Python_Cron/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PreCommit_GoPortable_Cron/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PreCommit_GoPortable_Cron/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>---</td>
         <td>---</td>
         <td>---</td>
       </tr>
     </tbody>
   </table>
   
   See [.test-infra/jenkins/README](https://github.com/apache/beam/blob/master/.test-infra/jenkins/README.md) for trigger phrase, status and link of all Jenkins jobs.
   
   
   GitHub Actions Tests Status (on master branch)
   ------------------------------------------------------------------------------------------------
   [![Build python source distribution and wheels](https://github.com/apache/beam/workflows/Build%20python%20source%20distribution%20and%20wheels/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Build+python+source+distribution+and+wheels%22+branch%3Amaster+event%3Aschedule)
   [![Python tests](https://github.com/apache/beam/workflows/Python%20tests/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Python+Tests%22+branch%3Amaster+event%3Aschedule)
   [![Java tests](https://github.com/apache/beam/workflows/Java%20Tests/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Java+Tests%22+branch%3Amaster+event%3Aschedule)
   
   See [CI.md](https://github.com/apache/beam/blob/master/CI.md) for more information about GitHub Actions CI.
   


-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] amuletxheart commented on pull request #15485: [BEAM-10655] Fix conversion of NanosInstant to BigQuery Timestamp

Posted by GitBox <gi...@apache.org>.
amuletxheart commented on pull request #15485:
URL: https://github.com/apache/beam/pull/15485#issuecomment-1020686106


    ​retest this please


-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] TheNeuralBit commented on a change in pull request #15485: [BEAM-10655] Fix conversion of NanosInstant to BigQuery Timestamp

Posted by GitBox <gi...@apache.org>.
TheNeuralBit commented on a change in pull request #15485:
URL: https://github.com/apache/beam/pull/15485#discussion_r801149024



##########
File path: sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java
##########
@@ -2320,6 +2322,46 @@ public void testWriteToTableDecorator() throws Exception {
     p.run();
   }
 
+  @Test
+  public void testWriteWithAllowTruncatedTimestamps() throws IOException, InterruptedException {
+    if (useStorageApi) {
+      // TODO: to support storage API, changes have to be made to
+      // org.apache.beam.sdk.io.gcp.bigquery.BeamRowToStorageApiProto
+      return;

Review comment:
       Could this test that we fail gracefully in this case? It would also be good to file a jira for the follow up work and reference that, e.g. TODO(BEAM-XXX): 

##########
File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java
##########
@@ -594,10 +604,32 @@ public static TableRow toTableRow(Row row) {
           java.time.format.DateTimeFormatter localDateTimeFormatter =
               (0 == localDateTime.getNano()) ? ISO_LOCAL_DATE_TIME : BIGQUERY_DATETIME_FORMATTER;
           return localDateTimeFormatter.format(localDateTime);
-        } else if ("Enum".equals(identifier)) {
+        } else if (EnumerationType.IDENTIFIER.equals(identifier)) {

Review comment:
       Thanks for the cleanup :)

##########
File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java
##########
@@ -2372,6 +2377,10 @@ static String getExtractDestinationUri(String extractDestinationDir) {
       return toBuilder().setExtendedErrorInfo(true).build();
     }
 
+    public Write<T> withAllowTruncatedTimestamps() {

Review comment:
       Please add a docstring for this

##########
File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TestBigQuery.java
##########
@@ -211,7 +211,7 @@ public TableReference tableReference() {
   public TableDataInsertAllResponse insertRows(Schema rowSchema, Row... rows) throws IOException {
     List<Rows> bqRows =
         Arrays.stream(rows)
-            .map(row -> new Rows().setJson(BigQueryUtils.toTableRow(row)))

Review comment:
       Could you also add an overload, `BigQueryUtils.toTableRow(row)`, that preserves the existing API like you did with `BigQueryUtils.toTableRow()`? Then you wouldn't have to change calls like this one, and we preserve the existing API which is technically public (although practically no one should be relying on this).




-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] kileys commented on pull request #15485: [BEAM-10655] Fix conversion of NanosInstant to BigQuery Timestamp

Posted by GitBox <gi...@apache.org>.
kileys commented on pull request #15485:
URL: https://github.com/apache/beam/pull/15485#issuecomment-1082328175


   @amuletxheart Could you follow up on the final changes so we can get this merged? Thanks!


-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] TheNeuralBit commented on a change in pull request #15485: [BEAM-10655] Fix conversion of NanosInstant to BigQuery Timestamp

Posted by GitBox <gi...@apache.org>.
TheNeuralBit commented on a change in pull request #15485:
URL: https://github.com/apache/beam/pull/15485#discussion_r717829804



##########
File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java
##########
@@ -551,9 +548,9 @@ public static TableRow toTableRow(Row row) {
         return toTableRow((Row) fieldValue);
 
       case DATETIME:
-        return ((Instant) fieldValue)
-            .toDateTime(DateTimeZone.UTC)
-            .toString(BIGQUERY_TIMESTAMP_PRINTER);
+        org.joda.time.Instant jodaInstant = (org.joda.time.Instant) fieldValue;
+        java.time.Instant javaInstant = java.time.Instant.ofEpochMilli(jodaInstant.getMillis());
+        return BIGQUERY_TIMESTAMP_PRINTER.format(javaInstant);

Review comment:
       (FYI I edited your message, you said DATETIME/joda Instant was limited to microseconds, but I think you meant milliseconds)
   
   You're right this is an improvement over the existing solution (converting NanosInstant to DATETIME with millisecond precision), because it gets the full supported microsecond precision into BigQuery. But there is an advantage with the existing solution: the user has to opt-in to the truncation, by converting to DATETIME. I feel strongly that we shouldn't be truncating implicitly, and if we're doing that elsewhere I'd consider it a bug. Perhaps another reviewer would feel differently, but that is my position.
   
   That being said, you make a fair point that adding an option may be more work than you want to take on. Would it be a reasonable compromise to make this PR use the "erroring" approach, and file a follow-up ticket to add an option for truncation?
   
   




-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] amuletxheart removed a comment on pull request #15485: [BEAM-10655] Fix conversion of NanosInstant to BigQuery Timestamp

Posted by GitBox <gi...@apache.org>.
amuletxheart removed a comment on pull request #15485:
URL: https://github.com/apache/beam/pull/15485#issuecomment-1020686106


    ​retest this please


-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] blackhogz commented on a change in pull request #15485: [BEAM-10655] Fix conversion of NanosInstant to BigQuery Timestamp

Posted by GitBox <gi...@apache.org>.
blackhogz commented on a change in pull request #15485:
URL: https://github.com/apache/beam/pull/15485#discussion_r719102764



##########
File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java
##########
@@ -551,9 +548,9 @@ public static TableRow toTableRow(Row row) {
         return toTableRow((Row) fieldValue);
 
       case DATETIME:
-        return ((Instant) fieldValue)
-            .toDateTime(DateTimeZone.UTC)
-            .toString(BIGQUERY_TIMESTAMP_PRINTER);
+        org.joda.time.Instant jodaInstant = (org.joda.time.Instant) fieldValue;
+        java.time.Instant javaInstant = java.time.Instant.ofEpochMilli(jodaInstant.getMillis());
+        return BIGQUERY_TIMESTAMP_PRINTER.format(javaInstant);

Review comment:
       Got it. IIUC, this means we should not error out in `BigQueryUtils.toTableSchema()` but rather check the flag and error out case by case in `BigQueryUtils.toTableRow()` similar to the snippet you linked.
   
   I think @amuletxheart and I will have to defer the re-architecting work to the maintainers who definitely have a much better understanding of the overall architecture than us, even though I agree passing an argument to `toTableRow()` is a bit messy.
   
   That said, are you okay with the proposed `BigQueryIO.Write#allowTruncatedTimestamps()` change, as I think this is kinda part of the "public API" of the BigQueryIO, while `toTableRow()` is an private implementation that is easier to change when we later undergo the re-architecturing work.




-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] amuletxheart commented on a change in pull request #15485: [BEAM-10655] Fix conversion of NanosInstant to BigQuery Timestamp

Posted by GitBox <gi...@apache.org>.
amuletxheart commented on a change in pull request #15485:
URL: https://github.com/apache/beam/pull/15485#discussion_r839076207



##########
File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TestBigQuery.java
##########
@@ -211,7 +211,7 @@ public TableReference tableReference() {
   public TableDataInsertAllResponse insertRows(Schema rowSchema, Row... rows) throws IOException {
     List<Rows> bqRows =
         Arrays.stream(rows)
-            .map(row -> new Rows().setJson(BigQueryUtils.toTableRow(row)))

Review comment:
       @TheNeuralBit Sure, I've added them.




-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] TheNeuralBit commented on a change in pull request #15485: [BEAM-10655] Fix conversion of NanosInstant to BigQuery Timestamp

Posted by GitBox <gi...@apache.org>.
TheNeuralBit commented on a change in pull request #15485:
URL: https://github.com/apache/beam/pull/15485#discussion_r709603791



##########
File path: sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilsTest.java
##########
@@ -626,6 +666,14 @@ public void testToTableRow_enum() {
     assertThat(row, hasEntry("color", "GREEN"));
   }
 
+  @Test
+  public void testToTableRow_nanosInstant() {
+    TableRow row = toTableRow().apply(NANOS_INSTANT_ROW);
+
+    assertThat(row.size(), equalTo(1));
+    assertThat(row, hasEntry("nanos_instant", "2021-09-03T18:12:12.123456789Z"));

Review comment:
       Will this TableRow work when it hits BigQuery? or will it raise an error? I think we may need to either:
   - Reject this row in Beam, or
   - Truncate the timestamp proactively
   
   It would be best if the user could choose between these approaches with an option on `BigQueryIO.Write`




-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] amuletxheart commented on a change in pull request #15485: [BEAM-10655] Fix conversion of NanosInstant to BigQuery Timestamp

Posted by GitBox <gi...@apache.org>.
amuletxheart commented on a change in pull request #15485:
URL: https://github.com/apache/beam/pull/15485#discussion_r715944187



##########
File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java
##########
@@ -551,9 +548,9 @@ public static TableRow toTableRow(Row row) {
         return toTableRow((Row) fieldValue);
 
       case DATETIME:
-        return ((Instant) fieldValue)
-            .toDateTime(DateTimeZone.UTC)
-            .toString(BIGQUERY_TIMESTAMP_PRINTER);
+        org.joda.time.Instant jodaInstant = (org.joda.time.Instant) fieldValue;
+        java.time.Instant javaInstant = java.time.Instant.ofEpochMilli(jodaInstant.getMillis());
+        return BIGQUERY_TIMESTAMP_PRINTER.format(javaInstant);

Review comment:
       @pabloem @TheNeuralBit This part was missing from my earlier attempt as I did not fully understand the codebase at that point in time. The TableRow just contained a String in the Java Instant format, and not the native BigQuery TIMESTAMP format as it was skipping past all the if statements. I apologize for this slip-up.
   
   As for whether to reject or truncate nanos to micros, I think we should follow the pattern established by other SQL types and do the truncation implicitly. Anyway, this commit will be an improvement as the previous SQL DATETIME used org.joda.time.Instant, which is limited to microseconds precision. The switch to java.time.Instant will improve it to nanos precision, and then truncating to micros at the last step when formatting to String for TableRow to accept.
   
   Regarding @TheNeuralBit suggestion to allow the user to select the option of truncating vs erroring, I think it will increase the scope of the ticket, which is originally to solve the Protobuf -> BigQuery pipeline for google.protobuf.Timestamp fields. We can open another ticket to figure out what options are open to user configuration, and how best to pass them to the existing control flow.




-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] amuletxheart commented on pull request #15485: [BEAM-10655] Fix conversion of NanosInstant to BigQuery Timestamp

Posted by GitBox <gi...@apache.org>.
amuletxheart commented on pull request #15485:
URL: https://github.com/apache/beam/pull/15485#issuecomment-1020686106


    ​retest this please


-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] amuletxheart commented on a change in pull request #15485: [BEAM-10655] Fix conversion of NanosInstant to BigQuery Timestamp

Posted by GitBox <gi...@apache.org>.
amuletxheart commented on a change in pull request #15485:
URL: https://github.com/apache/beam/pull/15485#discussion_r839083729



##########
File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java
##########
@@ -2372,6 +2377,10 @@ static String getExtractDestinationUri(String extractDestinationDir) {
       return toBuilder().setExtendedErrorInfo(true).build();
     }
 
+    public Write<T> withAllowTruncatedTimestamps() {

Review comment:
       @TheNeuralBit Added the comments, please take a look.




-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] blackhogz commented on a change in pull request #15485: [BEAM-10655] Fix conversion of NanosInstant to BigQuery Timestamp

Posted by GitBox <gi...@apache.org>.
blackhogz commented on a change in pull request #15485:
URL: https://github.com/apache/beam/pull/15485#discussion_r719102764



##########
File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java
##########
@@ -551,9 +548,9 @@ public static TableRow toTableRow(Row row) {
         return toTableRow((Row) fieldValue);
 
       case DATETIME:
-        return ((Instant) fieldValue)
-            .toDateTime(DateTimeZone.UTC)
-            .toString(BIGQUERY_TIMESTAMP_PRINTER);
+        org.joda.time.Instant jodaInstant = (org.joda.time.Instant) fieldValue;
+        java.time.Instant javaInstant = java.time.Instant.ofEpochMilli(jodaInstant.getMillis());
+        return BIGQUERY_TIMESTAMP_PRINTER.format(javaInstant);

Review comment:
       Got it. IIUC, this means we should not error out in `BigQueryUtils.toTableSchema()` but rather check the flag and error out case by case in `BigQueryUtils.toTableRow()` as the snippet you linked.
   
   I think @amuletxheart and I will have to defer the re-architecting work to the maintainers who definitely have a much better understanding of the overall architecture than us, even though I agree passing an argument to `toTableRow()` is a bit messy.
   
   That said, are you okay with the proposed `BigQueryIO.Write#allowTruncatedTimestamps()` change, as I think this is kinda part of the "public API" of the BigQueryIO, while `toTableRow()` is an private implementation that is easier to change when we later undergo the re-architecturing work.




-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] amuletxheart commented on pull request #15485: [BEAM-10655] Fix conversion of NanosInstant to BigQuery Timestamp

Posted by GitBox <gi...@apache.org>.
amuletxheart commented on pull request #15485:
URL: https://github.com/apache/beam/pull/15485#issuecomment-915675292


   retest this please


-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] amuletxheart commented on a change in pull request #15485: [BEAM-10655] Fix conversion of NanosInstant to BigQuery Timestamp

Posted by GitBox <gi...@apache.org>.
amuletxheart commented on a change in pull request #15485:
URL: https://github.com/apache/beam/pull/15485#discussion_r715944187



##########
File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java
##########
@@ -551,9 +548,9 @@ public static TableRow toTableRow(Row row) {
         return toTableRow((Row) fieldValue);
 
       case DATETIME:
-        return ((Instant) fieldValue)
-            .toDateTime(DateTimeZone.UTC)
-            .toString(BIGQUERY_TIMESTAMP_PRINTER);
+        org.joda.time.Instant jodaInstant = (org.joda.time.Instant) fieldValue;
+        java.time.Instant javaInstant = java.time.Instant.ofEpochMilli(jodaInstant.getMillis());
+        return BIGQUERY_TIMESTAMP_PRINTER.format(javaInstant);

Review comment:
       @pabloem @TheNeuralBit This part was missing from my earlier attempt as I did not fully understand the codebase at that point in time. The TableRow just contained a String in the Java Instant format, and not the native BigQuery TIMESTAMP format as it was skipping past all the if statements. I apologize for this slip-up.
   
   As for whether to reject or truncate nanos to micros, I think we should follow the pattern established by other SQL types and do the truncation implicitly. Anyway, this commit will be an improvement as the previous SQL DATETIME used org.joda.time.Instant, which is limited to microseconds precision. The switch to java.time.Instant will improve it to nanos precision, and then truncating to micros at the last step when formatting to String for TableRow to accept.
   
   Regarding @TheNeuralBit suggestion to allow the user to select the option of truncating vs erroring, I think it will increase the scope of the ticket, which is originally to solve the Protobuf -> BigQuery pipeline for google.protobuf.Timestamp fields. We can open another ticket to figure out what options are open to user configuration, and how best to pass it to the existing control flow.




-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] amuletxheart commented on pull request #15485: [BEAM-10655] Fix conversion of NanosInstant to BigQuery Timestamp

Posted by GitBox <gi...@apache.org>.
amuletxheart commented on pull request #15485:
URL: https://github.com/apache/beam/pull/15485#issuecomment-926071894


   > any updates @amuletxheart ?
   
   @pabloem Sorry for the protracted updates. I got caught up with problems in my own codebase over the past weeks. It's all good now and I'm now back into focusing on this. I am trying to resolve the issues brought up by @TheNeuralBit as we're speaking.


-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] TheNeuralBit commented on a change in pull request #15485: [BEAM-10655] Fix conversion of NanosInstant to BigQuery Timestamp

Posted by GitBox <gi...@apache.org>.
TheNeuralBit commented on a change in pull request #15485:
URL: https://github.com/apache/beam/pull/15485#discussion_r718894348



##########
File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java
##########
@@ -551,9 +548,9 @@ public static TableRow toTableRow(Row row) {
         return toTableRow((Row) fieldValue);
 
       case DATETIME:
-        return ((Instant) fieldValue)
-            .toDateTime(DateTimeZone.UTC)
-            .toString(BIGQUERY_TIMESTAMP_PRINTER);
+        org.joda.time.Instant jodaInstant = (org.joda.time.Instant) fieldValue;
+        java.time.Instant javaInstant = java.time.Instant.ofEpochMilli(jodaInstant.getMillis());
+        return BIGQUERY_TIMESTAMP_PRINTER.format(javaInstant);

Review comment:
       It's a little messy that we'd need to pass the option as an argument in `toTableRow`, but I think to do anything cleaner would take a major re-architecting. We might create another type like ConversionOptions for this. 
   
   I have an important note regarding this statement though:
   > This should reject, i.e. error out, before any rows are processed, i.e. before formatFunction is triggered.
   
   It shouldn't error out universally. Instead it should raise an error _if the timestamp requires nanosecond precision_. That is, the timestamp 0.123456000 would be processed fine, since we can truncate it to microsecond precision without losing any information. However, the timestamp 0.123456789 would yield an error at execution time.
   
   Similar to this logic from the read side: https://github.com/apache/beam/blob/c8568eb372e2e432ce0a9998fa2864c090ddf44a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java#L762-L765
   
   




-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] amuletxheart commented on a change in pull request #15485: [BEAM-10655] Fix conversion of NanosInstant to BigQuery Timestamp

Posted by GitBox <gi...@apache.org>.
amuletxheart commented on a change in pull request #15485:
URL: https://github.com/apache/beam/pull/15485#discussion_r715944187



##########
File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java
##########
@@ -551,9 +548,9 @@ public static TableRow toTableRow(Row row) {
         return toTableRow((Row) fieldValue);
 
       case DATETIME:
-        return ((Instant) fieldValue)
-            .toDateTime(DateTimeZone.UTC)
-            .toString(BIGQUERY_TIMESTAMP_PRINTER);
+        org.joda.time.Instant jodaInstant = (org.joda.time.Instant) fieldValue;
+        java.time.Instant javaInstant = java.time.Instant.ofEpochMilli(jodaInstant.getMillis());
+        return BIGQUERY_TIMESTAMP_PRINTER.format(javaInstant);

Review comment:
       @pabloem @TheNeuralBit This part was missing from my earlier attempt as I did not fully understand the codebase at that point in time. The TableRow just contained a String in the Java Instant format, and not the native BigQuery TIMESTAMP format as it was skipping past all the if statements. I apologize for this slip-up.
   
   As for whether to reject or truncate nanos to micros, I think we should follow the pattern established by other SQL types and do the truncation implicitly. Anyway, this commit will be an improvement as the previous SQL DATETIME used org.joda.time.Instant, which is limited to milliseconds precision. The switch to java.time.Instant will improve it to nanos precision, and then truncating to micros at the last step when formatting to String for TableRow to accept.
   
   Regarding @TheNeuralBit suggestion to allow the user to select the option of truncating vs erroring, I think it will increase the scope of the ticket, which is originally to solve the Protobuf -> BigQuery pipeline for google.protobuf.Timestamp fields. We can open another ticket to figure out what options are open to user configuration, and how best to pass them to the existing control flow.




-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] blackhogz commented on a change in pull request #15485: [BEAM-10655] Fix conversion of NanosInstant to BigQuery Timestamp

Posted by GitBox <gi...@apache.org>.
blackhogz commented on a change in pull request #15485:
URL: https://github.com/apache/beam/pull/15485#discussion_r720040554



##########
File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java
##########
@@ -551,9 +548,9 @@ public static TableRow toTableRow(Row row) {
         return toTableRow((Row) fieldValue);
 
       case DATETIME:
-        return ((Instant) fieldValue)
-            .toDateTime(DateTimeZone.UTC)
-            .toString(BIGQUERY_TIMESTAMP_PRINTER);
+        org.joda.time.Instant jodaInstant = (org.joda.time.Instant) fieldValue;
+        java.time.Instant javaInstant = java.time.Instant.ofEpochMilli(jodaInstant.getMillis());
+        return BIGQUERY_TIMESTAMP_PRINTER.format(javaInstant);

Review comment:
       Sorry forgot to tag @TheNeuralBit in my previous reply. Edited.




-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] TheNeuralBit commented on a change in pull request #15485: [BEAM-10655] Fix conversion of NanosInstant to BigQuery Timestamp

Posted by GitBox <gi...@apache.org>.
TheNeuralBit commented on a change in pull request #15485:
URL: https://github.com/apache/beam/pull/15485#discussion_r718894348



##########
File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java
##########
@@ -551,9 +548,9 @@ public static TableRow toTableRow(Row row) {
         return toTableRow((Row) fieldValue);
 
       case DATETIME:
-        return ((Instant) fieldValue)
-            .toDateTime(DateTimeZone.UTC)
-            .toString(BIGQUERY_TIMESTAMP_PRINTER);
+        org.joda.time.Instant jodaInstant = (org.joda.time.Instant) fieldValue;
+        java.time.Instant javaInstant = java.time.Instant.ofEpochMilli(jodaInstant.getMillis());
+        return BIGQUERY_TIMESTAMP_PRINTER.format(javaInstant);

Review comment:
       It's a little messy that we'd need to pass the option as an argument in `toTableRow`, but I think to do anything cleaner would take a major re-architecting. We might create another type like ConversionOptions for this. 
   
   I have an important note regarding this statement though:
   > This should reject, i.e. error out, before any rows are processed, i.e. before formatFunction is triggered.
   
   It shouldn't error out universally. Instead it should raise an error _if the timestamp requires nanosecond precision_. That is, the timestamp 0.123456000 would be processed fine, since we can truncate it to microsecond precision without losing any information. However, the timestamp 0.123456789 would yield an error at execution time.
   
   Similar to this logic from the read side: https://github.com/apache/beam/blob/c8568eb372e2e432ce0a9998fa2864c090ddf44a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java#L762-L765
   
   




-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] pabloem commented on pull request #15485: [BEAM-10655] Fix conversion of NanosInstant to BigQuery Timestamp

Posted by GitBox <gi...@apache.org>.
pabloem commented on pull request #15485:
URL: https://github.com/apache/beam/pull/15485#issuecomment-926062080


   any updates @amuletxheart ?


-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] amuletxheart commented on pull request #15485: [BEAM-10655] Fix conversion of NanosInstant to BigQuery Timestamp

Posted by GitBox <gi...@apache.org>.
amuletxheart commented on pull request #15485:
URL: https://github.com/apache/beam/pull/15485#issuecomment-915672911


   R: @lukecwik R: @chamikaramj R: @TheNeuralBit R: @robinyqiu 


-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] blackhogz commented on a change in pull request #15485: [BEAM-10655] Fix conversion of NanosInstant to BigQuery Timestamp

Posted by GitBox <gi...@apache.org>.
blackhogz commented on a change in pull request #15485:
URL: https://github.com/apache/beam/pull/15485#discussion_r718240426



##########
File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java
##########
@@ -551,9 +548,9 @@ public static TableRow toTableRow(Row row) {
         return toTableRow((Row) fieldValue);
 
       case DATETIME:
-        return ((Instant) fieldValue)
-            .toDateTime(DateTimeZone.UTC)
-            .toString(BIGQUERY_TIMESTAMP_PRINTER);
+        org.joda.time.Instant jodaInstant = (org.joda.time.Instant) fieldValue;
+        java.time.Instant javaInstant = java.time.Instant.ofEpochMilli(jodaInstant.getMillis());
+        return BIGQUERY_TIMESTAMP_PRINTER.format(javaInstant);

Review comment:
       Thanks @TheNeuralBit . I'm working together with @amuletxheart and also looking to see if I can be of any help.
   
   With the opt-in flag, would this below approach be a reasonable venue to proceed?
   
   - add a `BigQueryIO.Write#allowTruncatedTimestamps()` method for explicit opt-in (i.e. default false)
   - pass the value to `BigQueryUtils.toTableRow()` as another parameter at the call site [here](https://github.com/apache/beam/blob/0111cff88025f0dc783a0890078b769139c8ae36/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java#L2687), and `BigQueryUtils.toTableSchema()` [here](https://github.com/apache/beam/blob/0111cff88025f0dc783a0890078b769139c8ae36/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java#L2690)
   - Update `BigQueryUtils.toTableSchema()` to only accept NanosInstant logical type if the `allowTruncatedTimestamps` parameter passed in is true. This should reject, i.e. error out, before any rows are processed, i.e. before formatFunction is triggered. In fact, with this, I'm thinking we don't even need to pass `allowTruncatedTimestamps` to `BigQueryUtils.toTableRow()` any more.
   
   Please let us know what do you think? @TheNeuralBit and other maintainers. Thanks a lot!




-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] TheNeuralBit commented on a change in pull request #15485: [BEAM-10655] Fix conversion of NanosInstant to BigQuery Timestamp

Posted by GitBox <gi...@apache.org>.
TheNeuralBit commented on a change in pull request #15485:
URL: https://github.com/apache/beam/pull/15485#discussion_r717829804



##########
File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java
##########
@@ -551,9 +548,9 @@ public static TableRow toTableRow(Row row) {
         return toTableRow((Row) fieldValue);
 
       case DATETIME:
-        return ((Instant) fieldValue)
-            .toDateTime(DateTimeZone.UTC)
-            .toString(BIGQUERY_TIMESTAMP_PRINTER);
+        org.joda.time.Instant jodaInstant = (org.joda.time.Instant) fieldValue;
+        java.time.Instant javaInstant = java.time.Instant.ofEpochMilli(jodaInstant.getMillis());
+        return BIGQUERY_TIMESTAMP_PRINTER.format(javaInstant);

Review comment:
       (FYI I edited your message, you said DATETIME/joda Instant was limited to microseconds, but I think you meant milliseconds)
   
   You're right this is an improvement over the existing solution (converting NanosInstant to DATETIME with millisecond precision), because it gets the full supported microsecond precision into BigQuery. But there is an advantage with the existing solution: the user has to opt-in to the truncation, by converting to DATETIME. I feel strongly that we shouldn't be truncating implicitly, and if we're doing that elsewhere I'd consider it a bug. Perhaps another reviewer would feel differently, but that is my position.
   
   That being said, you make a fair point that adding an option may be more work than you want to take on. Would it be a reasonable compromise to make this PR use the "erroring" approach, and file a follow-up ticket to add an option for truncation?
   
   




-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] blackhogz commented on a change in pull request #15485: [BEAM-10655] Fix conversion of NanosInstant to BigQuery Timestamp

Posted by GitBox <gi...@apache.org>.
blackhogz commented on a change in pull request #15485:
URL: https://github.com/apache/beam/pull/15485#discussion_r718249345



##########
File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java
##########
@@ -594,10 +591,15 @@ public static TableRow toTableRow(Row row) {
           java.time.format.DateTimeFormatter localDateTimeFormatter =
               (0 == localDateTime.getNano()) ? ISO_LOCAL_DATE_TIME : BIGQUERY_DATETIME_FORMATTER;
           return localDateTimeFormatter.format(localDateTime);
-        } else if ("Enum".equals(identifier)) {
+        } else if (EnumerationType.IDENTIFIER.equals(identifier)) {
           return fieldType
               .getLogicalType(EnumerationType.class)
               .toString((EnumerationType.Value) fieldValue);
+        } else if (NanosInstant.IDENTIFIER.equals(identifier)) {
+          if (fieldValue instanceof java.time.Instant) {

Review comment:
       @amuletxheart I think we should align with other field types and just perform the type coercion without the if clause here. This allows us to fail loudly instead of silently dropping the field. 




-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] amuletxheart edited a comment on pull request #15485: [BEAM-10655] Fix conversion of NanosInstant to BigQuery Timestamp

Posted by GitBox <gi...@apache.org>.
amuletxheart edited a comment on pull request #15485:
URL: https://github.com/apache/beam/pull/15485#issuecomment-1006046004


   @aaltay @TheNeuralBit I was working on other stuff for a while and am getting back to this issue now. Will be implementing the BigQueryIO.Write#withAllowTruncatedTimestamps() solution.


-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] amuletxheart commented on pull request #15485: [BEAM-10655] Fix conversion of NanosInstant to BigQuery Timestamp

Posted by GitBox <gi...@apache.org>.
amuletxheart commented on pull request #15485:
URL: https://github.com/apache/beam/pull/15485#issuecomment-1018985528


   @pabloem @aaltay @TheNeuralBit @chamikaramj 
   Please have a look at my latest attempt at this ticket. Added a new method BigQueryIO.Write.withAllowTruncatedTimestamps() and propagated the boolean into BigQueryUtils.toTableRow(). 
   
   Following this change, the expected behavior is: 
   1. Without the withAllowTruncatedTimestamps() option - On a row by row basis, if the NanosInstant contains values which are effectively microseconds precision (trailing 3 zeros), it will be truncated to a microsecond precision string when sent to BigQuery API. If the NanosInstant contains values which overflow (last 3 digits non zeroes), it will error out.
   2. With the withAllowTruncatedTimestamps() option - We always truncate the NanosInstant to microsecond precision when converting to string required by BigQuery API.


-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] amuletxheart removed a comment on pull request #15485: [BEAM-10655] Fix conversion of NanosInstant to BigQuery Timestamp

Posted by GitBox <gi...@apache.org>.
amuletxheart removed a comment on pull request #15485:
URL: https://github.com/apache/beam/pull/15485#issuecomment-915675292


   retest this please


-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] amuletxheart commented on pull request #15485: [BEAM-10655] Fix conversion of NanosInstant to BigQuery Timestamp

Posted by GitBox <gi...@apache.org>.
amuletxheart commented on pull request #15485:
URL: https://github.com/apache/beam/pull/15485#issuecomment-1083920226


   @kileys @TheNeuralBit Sorry, I was working on other project these past weeks. I've made some changes to the latest version of this PR, please help to review. Thanks!


-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] amuletxheart commented on a change in pull request #15485: [BEAM-10655] Fix conversion of NanosInstant to BigQuery Timestamp

Posted by GitBox <gi...@apache.org>.
amuletxheart commented on a change in pull request #15485:
URL: https://github.com/apache/beam/pull/15485#discussion_r839085893



##########
File path: sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java
##########
@@ -2320,6 +2322,46 @@ public void testWriteToTableDecorator() throws Exception {
     p.run();
   }
 
+  @Test
+  public void testWriteWithAllowTruncatedTimestamps() throws IOException, InterruptedException {
+    if (useStorageApi) {
+      // TODO: to support storage API, changes have to be made to
+      // org.apache.beam.sdk.io.gcp.bigquery.BeamRowToStorageApiProto
+      return;

Review comment:
       @TheNeuralBit FYI this is the error that the test cases with `useStorageApi = true` fails with:
   
   `Caused by: java.lang.RuntimeException: Unsupported logical type LOGICAL_TYPE
   	at org.apache.beam.sdk.io.gcp.bigquery.BeamRowToStorageApiProto.fieldDescriptorFromBeamField(BeamRowToStorageApiProto.java:224)
   	at org.apache.beam.sdk.io.gcp.bigquery.BeamRowToStorageApiProto.descriptorSchemaFromBeamSchema(BeamRowToStorageApiProto.java:181)
   	at org.apache.beam.sdk.io.gcp.bigquery.BeamRowToStorageApiProto.getDescriptorFromSchema(BeamRowToStorageApiProto.java:143)
   	at org.apache.beam.sdk.io.gcp.bigquery.StorageApiDynamicDestinationsBeamRow$1.lambda$$0(StorageApiDynamicDestinationsBeamRow.java:54)
   	at org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.LocalCache$LocalManualCache$1.load(LocalCache.java:4876)
   	at org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3528)
   	at org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.LocalCache$Segment.loadSync(LocalCache.java:2277)
   	at org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2154)
   	at org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2044)`




-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] amuletxheart commented on a change in pull request #15485: [BEAM-10655] Fix conversion of NanosInstant to BigQuery Timestamp

Posted by GitBox <gi...@apache.org>.
amuletxheart commented on a change in pull request #15485:
URL: https://github.com/apache/beam/pull/15485#discussion_r715944187



##########
File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java
##########
@@ -551,9 +548,9 @@ public static TableRow toTableRow(Row row) {
         return toTableRow((Row) fieldValue);
 
       case DATETIME:
-        return ((Instant) fieldValue)
-            .toDateTime(DateTimeZone.UTC)
-            .toString(BIGQUERY_TIMESTAMP_PRINTER);
+        org.joda.time.Instant jodaInstant = (org.joda.time.Instant) fieldValue;
+        java.time.Instant javaInstant = java.time.Instant.ofEpochMilli(jodaInstant.getMillis());
+        return BIGQUERY_TIMESTAMP_PRINTER.format(javaInstant);

Review comment:
       @pabloem @TheNeuralBit This part was missing from my earlier attempt as I did not fully understand the codebase at that point in time. The TableRow just contained a String in the Java Instant format, and not the native BigQuery TIMESTAMP format as it was skipping past all the if statements. I apologize for this slip-up.
   
   As for whether to reject or truncate nanos to micros, I think we should follow the pattern established by other SQL types and do the truncation implicitly. Anyway, this commit will be an improvement as the previous SQL DATETIME used org.joda.time.Instant, which is limited to microseconds precision. The switch to java.time.Instant will improve it to nanos precision, and then truncating to micros at the last step when formatting to String for TableRow to accept.
   
   Regarding @TheNeuralBit suggestion to allow the option for selecting, I think it will increase the scope of the ticket, which is originally to solve the Protobuf -> BigQuery pipeline for google.protobuf.Timestamp fields. We can open another ticket to figure out what options are open to user configuration, and how best to pass it to the existing control flow.




-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] blackhogz commented on a change in pull request #15485: [BEAM-10655] Fix conversion of NanosInstant to BigQuery Timestamp

Posted by GitBox <gi...@apache.org>.
blackhogz commented on a change in pull request #15485:
URL: https://github.com/apache/beam/pull/15485#discussion_r719102764



##########
File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java
##########
@@ -551,9 +548,9 @@ public static TableRow toTableRow(Row row) {
         return toTableRow((Row) fieldValue);
 
       case DATETIME:
-        return ((Instant) fieldValue)
-            .toDateTime(DateTimeZone.UTC)
-            .toString(BIGQUERY_TIMESTAMP_PRINTER);
+        org.joda.time.Instant jodaInstant = (org.joda.time.Instant) fieldValue;
+        java.time.Instant javaInstant = java.time.Instant.ofEpochMilli(jodaInstant.getMillis());
+        return BIGQUERY_TIMESTAMP_PRINTER.format(javaInstant);

Review comment:
       Got it. IIUC, this means we should not error out in `BigQueryUtils.toTableSchema()` but rather check the flag and error out case by case in `BigQueryUtils.toTableRow()` similar to the snippet you linked.
   
   I think @amuletxheart and I will have to defer the re-architecting work to the maintainers who definitely have a much better understanding of the overall architecture than us, even though I agree passing an argument to `toTableRow()` is a bit messy.
   
   That said, @TheNeuralBit are you okay with the proposed `BigQueryIO.Write#allowTruncatedTimestamps()` change, as I think this is kinda part of the "public API" of the BigQueryIO, while `toTableRow()` is an private implementation that is easier to change when we later undergo the re-architecturing work.




-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] amuletxheart commented on pull request #15485: [BEAM-10655] Fix conversion of NanosInstant to BigQuery Timestamp

Posted by GitBox <gi...@apache.org>.
amuletxheart commented on pull request #15485:
URL: https://github.com/apache/beam/pull/15485#issuecomment-1006046004


   @aaltay @TheNeuralBit I was working on other stuff for a while and am getting back to this issue now. Will be implementing the BigQueryIO.Write#allowTruncatedTimestamps() solution.


-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] aaltay commented on pull request #15485: [BEAM-10655] Fix conversion of NanosInstant to BigQuery Timestamp

Posted by GitBox <gi...@apache.org>.
aaltay commented on pull request #15485:
URL: https://github.com/apache/beam/pull/15485#issuecomment-1030503195


   Thank you for the update @amuletxheart.
   
   @TheNeuralBit - could you please continue the review if you have time?


-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] amuletxheart removed a comment on pull request #15485: [BEAM-10655] Fix conversion of NanosInstant to BigQuery Timestamp

Posted by GitBox <gi...@apache.org>.
amuletxheart removed a comment on pull request #15485:
URL: https://github.com/apache/beam/pull/15485#issuecomment-1020686106


    ​retest this please


-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] blackhogz commented on a change in pull request #15485: [BEAM-10655] Fix conversion of NanosInstant to BigQuery Timestamp

Posted by GitBox <gi...@apache.org>.
blackhogz commented on a change in pull request #15485:
URL: https://github.com/apache/beam/pull/15485#discussion_r718240426



##########
File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java
##########
@@ -551,9 +548,9 @@ public static TableRow toTableRow(Row row) {
         return toTableRow((Row) fieldValue);
 
       case DATETIME:
-        return ((Instant) fieldValue)
-            .toDateTime(DateTimeZone.UTC)
-            .toString(BIGQUERY_TIMESTAMP_PRINTER);
+        org.joda.time.Instant jodaInstant = (org.joda.time.Instant) fieldValue;
+        java.time.Instant javaInstant = java.time.Instant.ofEpochMilli(jodaInstant.getMillis());
+        return BIGQUERY_TIMESTAMP_PRINTER.format(javaInstant);

Review comment:
       Thanks @TheNeuralBit . I'm working together with @amuletxheart and also looking to see if I can be of any help.
   
   With the opt-in flag, would this below approach be a reasonable venue to proceed?
   
   - add a `BigQueryIO.Write#allowTruncatedTimestamps()` method for explicit opt-in (i.e. default false)
   - pass the value to `BigQueryUtils.toTableRow()` as another parameter at the call site [here](https://github.com/apache/beam/blob/0111cff88025f0dc783a0890078b769139c8ae36/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java#L2687), and `BigQueryUtils.toTableSchema()` [here](https://github.com/apache/beam/blob/0111cff88025f0dc783a0890078b769139c8ae36/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java#L2690)
   - Update `BigQueryUtils.toTableSchema()` to only accept NanosInstant logical type if the `allowTruncatedTimestamps` parameter passed in is true. This should reject, i.e. error out, before any rows are processed, i.e. before formatFunction is triggered. In fact, with this, I'm thinking we don't even need to pass `allowTruncatedTimestamps` to `BigQueryUtils.toTableRow()` any more.
   
   Please let us know what do you think? @TheNeuralBit and other maintainers. Thanks a lot!

##########
File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java
##########
@@ -594,10 +591,15 @@ public static TableRow toTableRow(Row row) {
           java.time.format.DateTimeFormatter localDateTimeFormatter =
               (0 == localDateTime.getNano()) ? ISO_LOCAL_DATE_TIME : BIGQUERY_DATETIME_FORMATTER;
           return localDateTimeFormatter.format(localDateTime);
-        } else if ("Enum".equals(identifier)) {
+        } else if (EnumerationType.IDENTIFIER.equals(identifier)) {
           return fieldType
               .getLogicalType(EnumerationType.class)
               .toString((EnumerationType.Value) fieldValue);
+        } else if (NanosInstant.IDENTIFIER.equals(identifier)) {
+          if (fieldValue instanceof java.time.Instant) {

Review comment:
       @amuletxheart I think we should align with other field types and just perform the type coercion without the if clause here. This allows us to fail loudly instead of silently dropping the field. 




-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] TheNeuralBit commented on a change in pull request #15485: [BEAM-10655] Fix conversion of NanosInstant to BigQuery Timestamp

Posted by GitBox <gi...@apache.org>.
TheNeuralBit commented on a change in pull request #15485:
URL: https://github.com/apache/beam/pull/15485#discussion_r721585635



##########
File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java
##########
@@ -551,9 +548,9 @@ public static TableRow toTableRow(Row row) {
         return toTableRow((Row) fieldValue);
 
       case DATETIME:
-        return ((Instant) fieldValue)
-            .toDateTime(DateTimeZone.UTC)
-            .toString(BIGQUERY_TIMESTAMP_PRINTER);
+        org.joda.time.Instant jodaInstant = (org.joda.time.Instant) fieldValue;
+        java.time.Instant javaInstant = java.time.Instant.ofEpochMilli(jodaInstant.getMillis());
+        return BIGQUERY_TIMESTAMP_PRINTER.format(javaInstant);

Review comment:
       Yes `BigQueryIO.Write#allowTruncatedTimestamps()` sounds like a reasonable addition to the public API of BigQuery.  I might suggest `withAllowTruncatedTimestamps()` to be consistent with other boolean parameters, e.g. [`withExtendedErrorInfo`](https://beam.apache.org/releases/javadoc/2.32.0/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.Write.html#withExtendedErrorInfo--). @chamikaramj may be a better judge of what the public API should look like though.
   
   The reason I clarified my suggested default (non-truncating, error-raising) behavior was just in case that behavior is acceptable for your use-case. If it is then you could avoid creating the parameter for now if you want.
   




-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] TheNeuralBit commented on pull request #15485: [BEAM-10655] Fix conversion of NanosInstant to BigQuery Timestamp

Posted by GitBox <gi...@apache.org>.
TheNeuralBit commented on pull request #15485:
URL: https://github.com/apache/beam/pull/15485#issuecomment-944789869


   > What is the next step on this PR?
   
   IIUC @blackhogz and @amuletxheart are working on adding a `withAllowTruncatedTimestamps()` option. By default BigQueryIO will handle nanosecond precision timestamps by raising an (execution time) error for elements that that have precision beyond microsecond.


-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] amuletxheart commented on a change in pull request #15485: [BEAM-10655] Fix conversion of NanosInstant to BigQuery Timestamp

Posted by GitBox <gi...@apache.org>.
amuletxheart commented on a change in pull request #15485:
URL: https://github.com/apache/beam/pull/15485#discussion_r715944187



##########
File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java
##########
@@ -551,9 +548,9 @@ public static TableRow toTableRow(Row row) {
         return toTableRow((Row) fieldValue);
 
       case DATETIME:
-        return ((Instant) fieldValue)
-            .toDateTime(DateTimeZone.UTC)
-            .toString(BIGQUERY_TIMESTAMP_PRINTER);
+        org.joda.time.Instant jodaInstant = (org.joda.time.Instant) fieldValue;
+        java.time.Instant javaInstant = java.time.Instant.ofEpochMilli(jodaInstant.getMillis());
+        return BIGQUERY_TIMESTAMP_PRINTER.format(javaInstant);

Review comment:
       @pabloem @TheNeuralBit This part was missing from my earlier attempt as I did not fully understand the codebase at that point in time. The TableRow just contained a String in the Java Instant format, and not the native BigQuery TIMESTAMP format as it was skipping past all the if statements. I apologize for this slip-up.
   
   As for whether to reject or truncate nanos to micros, I think we should follow the pattern established by other SQL types and do the truncation implicitly. Anyway, this commit will be an improvement as the previous SQL DATETIME used org.joda.time.Instant, which is limited to milliseconds precision. The switch to java.time.Instant will improve it to nanos precision, and then truncating to micros at the last step when formatting to String for TableRow to accept.
   
   Regarding @TheNeuralBit suggestion to allow the user to select the option of truncating vs erroring, I think it will increase the scope of the ticket, which is originally to solve the Protobuf -> BigQuery pipeline for google.protobuf.Timestamp fields. We can open another ticket to figure out what options are open to user configuration, and how best to pass them to the existing control flow.




-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] amuletxheart commented on a change in pull request #15485: [BEAM-10655] Fix conversion of NanosInstant to BigQuery Timestamp

Posted by GitBox <gi...@apache.org>.
amuletxheart commented on a change in pull request #15485:
URL: https://github.com/apache/beam/pull/15485#discussion_r839077641



##########
File path: sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java
##########
@@ -2320,6 +2322,46 @@ public void testWriteToTableDecorator() throws Exception {
     p.run();
   }
 
+  @Test
+  public void testWriteWithAllowTruncatedTimestamps() throws IOException, InterruptedException {
+    if (useStorageApi) {
+      // TODO: to support storage API, changes have to be made to
+      // org.apache.beam.sdk.io.gcp.bigquery.BeamRowToStorageApiProto
+      return;

Review comment:
       @TheNeuralBit I'm going to create the Jira ticket. What do you mean by "fail gracefully"? Can you elaborate a little more, thanks. 




-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org