Uploaded image for project: 'Beam'
  1. Beam
  2. BEAM-8897

JdbcIO.readRows issue when type is Numeric

Details

    • Bug
    • Status: Resolved
    • P3
    • Resolution: Fixed
    • 2.16.0
    • Missing
    • io-java-jdbc
    • None
    • Java 8

    Description

      The actual use case is such that I am connecting to Oracle via JDBC making the query: "select 1 from dual". If I try this on real data with a Number type column the same issue occurs.

      The number type results in an exception. Example:

      PCollection<Row> row = p.apply(
              JdbcIO.readRows()
                  .withDataSourceProviderFn(JdbcIO.PoolableDataSourceProvider.of(configuration))
                  .withQuery("select 1 as t1 from dual")
                  .withFetchSize(10000)
                  .withOutputParallelization(true));
      
      
          row.apply(ParDo.of(new DoFn<Row, Row>() {
            @ProcessElement
            public void processElement(@Element Row row, OutputReceiver<Row> out, ProcessContext pc) {
              System.out.println(row.getSchema().toString() + " : " + row.toString());
              out.output(row);
            }
          }));
      
          p.run().waitUntilFinish();
      

      Exception:
      Exception in thread "main" org.apache.beam.sdk.Pipeline$PipelineExecutionException: java.lang.IllegalArgumentException
      at org.apache.beam.runners.direct.DirectRunner$DirectPipelineResult.waitUntilFinish(DirectRunner.java:348)
      at org.apache.beam.runners.direct.DirectRunner$DirectPipelineResult.waitUntilFinish(DirectRunner.java:318)
      at org.apache.beam.runners.direct.DirectRunner.run(DirectRunner.java:213)
      at org.apache.beam.runners.direct.DirectRunner.run(DirectRunner.java:67)
      at org.apache.beam.sdk.Pipeline.run(Pipeline.java:315)
      at org.apache.beam.sdk.Pipeline.run(Pipeline.java:301)
      at SimpleJdbc.main(SimpleJdbc.java:45)
      Caused by: java.lang.IllegalArgumentException
      at org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument(Preconditions.java:127)
      at org.apache.beam.sdk.io.jdbc.LogicalTypes$FixedPrecisionNumeric.toInputType(LogicalTypes.java:237)
      at org.apache.beam.sdk.io.jdbc.LogicalTypes$FixedPrecisionNumeric.toInputType(LogicalTypes.java:221)
      at org.apache.beam.sdk.io.jdbc.SchemaUtil.lambda$createLogicalTypeExtractor$ca0ab2ec$1(SchemaUtil.java:272)
      at org.apache.beam.sdk.io.jdbc.SchemaUtil$BeamRowMapper.mapRow(SchemaUtil.java:337)
      at org.apache.beam.sdk.io.jdbc.SchemaUtil$BeamRowMapper.mapRow(SchemaUtil.java:315)
      at org.apache.beam.sdk.io.jdbc.JdbcIO$ReadFn.processElement(JdbcIO.java:854)

      Attachments

        Issue Links

          Activity

            People

              Unassigned Unassigned
              terekete Mark Gates
              Votes:
              1 Vote for this issue
              Watchers:
              5 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved:

                Time Tracking

                  Estimated:
                  Original Estimate - Not Specified
                  Not Specified
                  Remaining:
                  Remaining Estimate - 0h
                  0h
                  Logged:
                  Time Spent - 0.5h
                  0.5h