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

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

                Author: ASF GitHub Bot
            Created on: 28/Sep/21 18:04
            Start Date: 28/Sep/21 18:04
    Worklog Time Spent: 10m 
      Work Description: chrismgrayftsinc commented on a change in pull request 
#15566:
URL: https://github.com/apache/beam/pull/15566#discussion_r717838656



##########
File path: 
sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/SchemaUtil.java
##########
@@ -351,7 +351,12 @@ private BeamRowMapper(Schema schema, 
List<ResultSetFieldExtractor> fieldExtracto
     public Row mapRow(ResultSet rs) throws Exception {
       Row.Builder rowBuilder = Row.withSchema(schema);
       for (int i = 0; i < schema.getFieldCount(); i++) {
-        rowBuilder.addValue(fieldExtractors.get(i).extract(rs, i + 1));
+        Object value = fieldExtractors.get(i).extract(rs, i + 1);
+        if (rs.wasNull() && schema.getField(i).getType().getNullable()) {
+          rowBuilder.addValue(null);
+        } else {
+          rowBuilder.addValue(value);

Review comment:
       Just to expand on backwards-compatibility a bit, if we added the null 
value to rows where the schema did not have the field as being nullable, then 
an exception would be thrown where it wasn't thrown before from 
`RowFieldMatcher.match`.  So the question is whether incorrect data is worse or 
newly-introduced crashes are worse.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


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

    Worklog Id:     (was: 656302)
    Time Spent: 50m  (was: 40m)

> JdbcIo readRows does not handle null numerical values properly
> --------------------------------------------------------------
>
>                 Key: BEAM-12925
>                 URL: https://issues.apache.org/jira/browse/BEAM-12925
>             Project: Beam
>          Issue Type: Bug
>          Components: io-java-jdbc
>    Affects Versions: 2.32.0
>            Reporter: Chris Gray
>            Priority: P2
>          Time Spent: 50m
>  Remaining Estimate: 0h
>
> Numerical values that are SQL NULL in the database are converted to 0 when 
> read by `readRows`.  This appears to be caused by SchemaUtil.java not 
> checking `ResultSet::wasNull` at line 253 in version 2.32.0.  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to