yuxiqian commented on code in PR #3651: URL: https://github.com/apache/flink-cdc/pull/3651#discussion_r1837584244
########## flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/SchemaOperator.java: ########## @@ -611,14 +614,42 @@ public Object getFieldOrNull(RecordData recordData) { } else if (originalField instanceof Integer) { // INT return ((Integer) originalField).longValue(); + } else if (originalField instanceof Long) { + // BIGINT + return originalField; } else { return fail( new IllegalArgumentException( String.format( "Cannot fit type \"%s\" into a BIGINT column. " - + "Currently only TINYINT / SMALLINT / INT can be accepted by a BIGINT column", + + "Currently only TINYINT / SMALLINT / INT / LONG can be accepted by a BIGINT column", + originalField.getClass()))); + } + } else if (destinationType instanceof DecimalType) { + DecimalType decimalType = (DecimalType) destinationType; + BigDecimal decimalValue; + if (originalField instanceof Byte) { + decimalValue = BigDecimal.valueOf(((Byte) originalField).longValue(), 0); + } else if (originalField instanceof Short) { + decimalValue = BigDecimal.valueOf(((Short) originalField).longValue(), 0); + } else if (originalField instanceof Integer) { + decimalValue = BigDecimal.valueOf(((Integer) originalField).longValue(), 0); + } else if (originalField instanceof Long) { + decimalValue = BigDecimal.valueOf((Long) originalField, 0); + } else if (originalField instanceof DecimalData) { + decimalValue = ((DecimalData) originalField).toBigDecimal(); + } else { + return fail( + new IllegalArgumentException( + String.format( + "Cannot fit type \"%s\" into a DECIMAL column. " + + "Currently only BYTE / SHORT / INT / LONG / DECIMAL can be accepted by a DECIMAL column", Review Comment: Unfortunately, it's not possible to determine a "wide enough" Decimal type to store FLOAT / DOUBLE losslessly (since floating point numbers' expression range doesn't have an uplimit -- it could be up to `Infinity`), so maybe it's a not good idea to let this type conversion happen implicitly. Explicit `CAST`ing might be better, which was already implemented in FLINK-34877. -- 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: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org