You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@beam.apache.org by "Denis (Jira)" <ji...@apache.org> on 2021/11/15 10:02:00 UTC

[jira] [Updated] (BEAM-13242) IllegalArgumentException while reading a Numeric column with a fixed precision

     [ https://issues.apache.org/jira/browse/BEAM-13242?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Denis updated BEAM-13242:
-------------------------
    Description: 
*Description*

 

I am trying to read a table in a Postgresql database. One of columns has a type definition 
{code:java}
Numeric(15, 2){code}
 

At runtime I am getting:
{code:java}
Caused by: java.lang.IllegalArgumentException: Expected BigDecimal base to be null or have precision = 15 (was 6), scale = 2 (was 2)	at org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument(Preconditions.java:477)	at org.apache.beam.sdk.io.jdbc.LogicalTypes$FixedPrecisionNumeric.toInputType(LogicalTypes.java:268)	at org.apache.beam.sdk.io.jdbc.LogicalTypes$FixedPrecisionNumeric.toInputType(LogicalTypes.java:246)	at org.apache.beam.sdk.io.jdbc.SchemaUtil.lambda$createLogicalTypeExtractor$ca0ab2ec$1(SchemaUtil.java:289)	at org.apache.beam.sdk.io.jdbc.SchemaUtil$BeamRowMapper.mapRow(SchemaUtil.java:354)	at org.apache.beam.sdk.io.jdbc.SchemaUtil$BeamRowMapper.mapRow(SchemaUtil.java:332)	at org.apache.beam.sdk.io.jdbc.JdbcIO$ReadFn.processElement(JdbcIO.java:1172) {code}
 

It seems that when JdbcIO reads a numeric column it might have smaller precision/scale than defined in DDL.

 

*Steps to reproduce:*

 
{code:java}
import org.apache.beam.sdk.io.jdbc.JdbcIO
import org.apache.beam.sdk.testing.PAssert
import org.apache.beam.sdk.testing.TestPipeline
import org.apache.beam.sdk.transforms.MapElements
import org.apache.beam.sdk.transforms.ProcessFunction
import org.apache.beam.sdk.values.TypeDescriptors
import org.junit.AfterClass
import org.junit.BeforeClass
import org.junit.Test
import org.testcontainers.containers.PostgreSQLContainer
import java.math.BigDecimal
import java.sql.Connection
import java.sql.DriverManager

class PostgresNumericBugTest {
    companion object {
        val pgContainer = PostgreSQLContainer<Nothing>("postgres:14.0")

        @BeforeClass
        @JvmStatic
        fun start() {
            pgContainer.start()

            getConnection().use {
                val stmt = it.createStatement()
                stmt.execute(
                    """create table account (
                          id integer not null primary key,
                          balance numeric(15, 2)
                        )
                    """.trimIndent()
                )
                stmt.execute("insert into account(id, balance) values(1, 5755.94)")
            }
        }

        @AfterClass
        @JvmStatic
        fun stop() {
            pgContainer.stop()
        }

        private fun getConnection(): Connection {
            return DriverManager.getConnection(
                pgContainer.jdbcUrl, pgContainer.username, pgContainer.password
            )
        }

        private fun getDataSourceConfiguration(): JdbcIO.DataSourceConfiguration =
            JdbcIO.DataSourceConfiguration.create(
                pgContainer.driverClassName,
                pgContainer.jdbcUrl
            )
                .withUsername(pgContainer.username)
                .withPassword(pgContainer.password)
    }

    @Test
    fun readNumeric() {
        val pipeline = TestPipeline.create().enableAbandonedNodeEnforcement(false)

        val pBalance = pipeline
            .apply(
                "Read Account from DB",
                JdbcIO
                    .readRows()
                    .withQuery("select balance from account")
                    .withDataSourceConfiguration(getDataSourceConfiguration())
            )
            .apply(
                "Get Balance",
                MapElements.into(TypeDescriptors.bigdecimals()).via(ProcessFunction { it.getDecimal(0) })
            )

        PAssert.that(pBalance).containsInAnyOrder(BigDecimal("5755.94"))

        pipeline.run()
    }
}
 {code}
 

*Expected result*

The test passes successfully.

  was:
I am trying to read a table in a Postgresql database. One of columns has a type definition 
{code:java}
Numeric(15, 2){code}
 

At runtime I am getting:
{code:java}
Caused by: java.lang.IllegalArgumentException: Expected BigDecimal base to be null or have precision = 15 (was 6), scale = 2 (was 2)	at org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument(Preconditions.java:477)	at org.apache.beam.sdk.io.jdbc.LogicalTypes$FixedPrecisionNumeric.toInputType(LogicalTypes.java:268)	at org.apache.beam.sdk.io.jdbc.LogicalTypes$FixedPrecisionNumeric.toInputType(LogicalTypes.java:246)	at org.apache.beam.sdk.io.jdbc.SchemaUtil.lambda$createLogicalTypeExtractor$ca0ab2ec$1(SchemaUtil.java:289)	at org.apache.beam.sdk.io.jdbc.SchemaUtil$BeamRowMapper.mapRow(SchemaUtil.java:354)	at org.apache.beam.sdk.io.jdbc.SchemaUtil$BeamRowMapper.mapRow(SchemaUtil.java:332)	at org.apache.beam.sdk.io.jdbc.JdbcIO$ReadFn.processElement(JdbcIO.java:1172) {code}
 

It seems that when JdbcIO reads a numeric column it might have smaller precision/scale than defined in DDL.

 

Here is a test I used to reproduce the issue:

 
{code:java}
import org.apache.beam.sdk.io.jdbc.JdbcIO
import org.apache.beam.sdk.testing.PAssert
import org.apache.beam.sdk.testing.TestPipeline
import org.apache.beam.sdk.transforms.MapElements
import org.apache.beam.sdk.transforms.ProcessFunction
import org.apache.beam.sdk.values.TypeDescriptors
import org.junit.AfterClass
import org.junit.BeforeClass
import org.junit.Test
import org.testcontainers.containers.PostgreSQLContainer
import java.math.BigDecimal
import java.sql.Connection
import java.sql.DriverManager

class PostgresNumericBugTest {
    companion object {
        val pgContainer = PostgreSQLContainer<Nothing>("postgres:14.0")

        @BeforeClass
        @JvmStatic
        fun start() {
            pgContainer.start()

            getConnection().use {
                val stmt = it.createStatement()
                stmt.execute(
                    """create table account (
                          id integer not null primary key,
                          balance numeric(15, 2)
                        )
                    """.trimIndent()
                )
                stmt.execute("insert into account(id, balance) values(1, 5755.94)")
            }
        }

        @AfterClass
        @JvmStatic
        fun stop() {
            pgContainer.stop()
        }

        private fun getConnection(): Connection {
            return DriverManager.getConnection(
                pgContainer.jdbcUrl, pgContainer.username, pgContainer.password
            )
        }

        private fun getDataSourceConfiguration(): JdbcIO.DataSourceConfiguration =
            JdbcIO.DataSourceConfiguration.create(
                pgContainer.driverClassName,
                pgContainer.jdbcUrl
            )
                .withUsername(pgContainer.username)
                .withPassword(pgContainer.password)
    }

    @Test
    fun readNumeric() {
        val pipeline = TestPipeline.create().enableAbandonedNodeEnforcement(false)

        val pBalance = pipeline
            .apply(
                "Read Account from DB",
                JdbcIO
                    .readRows()
                    .withQuery("select balance from account")
                    .withDataSourceConfiguration(getDataSourceConfiguration())
            )
            .apply(
                "Get Balance",
                MapElements.into(TypeDescriptors.bigdecimals()).via(ProcessFunction { it.getDecimal(0) })
            )

        PAssert.that(pBalance).containsInAnyOrder(BigDecimal("5755.94"))

        pipeline.run()
    }
}
 {code}


> IllegalArgumentException while reading a Numeric column with a fixed precision
> ------------------------------------------------------------------------------
>
>                 Key: BEAM-13242
>                 URL: https://issues.apache.org/jira/browse/BEAM-13242
>             Project: Beam
>          Issue Type: Bug
>          Components: io-java-jdbc
>    Affects Versions: 2.33.0
>            Reporter: Denis
>            Priority: P1
>
> *Description*
>  
> I am trying to read a table in a Postgresql database. One of columns has a type definition 
> {code:java}
> Numeric(15, 2){code}
>  
> At runtime I am getting:
> {code:java}
> Caused by: java.lang.IllegalArgumentException: Expected BigDecimal base to be null or have precision = 15 (was 6), scale = 2 (was 2)	at org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument(Preconditions.java:477)	at org.apache.beam.sdk.io.jdbc.LogicalTypes$FixedPrecisionNumeric.toInputType(LogicalTypes.java:268)	at org.apache.beam.sdk.io.jdbc.LogicalTypes$FixedPrecisionNumeric.toInputType(LogicalTypes.java:246)	at org.apache.beam.sdk.io.jdbc.SchemaUtil.lambda$createLogicalTypeExtractor$ca0ab2ec$1(SchemaUtil.java:289)	at org.apache.beam.sdk.io.jdbc.SchemaUtil$BeamRowMapper.mapRow(SchemaUtil.java:354)	at org.apache.beam.sdk.io.jdbc.SchemaUtil$BeamRowMapper.mapRow(SchemaUtil.java:332)	at org.apache.beam.sdk.io.jdbc.JdbcIO$ReadFn.processElement(JdbcIO.java:1172) {code}
>  
> It seems that when JdbcIO reads a numeric column it might have smaller precision/scale than defined in DDL.
>  
> *Steps to reproduce:*
>  
> {code:java}
> import org.apache.beam.sdk.io.jdbc.JdbcIO
> import org.apache.beam.sdk.testing.PAssert
> import org.apache.beam.sdk.testing.TestPipeline
> import org.apache.beam.sdk.transforms.MapElements
> import org.apache.beam.sdk.transforms.ProcessFunction
> import org.apache.beam.sdk.values.TypeDescriptors
> import org.junit.AfterClass
> import org.junit.BeforeClass
> import org.junit.Test
> import org.testcontainers.containers.PostgreSQLContainer
> import java.math.BigDecimal
> import java.sql.Connection
> import java.sql.DriverManager
> class PostgresNumericBugTest {
>     companion object {
>         val pgContainer = PostgreSQLContainer<Nothing>("postgres:14.0")
>         @BeforeClass
>         @JvmStatic
>         fun start() {
>             pgContainer.start()
>             getConnection().use {
>                 val stmt = it.createStatement()
>                 stmt.execute(
>                     """create table account (
>                           id integer not null primary key,
>                           balance numeric(15, 2)
>                         )
>                     """.trimIndent()
>                 )
>                 stmt.execute("insert into account(id, balance) values(1, 5755.94)")
>             }
>         }
>         @AfterClass
>         @JvmStatic
>         fun stop() {
>             pgContainer.stop()
>         }
>         private fun getConnection(): Connection {
>             return DriverManager.getConnection(
>                 pgContainer.jdbcUrl, pgContainer.username, pgContainer.password
>             )
>         }
>         private fun getDataSourceConfiguration(): JdbcIO.DataSourceConfiguration =
>             JdbcIO.DataSourceConfiguration.create(
>                 pgContainer.driverClassName,
>                 pgContainer.jdbcUrl
>             )
>                 .withUsername(pgContainer.username)
>                 .withPassword(pgContainer.password)
>     }
>     @Test
>     fun readNumeric() {
>         val pipeline = TestPipeline.create().enableAbandonedNodeEnforcement(false)
>         val pBalance = pipeline
>             .apply(
>                 "Read Account from DB",
>                 JdbcIO
>                     .readRows()
>                     .withQuery("select balance from account")
>                     .withDataSourceConfiguration(getDataSourceConfiguration())
>             )
>             .apply(
>                 "Get Balance",
>                 MapElements.into(TypeDescriptors.bigdecimals()).via(ProcessFunction { it.getDecimal(0) })
>             )
>         PAssert.that(pBalance).containsInAnyOrder(BigDecimal("5755.94"))
>         pipeline.run()
>     }
> }
>  {code}
>  
> *Expected result*
> The test passes successfully.



--
This message was sent by Atlassian Jira
(v8.20.1#820001)