Mark Gates created BEAM-8897: -------------------------------- Summary: JdbcIO issue when type is Numeric Key: BEAM-8897 URL: https://issues.apache.org/jira/browse/BEAM-8897 Project: Beam Issue Type: Bug Components: io-java-jdbc Affects Versions: 2.16.0 Environment: Java 8 Reporter: Mark Gates
The actual use case is such that I am connecting to Oracle via JDBC making the query: "select 1 from dual". The number type results in an exception. Example: {code:java} 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(); {code} 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) -- This message was sent by Atlassian Jira (v8.3.4#803005)