Uploaded image for project: 'Debezium'
  1. Debezium
  2. DBZ-4137

Error: PostgresDefaultValueConverter - Cannot parse column default value 'NULL::numeric' to type 'numeric'. Expression evaluation is not supported.

    XMLWordPrintable

Details

    • False
    • False
    • Hide

      I do not have a repro at this point; upgrade failed at the customer site and was reverted. 

      I see that default values handling was brought into 1.6 with https://github.com/debezium/debezium/commit/d2ab8baddf1deb85d83a4e5623e1992dbc16f029 which says "Using the `SqlServerDefaultValueConverter` class as a starting point" 
      Can this explain that postgres provides "NULL::numeric" as default while the code expects single quotes?

          private String extractDefault(String defaultValue) {        // Values are either "raw", such as `1234`, or "type casted", such as `'9223372036854775807'::bigint`.        // If the value does NOT contain a single quote it is assumed to be a raw value. Otherwise the value is        // extracted from inside the single quotes.
      

       

      Show
      I do not have a repro at this point; upgrade failed at the customer site and was reverted.  I see that default values handling was brought into 1.6 with https://github.com/debezium/debezium/commit/d2ab8baddf1deb85d83a4e5623e1992dbc16f029  which says "Using the `SqlServerDefaultValueConverter` class as a starting point"  Can this explain that postgres provides "NULL::numeric" as default while the code expects single quotes? private String extractDefault( String defaultValue) { // Values are either "raw" , such as `1234`, or "type casted" , such as ` '9223372036854775807' ::bigint`. // If the value does NOT contain a single quote it is assumed to be a raw value. Otherwise the value is // extracted from inside the single quotes.  

    Description

      Postgres connector failed after upgrade from 1.5.4 to 1.7.0.

      Relevant logs:

      io.debezium.relational.RelationalDatabaseSchema - refreshing DB schema for table 'public.table_name'2021-10-11 12:18:48 19:18:48.856  DEBUG io.debezium.relational.TableSchemaBuilder - Mapping table 'public.table_name' to schemas under 'console-1.public.table_name'2021-10-11 12:18:48 19:18:48.857  DEBUG io.debezium.relational.TableSchemaBuilder - - field 'user_targeting_id' (INT32) from column user_targeting_id int4(10, 0) NOT NULL2021-10-11 12:18:48 19:18:48.857  DEBUG io.debezium.relational.TableSchemaBuilder - - field 'ad_group_id' (INT32) from column ad_group_id int4(10, 0) NOT NULL2021-10-11 12:18:48 19:18:48.857  DEBUG io.debezium.relational.TableSchemaBuilder - - field 'type' (STRING) from column type varchar(50, 0) DEFAULT VALUE NULL2021-10-11 12:18:48 19:18:48.857  DEBUG io.debezium.relational.TableSchemaBuilder - - field 'amount' (FLOAT64) from column amount numeric(19, 4) DEFAULT VALUE NULL2021-10-11 12:18:48 19:18:48.857  DEBUG io.debezium.relational.TableSchemaBuilder - - field 'targeting' (STRING) from column targeting json(2147483647, 0) DEFAULT VALUE NULL2021-10-11 12:18:48 19:18:48.857  DEBUG io.debezium.relational.TableSchemaBuilder - - field 'last_edited_by' (INT32) from column last_edited_by int4(10, 0) DEFAULT VALUE NULL2021-10-11 12:18:48 19:18:48.857  DEBUG io.debezium.relational.TableSchemaBuilder - - field 'deleted' (BOOLEAN) from column deleted bool(1, 0) DEFAULT VALUE NULL2021-10-11 12:18:48 19:18:48.857  DEBUG io.debezium.relational.TableSchemaBuilder - - field 'last_edited_at' (INT64) from column last_edited_at timestamp(29, 6) DEFAULT VALUE NULL2021-10-11 12:18:48 19:18:48.857  DEBUG io.debezium.relational.TableSchemaBuilder - - field 'created_at' (INT64) from column created_at timestamp(29, 6) DEFAULT VALUE NULL2021-10-11 12:18:48 19:18:48.857  DEBUG io.debezium.relational.TableSchemaBuilder - - field 'created_by' (INT32) from column created_by int4(10, 0) DEFAULT VALUE NULL2021-10-11 12:18:48 19:18:48.857  DEBUG io.debezium.relational.TableSchemaBuilder - - field 'user_targeting_id' (INT32) from column user_targeting_id int4(10, 0) NOT NULL2021-10-11 12:18:48 19:18:48.857  DEBUG io.debezium.relational.TableSchemaBuilder - - field 'bucket_id' (INT32) from column bucket_id int4(10, 0) DEFAULT VALUE NULL2021-10-11 12:18:48 19:18:48.857  DEBUG io.debezium.relational.TableSchemaBuilder - - field 'legacy_id' (INT32) from column legacy_id int4(10, 0) DEFAULT VALUE NULL2021-10-11 12:18:48 19:18:48.857  DEBUG io.debezium.relational.TableSchemaBuilder - Mapped primary key for table 'public.table_name' to schema: {"name" : "console_1.public.table_name.Key", "type" : "STRUCT", "optional" : "false", "fields" : [{"name" : "user_targeting_id", "index" : "0", "schema" : {"type" : "INT32", "optional" : "false"}}]}2021-10-11 12:18:48 19:18:48.857  DEBUG io.debezium.relational.TableSchemaBuilder - Mapped columns for table 'public.table_name' to schema: {"name" : "console_1.public.table_name.Value", "type" : "STRUCT", "optional" : "true", "fields" : [{"name" : "ad_group_id", "index" : "0", "schema" : {"type" : "INT32", "optional" : "false"}}, {"name" : "type", "index" : "1", "schema" : {"type" : "STRING", "optional" : "true"}}, {"name" : "amount", "index" : "2", "schema" : {"type" : "FLOAT64", "optional" : "true"}}, {"name" : "targeting", "index" : "3", "schema" : {"name" : "io.debezium.data.Json", "type" : "STRING", "optional" : "true", "version" : "1"}}, {"name" : "last_edited_by", "index" : "4", "schema" : {"type" : "INT32", "optional" : "true"}}, {"name" : "deleted", "index" : "5", "schema" : {"type" : "BOOLEAN", "optional" : "true"}}, {"name" : "last_edited_at", "index" : "6", "schema" : {"name" : "io.debezium.time.MicroTimestamp", "type" : "INT64", "optional" : "true", "version" : "1"}}, {"name" : "created_at", "index" : "7", "schema" : {"name" : "io.debezium.time.MicroTimestamp", "type" : "INT64", "optional" : "true", "version" : "1"}}, {"name" : "created_by", "index" : "8", "schema" : {"type" : "INT32", "optional" : "true"}}, {"name" : "user_targeting_id", "index" : "9", "schema" : {"type" : "INT32", "optional" : "false"}}, {"name" : "bucket_id", "index" : "10", "schema" : {"type" : "INT32", "optional" : "true"}}, {"name" : "legacy_id", "index" : "11", "schema" : {"type" : "INT32", "optional" : "true"}}]}2021-10-11 12:18:48 19:18:48.857  DEBUG io.debezium.connector.postgresql.PostgresSchema - Relation '19108' resolved to table 'public.table_name'2021-10-11 12:18:48 19:18:48.857  DEBUG io.debezium.connector.base.ChangeEventQueue - Enqueuing source record 'DataChangeEvent [record=SourceRecord{sourcePartition={server=console-1}, sourceOffset={transaction_id=null, lsn_proc=157558998056, lsn_commit=157558997880, lsn=157558998056, txId=112429630, ts_usec=1633971790997756}} ConnectRecord{topic='console-1.public.table_name', kafkaPartition=null, key=Struct{user_targeting_id=54942180}, keySchema=Schema{console_1.public.table_name.Key:STRUCT}, value=Struct{before=Struct{ad_group_id=1252496,type=Exact,amount=0.25,targeting=[{"bookingWindow":21,"operator":"<="},{"checkInDayOfWeek":1,"operator":">"},{"checkInDayOfWeek":6,"operator":"<"}],last_edited_by=22053,deleted=true,last_edited_at=1633841752929855,created_at=1597774972690000,created_by=160,user_targeting_id=54942180,bucket_id=3,legacy_id=2699183},after=Struct{ad_group_id=1252496,type=Exact,amount=0.25,targeting=[{"bookingWindow":21,"operator":"<="},{"checkInDayOfWeek":1,"operator":">"},{"checkInDayOfWeek":6,"operator":"<"}],last_edited_by=22053,deleted=true,last_edited_at=1633971788143169,created_at=1597774972690000,created_by=160,user_targeting_id=54942180,bucket_id=3,legacy_id=2699183},source=Struct{version=1.7.0.Final,connector=postgresql,name=console-1,ts_ms=1633971790997,db=DBNAME,sequence=["157558997880","157558998056"],schema=public,table=table_name,txId=112429630,lsn=157558998056},op=u,ts_ms=1633979928857}, valueSchema=Schema{console_1.public.table_name.Envelope:STRUCT}, timestamp=null, headers=ConnectHeaders(headers=)}]'2021-10-11 
      2021-10-11 12:18:4819:18:48.860  WARN io.debezium.connector.postgresql.connection.PostgresDefaultValueConverter - Cannot parse column default value 'NULL::numeric' to type 'numeric'. Expression evaluation is not supported. 2021-10-11 12:18:4819:18:48.861  DEBUG io.debezium.connector.postgresql.connection.PostgresDefaultValueConverter - Parsing failed due to error 2021-10-11 12:18:48java.lang.NumberFormatException: Character N is neither a decimal digit number, decimal point, nor "e" notation exponential mark. 2021-10-11 12:18:48 at java.math.BigDecimal.<init>(BigDecimal.java:518) ~[?:?] 2021-10-11 12:18:48 at java.math.BigDecimal.<init>(BigDecimal.java:401) ~[?:?] 2021-10-11 12:18:48 at java.math.BigDecimal.<init>(BigDecimal.java:834) ~[?:?] 2021-10-11 12:18:48 at io.debezium.connector.postgresql.connection.PostgresDefaultValueConverter.lambda$createDefaultValueMappers$6(PostgresDefaultValueConverter.java:136) ~[debezium-connector-postgres-1.7.0.Final.jar:1.7.0.Final] 2021-10-11 12:18:48 at io.debezium.connector.postgresql.connection.PostgresDefaultValueConverter.parseDefaultValue(PostgresDefaultValueConverter.java:78) ~[debezium-connector-postgres-1.7.0.Final.jar:1.7.0.Final] 2021-10-11 12:18:48 at io.debezium.connector.postgresql.connection.PostgresConnection.getDefaultValue(PostgresConnection.java:583) ~[debezium-connector-postgres-1.7.0.Final.jar:1.7.0.Final] 2021-10-11 12:18:48 at io.debezium.connector.postgresql.connection.PostgresConnection.doReadTableColumn(PostgresConnection.java:572) ~[debezium-connector-postgres-1.7.0.Final.jar:1.7.0.Final] 2021-10-11 12:18:48 at io.debezium.connector.postgresql.connection.PostgresConnection.readColumnForDecoder(PostgresConnection.java:525) ~[debezium-connector-postgres-1.7.0.Final.jar:1.7.0.Final] 2021-10-11 12:18:48 at io.debezium.connector.postgresql.connection.pgoutput.PgOutputMessageDecoder.getTableColumnsFromDatabase(PgOutputMessageDecoder.java:335) ~[debezium-connector-postgres-1.7.0.Final.jar:1.7.0.Final] 2021-10-11 12:18:48 at io.debezium.connector.postgresql.connection.pgoutput.PgOutputMessageDecoder.handleRelationMessage(PgOutputMessageDecoder.java:272) ~[debezium-connector-postgres-1.7.0.Final.jar:1.7.0.Final] 2021-10-11 12:18:48 at io.debezium.connector.postgresql.connection.pgoutput.PgOutputMessageDecoder.processNotEmptyMessage(PgOutputMessageDecoder.java:176) ~[debezium-connector-postgres-1.7.0.Final.jar:1.7.0.Final] 2021-10-11 12:18:48 at io.debezium.connector.postgresql.connection.AbstractMessageDecoder.processMessage(AbstractMessageDecoder.java:33) ~[debezium-connector-postgres-1.7.0.Final.jar:1.7.0.Final] 2021-10-11 12:18:48 at io.debezium.connector.postgresql.connection.PostgresReplicationConnection$1.deserializeMessages(PostgresReplicationConnection.java:493) ~[debezium-connector-postgres-1.7.0.Final.jar:1.7.0.Final] 2021-10-11 12:18:48 at io.debezium.connector.postgresql.connection.PostgresReplicationConnection$1.readPending(PostgresReplicationConnection.java:485) ~[debezium-connector-postgres-1.7.0.Final.jar:1.7.0.Final] 2021-10-11 12:18:48 at io.debezium.connector.postgresql.PostgresStreamingChangeEventSource.processMessages(PostgresStreamingChangeEventSource.java:205) ~[debezium-connector-postgres-1.7.0.Final.jar:1.7.0.Final] 2021-10-11 12:18:48 at io.debezium.connector.postgresql.PostgresStreamingChangeEventSource.execute(PostgresStreamingChangeEventSource.java:167) ~[debezium-connector-postgres-1.7.0.Final.jar:1.7.0.Final] 2021-10-11 12:18:48 at io.debezium.connector.postgresql.PostgresStreamingChangeEventSource.execute(PostgresStreamingChangeEventSource.java:40) ~[debezium-connector-postgres-1.7.0.Final.jar:1.7.0.Final] 2021-10-11 12:18:48 at io.debezium.pipeline.ChangeEventSourceCoordinator.streamEvents(ChangeEventSourceCoordinator.java:166) ~[debezium-core-1.7.0.Final.jar:1.7.0.Final] 2021-10-11 12:18:48 at io.debezium.pipeline.ChangeEventSourceCoordinator.lambda$start$0(ChangeEventSourceCoordinator.java:127) ~[debezium-core-1.7.0.Final.jar:1.7.0.Final] 2021-10-11 12:18:48 at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515) ~[?:?] 2021-10-11 12:18:48 at java.util.concurrent.FutureTask.run(FutureTask.java:264) ~[?:?] 2021-10-11 12:18:48 at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) ~[?:?] 2021-10-11 12:18:48 at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) ~[?:?] 2021-10-11 12:18:48 at java.lang.Thread.run(Thread.java:829) ~[?:?]
      

      Later:

      2021-10-11 12:18:4819:18:48.864  ERROR io.debezium.pipeline.ErrorHandler - Producer failure
      2021-10-11 12:18:48org.apache.kafka.connect.errors.SchemaBuilderException: Invalid default value
      2021-10-11 12:18:48	at org.apache.kafka.connect.data.SchemaBuilder.defaultValue(SchemaBuilder.java:131) ~[connect-api-2.7.0.jar:?]
      2021-10-11 12:18:48	at io.debezium.relational.TableSchemaBuilder.addField(TableSchemaBuilder.java:374) ~[debezium-core-1.7.0.Final.jar:1.7.0.Final]
      2021-10-11 12:18:48	at io.debezium.relational.TableSchemaBuilder.lambda$create$2(TableSchemaBuilder.java:119) ~[debezium-core-1.7.0.Final.jar:1.7.0.Final]
      2021-10-11 12:18:48	at java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:183) ~[?:?]
      2021-10-11 12:18:48	at java.util.stream.ReferencePipeline$2$1.accept(ReferencePipeline.java:177) ~[?:?]
      2021-10-11 12:18:48	at java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1655) ~[?:?]
      2021-10-11 12:18:48	at java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:484) ~[?:?]
      2021-10-11 12:18:48	at java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:474) ~[?:?]
      2021-10-11 12:18:48	at java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:150) ~[?:?]
      2021-10-11 12:18:48	at java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:173) ~[?:?]
      2021-10-11 12:18:48	at java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234) ~[?:?]
      2021-10-11 12:18:48	at java.util.stream.ReferencePipeline.forEach(ReferencePipeline.java:497) ~[?:?]
      2021-10-11 12:18:48	at io.debezium.relational.TableSchemaBuilder.create(TableSchemaBuilder.java:117) ~[debezium-core-1.7.0.Final.jar:1.7.0.Final]
      2021-10-11 12:18:48	at io.debezium.relational.RelationalDatabaseSchema.buildAndRegisterSchema(RelationalDatabaseSchema.java:135) ~[debezium-core-1.7.0.Final.jar:1.7.0.Final]
      2021-10-11 12:18:48	at io.debezium.relational.RelationalDatabaseSchema.refreshSchema(RelationalDatabaseSchema.java:209) ~[debezium-core-1.7.0.Final.jar:1.7.0.Final]
      2021-10-11 12:18:48	at io.debezium.relational.RelationalDatabaseSchema.refresh(RelationalDatabaseSchema.java:200) ~[debezium-core-1.7.0.Final.jar:1.7.0.Final]
      2021-10-11 12:18:48	at io.debezium.connector.postgresql.PostgresSchema.applySchemaChangesForTable(PostgresSchema.java:232) ~[debezium-connector-postgres-1.7.0.Final.jar:1.7.0.Final]
      2021-10-11 12:18:48	at io.debezium.connector.postgresql.connection.pgoutput.PgOutputMessageDecoder.handleRelationMessage(PgOutputMessageDecoder.java:326) ~[debezium-connector-postgres-1.7.0.Final.jar:1.7.0.Final]
      2021-10-11 12:18:48	at io.debezium.connector.postgresql.connection.pgoutput.PgOutputMessageDecoder.processNotEmptyMessage(PgOutputMessageDecoder.java:176) ~[debezium-connector-postgres-1.7.0.Final.jar:1.7.0.Final]
      2021-10-11 12:18:48	at io.debezium.connector.postgresql.connection.AbstractMessageDecoder.processMessage(AbstractMessageDecoder.java:33) ~[debezium-connector-postgres-1.7.0.Final.jar:1.7.0.Final]
      2021-10-11 12:18:48	at io.debezium.connector.postgresql.connection.PostgresReplicationConnection$1.deserializeMessages(PostgresReplicationConnection.java:493) ~[debezium-connector-postgres-1.7.0.Final.jar:1.7.0.Final]
      2021-10-11 12:18:48	at io.debezium.connector.postgresql.connection.PostgresReplicationConnection$1.readPending(PostgresReplicationConnection.java:485) ~[debezium-connector-postgres-1.7.0.Final.jar:1.7.0.Final]
      2021-10-11 12:18:48	at io.debezium.connector.postgresql.PostgresStreamingChangeEventSource.processMessages(PostgresStreamingChangeEventSource.java:205) ~[debezium-connector-postgres-1.7.0.Final.jar:1.7.0.Final]
      2021-10-11 12:18:48	at io.debezium.connector.postgresql.PostgresStreamingChangeEventSource.execute(PostgresStreamingChangeEventSource.java:167) ~[debezium-connector-postgres-1.7.0.Final.jar:1.7.0.Final]
      2021-10-11 12:18:48	at io.debezium.connector.postgresql.PostgresStreamingChangeEventSource.execute(PostgresStreamingChangeEventSource.java:40) ~[debezium-connector-postgres-1.7.0.Final.jar:1.7.0.Final]
      2021-10-11 12:18:48	at io.debezium.pipeline.ChangeEventSourceCoordinator.streamEvents(ChangeEventSourceCoordinator.java:166) ~[debezium-core-1.7.0.Final.jar:1.7.0.Final]
      2021-10-11 12:18:48	at io.debezium.pipeline.ChangeEventSourceCoordinator.lambda$start$0(ChangeEventSourceCoordinator.java:127) ~[debezium-core-1.7.0.Final.jar:1.7.0.Final]
      2021-10-11 12:18:48	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515) ~[?:?]
      2021-10-11 12:18:48	at java.util.concurrent.FutureTask.run(FutureTask.java:264) ~[?:?]
      2021-10-11 12:18:48	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) ~[?:?]
      2021-10-11 12:18:48	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) ~[?:?]
      2021-10-11 12:18:48	at java.lang.Thread.run(Thread.java:829) ~[?:?]
      2021-10-11 12:18:48Caused by: org.apache.kafka.connect.errors.DataException: Invalid Java object for schema type FLOAT64: class java.math.BigDecimal for field: "null"
      2021-10-11 12:18:48	at org.apache.kafka.connect.data.ConnectSchema.validateValue(ConnectSchema.java:245) ~[connect-api-2.7.0.jar:?]
      2021-10-11 12:18:48	at org.apache.kafka.connect.data.ConnectSchema.validateValue(ConnectSchema.java:213) ~[connect-api-2.7.0.jar:?]
      2021-10-11 12:18:48	at org.apache.kafka.connect.data.SchemaBuilder.defaultValue(SchemaBuilder.java:129) ~[connect-api-2.7.0.jar:?]
      2021-10-11 12:18:48	... 31 more
      

      and 

      2021-10-11 12:18:49org.apache.kafka.connect.errors.ConnectException: An exception occurred in the change event producer. This connector will be stopped.
      2021-10-11 12:18:49	at io.debezium.pipeline.ErrorHandler.setProducerThrowable(ErrorHandler.java:42) ~[debezium-core-1.7.0.Final.jar:1.7.0.Final]
      2021-10-11 12:18:49	at io.debezium.connector.postgresql.PostgresStreamingChangeEventSource.execute(PostgresStreamingChangeEventSource.java:170) ~[debezium-connector-postgres-1.7.0.Final.jar:1.7.0.Final]
      2021-10-11 12:18:49	at io.debezium.connector.postgresql.PostgresStreamingChangeEventSource.execute(PostgresStreamingChangeEventSource.java:40) ~[debezium-connector-postgres-1.7.0.Final.jar:1.7.0.Final]
      2021-10-11 12:18:49	at io.debezium.pipeline.ChangeEventSourceCoordinator.streamEvents(ChangeEventSourceCoordinator.java:166) ~[debezium-core-1.7.0.Final.jar:1.7.0.Final]
      2021-10-11 12:18:49	at io.debezium.pipeline.ChangeEventSourceCoordinator.lambda$start$0(ChangeEventSourceCoordinator.java:127) ~[debezium-core-1.7.0.Final.jar:1.7.0.Final]
      2021-10-11 12:18:49	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515) ~[?:?]
      2021-10-11 12:18:49	at java.util.concurrent.FutureTask.run(FutureTask.java:264) ~[?:?]
      2021-10-11 12:18:49	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) ~[?:?]
      2021-10-11 12:18:49	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) ~[?:?]
      2021-10-11 12:18:49	at java.lang.Thread.run(Thread.java:829) ~[?:?]
      2021-10-11 12:18:49Caused by: org.apache.kafka.connect.errors.SchemaBuilderException: Invalid default value
      2021-10-11 12:18:49	at org.apache.kafka.connect.data.SchemaBuilder.defaultValue(SchemaBuilder.java:131) ~[connect-api-2.7.0.jar:?]
      2021-10-11 12:18:49	at io.debezium.relational.TableSchemaBuilder.addField(TableSchemaBuilder.java:374) ~[debezium-core-1.7.0.Final.jar:1.7.0.Final]
      2021-10-11 12:18:49	at io.debezium.relational.TableSchemaBuilder.lambda$create$2(TableSchemaBuilder.java:119) ~[debezium-core-1.7.0.Final.jar:1.7.0.Final]
      2021-10-11 12:18:49	at java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:183) ~[?:?]
      2021-10-11 12:18:49	at java.util.stream.ReferencePipeline$2$1.accept(ReferencePipeline.java:177) ~[?:?]
      2021-10-11 12:18:49	at java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1655) ~[?:?]
      2021-10-11 12:18:49	at java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:484) ~[?:?]
      2021-10-11 12:18:49	at java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:474) ~[?:?]
      2021-10-11 12:18:49	at java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:150) ~[?:?]
      2021-10-11 12:18:49	at java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:173) ~[?:?]
      2021-10-11 12:18:49	at java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234) ~[?:?]
      2021-10-11 12:18:49	at java.util.stream.ReferencePipeline.forEach(ReferencePipeline.java:497) ~[?:?]
      2021-10-11 12:18:49	at io.debezium.relational.TableSchemaBuilder.create(TableSchemaBuilder.java:117) ~[debezium-core-1.7.0.Final.jar:1.7.0.Final]
      2021-10-11 12:18:49	at io.debezium.relational.RelationalDatabaseSchema.buildAndRegisterSchema(RelationalDatabaseSchema.java:135) ~[debezium-core-1.7.0.Final.jar:1.7.0.Final]
      2021-10-11 12:18:49	at io.debezium.relational.RelationalDatabaseSchema.refreshSchema(RelationalDatabaseSchema.java:209) ~[debezium-core-1.7.0.Final.jar:1.7.0.Final]
      2021-10-11 12:18:49	at io.debezium.relational.RelationalDatabaseSchema.refresh(RelationalDatabaseSchema.java:200) ~[debezium-core-1.7.0.Final.jar:1.7.0.Final]
      2021-10-11 12:18:49	at io.debezium.connector.postgresql.PostgresSchema.applySchemaChangesForTable(PostgresSchema.java:232) ~[debezium-connector-postgres-1.7.0.Final.jar:1.7.0.Final]
      2021-10-11 12:18:49	at io.debezium.connector.postgresql.connection.pgoutput.PgOutputMessageDecoder.handleRelationMessage(PgOutputMessageDecoder.java:326) ~[debezium-connector-postgres-1.7.0.Final.jar:1.7.0.Final]
      2021-10-11 12:18:49	at io.debezium.connector.postgresql.connection.pgoutput.PgOutputMessageDecoder.processNotEmptyMessage(PgOutputMessageDecoder.java:176) ~[debezium-connector-postgres-1.7.0.Final.jar:1.7.0.Final]
      2021-10-11 12:18:49	at io.debezium.connector.postgresql.connection.AbstractMessageDecoder.processMessage(AbstractMessageDecoder.java:33) ~[debezium-connector-postgres-1.7.0.Final.jar:1.7.0.Final]
      2021-10-11 12:18:49	at io.debezium.connector.postgresql.connection.PostgresReplicationConnection$1.deserializeMessages(PostgresReplicationConnection.java:493) ~[debezium-connector-postgres-1.7.0.Final.jar:1.7.0.Final]
      2021-10-11 12:18:49	at io.debezium.connector.postgresql.connection.PostgresReplicationConnection$1.readPending(PostgresReplicationConnection.java:485) ~[debezium-connector-postgres-1.7.0.Final.jar:1.7.0.Final]
      2021-10-11 12:18:49	at io.debezium.connector.postgresql.PostgresStreamingChangeEventSource.processMessages(PostgresStreamingChangeEventSource.java:205) ~[debezium-connector-postgres-1.7.0.Final.jar:1.7.0.Final]
      2021-10-11 12:18:49	at io.debezium.connector.postgresql.PostgresStreamingChangeEventSource.execute(PostgresStreamingChangeEventSource.java:167) ~[debezium-connector-postgres-1.7.0.Final.jar:1.7.0.Final]
      2021-10-11 12:18:49	... 8 more
      2021-10-11 12:18:49Caused by: org.apache.kafka.connect.errors.DataException: Invalid Java object for schema type FLOAT64: class java.math.BigDecimal for field: "null"
      2021-10-11 12:18:49	at org.apache.kafka.connect.data.ConnectSchema.validateValue(ConnectSchema.java:245) ~[connect-api-2.7.0.jar:?]
      2021-10-11 12:18:49	at org.apache.kafka.connect.data.ConnectSchema.validateValue(ConnectSchema.java:213) ~[connect-api-2.7.0.jar:?]
      2021-10-11 12:18:49	at org.apache.kafka.connect.data.SchemaBuilder.defaultValue(SchemaBuilder.java:129) ~[connect-api-2.7.0.jar:?]
      2021-10-11 12:18:49	at io.debezium.relational.TableSchemaBuilder.addField(TableSchemaBuilder.java:374) ~[debezium-core-1.7.0.Final.jar:1.7.0.Final]
      2021-10-11 12:18:49	at io.debezium.relational.TableSchemaBuilder.lambda$create$2(TableSchemaBuilder.java:119) ~[debezium-core-1.7.0.Final.jar:1.7.0.Final]
      2021-10-11 12:18:49	at java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:183) ~[?:?]
      2021-10-11 12:18:49	at java.util.stream.ReferencePipeline$2$1.accept(ReferencePipeline.java:177) ~[?:?]
      2021-10-11 12:18:49	at java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1655) ~[?:?]
      2021-10-11 12:18:49	at java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:484) ~[?:?]
      2021-10-11 12:18:49	at java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:474) ~[?:?]
      2021-10-11 12:18:49	at java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:150) ~[?:?]
      2021-10-11 12:18:49	at java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:173) ~[?:?]
      2021-10-11 12:18:49	at java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234) ~[?:?]
      2021-10-11 12:18:49	at java.util.stream.ReferencePipeline.forEach(ReferencePipeline.java:497) ~[?:?]
      2021-10-11 12:18:49	at io.debezium.relational.TableSchemaBuilder.create(TableSchemaBuilder.java:117) ~[debezium-core-1.7.0.Final.jar:1.7.0.Final]
      2021-10-11 12:18:49	at io.debezium.relational.RelationalDatabaseSchema.buildAndRegisterSchema(RelationalDatabaseSchema.java:135) ~[debezium-core-1.7.0.Final.jar:1.7.0.Final]
      2021-10-11 12:18:49	at io.debezium.relational.RelationalDatabaseSchema.refreshSchema(RelationalDatabaseSchema.java:209) ~[debezium-core-1.7.0.Final.jar:1.7.0.Final]
      2021-10-11 12:18:49	at io.debezium.relational.RelationalDatabaseSchema.refresh(RelationalDatabaseSchema.java:200) ~[debezium-core-1.7.0.Final.jar:1.7.0.Final]
      2021-10-11 12:18:49	at io.debezium.connector.postgresql.PostgresSchema.applySchemaChangesForTable(PostgresSchema.java:232) ~[debezium-connector-postgres-1.7.0.Final.jar:1.7.0.Final]
      2021-10-11 12:18:49	at io.debezium.connector.postgresql.connection.pgoutput.PgOutputMessageDecoder.handleRelationMessage(PgOutputMessageDecoder.java:326) ~[debezium-connector-postgres-1.7.0.Final.jar:1.7.0.Final]
      2021-10-11 12:18:49	at io.debezium.connector.postgresql.connection.pgoutput.PgOutputMessageDecoder.processNotEmptyMessage(PgOutputMessageDecoder.java:176) ~[debezium-connector-postgres-1.7.0.Final.jar:1.7.0.Final]
      2021-10-11 12:18:49	at io.debezium.connector.postgresql.connection.AbstractMessageDecoder.processMessage(AbstractMessageDecoder.java:33) ~[debezium-connector-postgres-1.7.0.Final.jar:1.7.0.Final]
      2021-10-11 12:18:49	at io.debezium.connector.postgresql.connection.PostgresReplicationConnection$1.deserializeMessages(PostgresReplicationConnection.java:493) ~[debezium-connector-postgres-1.7.0.Final.jar:1.7.0.Final]
      2021-10-11 12:18:49	at io.debezium.connector.postgresql.connection.PostgresReplicationConnection$1.readPending(PostgresReplicationConnection.java:485) ~[debezium-connector-postgres-1.7.0.Final.jar:1.7.0.Final]
      2021-10-11 12:18:49	at io.debezium.connector.postgresql.PostgresStreamingChangeEventSource.processMessages(PostgresStreamingChangeEventSource.java:205) ~[debezium-connector-postgres-1.7.0.Final.jar:1.7.0.Final]
      2021-10-11 12:18:49	at io.debezium.connector.postgresql.PostgresStreamingChangeEventSource.execute(PostgresStreamingChangeEventSource.java:167) ~[debezium-connector-postgres-1.7.0.Final.jar:1.7.0.Final]
      2021-10-11 12:18:49	... 8 more
      
      

      Attachments

        Activity

          People

            Unassigned Unassigned
            dlg99 Andrey Yegorov (Inactive)
            Votes:
            0 Vote for this issue
            Watchers:
            6 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: