Closed MarekMaj closed 2 weeks ago
Hi @MarekMaj
In the old version of ClickHouse, the Bool
type is just an alias for Int8
. I wrote this code to match with Flink's Boolean
, but the latest ClickHouse version is different.
A Test using ClickHouse 21.xx
Hi @itinycheng, After your comment I realized that newer versions of clickhouse server won't process booleans correctly also in other use cases. To confirm that I run below example using clickhouse version 23.3.8.22
CREATE TABLE test.lookup ON CLUSTER 'cluster' (
id BIGINT,
flag Boolean
)
ENGINE = ReplicatedReplacingMergeTree
ORDER BY (id)
desc test.lookup
id | Int64 | | | | |
flag | Bool | | | |
When using that table in flink SQL as lookup table:
CREATE TABLE LOOKUP_TEST (id BIGINT, flag Boolean) WITH (
'connector' = 'clickhouse',
'database-name' = 'test',
'table-name' = 'lookup',
'url' = 'clickhouse://${CLICKHOUSE_SERVER}',
'password' = '${CLICKHOUSE_PASSWORD}',
'username' = '${CLICKHOUSE_USER}'
);
SELECT * FROM INPUT i
LEFT JOIN LOOKUP_TEST FOR SYSTEM_TIME AS OF i.proc_time AS l
ON i.JOIN_KEY = l.id
The error occurs:
java.util.concurrent.ExecutionException: java.lang.RuntimeException: Failed to fetch next result
(..)
Caused by: org.apache.flink.shaded.clickhouse.ru.yandex.clickhouse.except.ClickHouseUnknownException: ClickHouse exception, message: Parse exception: ByteFragment{[1<TAB>true], start=2, len=4}; ClickHouse exception, message: Error parsing 'true' as java.lang.Integer; For input string: "true"
at org.apache.flink.shaded.clickhouse.ru.yandex.clickhouse.response.ClickHouseResultSet.getObject(ClickHouseResultSet.java:648)
at org.apache.flink.connector.clickhouse.internal.converter.ClickHouseRowConverter.toInternal(ClickHouseRowConverter.java:62)
at org.apache.flink.connector.clickhouse.internal.ClickHouseRowDataLookupFunction.lookup(ClickHouseRowDataLookupFunction.java:108)
... 39 more
Caused by: org.apache.flink.shaded.clickhouse.ru.yandex.clickhouse.except.ClickHouseUnknownException: ClickHouse exception, message: Error parsing 'true' as java.lang.Integer; For input string: "true"
at org.apache.flink.shaded.clickhouse.ru.yandex.clickhouse.response.parser.ClickHouseValueParser$ClickHouseValueParserFunctionWrapper.parse(ClickHouseValueParser.java:243)
at org.apache.flink.shaded.clickhouse.ru.yandex.clickhouse.response.ClickHouseResultSet.getObject(ClickHouseResultSet.java:749)
at org.apache.flink.shaded.clickhouse.ru.yandex.clickhouse.response.ClickHouseResultSet.getObject(ClickHouseResultSet.java:603)
... 41 more
Caused by: java.lang.NumberFormatException: For input string: "true"
at java.base/java.lang.NumberFormatException.forInputString(NumberFormatException.java:65)
at java.base/java.lang.Integer.parseInt(Integer.java:652)
at java.base/java.lang.Integer.valueOf(Integer.java:957)
at java.base/java.lang.Integer.decode(Integer.java:1436)
at org.apache.flink.shaded.clickhouse.ru.yandex.clickhouse.response.parser.ClickHouseValueParser$ClickHouseValueParserFunctionWrapper.parse(ClickHouseValueParser.java:240)
... 43 more
Thank you for the explanation. Are there any plans for adding support for newer versions of clickhouse server?
Thank you
Hi @MarekMaj
Happy New Year.
I want to add a type check the same as obj instanceof Boolean
before performing the type cast, maybe a little later.
Hi @czy006 Any suggestions?
Hi @itinycheng, After your comment I realized that newer versions of clickhouse server won't process booleans correctly also in other use cases. To confirm that I run below example using clickhouse version 23.3.8.22
CREATE TABLE test.lookup ON CLUSTER 'cluster' ( id BIGINT, flag Boolean ) ENGINE = ReplicatedReplacingMergeTree ORDER BY (id)
desc test.lookup id | Int64 | | | | | flag | Bool | | | |
When using that table in flink SQL as lookup table:
CREATE TABLE LOOKUP_TEST (id BIGINT, flag Boolean) WITH ( 'connector' = 'clickhouse', 'database-name' = 'test', 'table-name' = 'lookup', 'url' = 'clickhouse://${CLICKHOUSE_SERVER}', 'password' = '${CLICKHOUSE_PASSWORD}', 'username' = '${CLICKHOUSE_USER}' ); SELECT * FROM INPUT i LEFT JOIN LOOKUP_TEST FOR SYSTEM_TIME AS OF i.proc_time AS l ON i.JOIN_KEY = l.id
The error occurs:
java.util.concurrent.ExecutionException: java.lang.RuntimeException: Failed to fetch next result (..) Caused by: org.apache.flink.shaded.clickhouse.ru.yandex.clickhouse.except.ClickHouseUnknownException: ClickHouse exception, message: Parse exception: ByteFragment{[1<TAB>true], start=2, len=4}; ClickHouse exception, message: Error parsing 'true' as java.lang.Integer; For input string: "true" at org.apache.flink.shaded.clickhouse.ru.yandex.clickhouse.response.ClickHouseResultSet.getObject(ClickHouseResultSet.java:648) at org.apache.flink.connector.clickhouse.internal.converter.ClickHouseRowConverter.toInternal(ClickHouseRowConverter.java:62) at org.apache.flink.connector.clickhouse.internal.ClickHouseRowDataLookupFunction.lookup(ClickHouseRowDataLookupFunction.java:108) ... 39 more Caused by: org.apache.flink.shaded.clickhouse.ru.yandex.clickhouse.except.ClickHouseUnknownException: ClickHouse exception, message: Error parsing 'true' as java.lang.Integer; For input string: "true" at org.apache.flink.shaded.clickhouse.ru.yandex.clickhouse.response.parser.ClickHouseValueParser$ClickHouseValueParserFunctionWrapper.parse(ClickHouseValueParser.java:243) at org.apache.flink.shaded.clickhouse.ru.yandex.clickhouse.response.ClickHouseResultSet.getObject(ClickHouseResultSet.java:749) at org.apache.flink.shaded.clickhouse.ru.yandex.clickhouse.response.ClickHouseResultSet.getObject(ClickHouseResultSet.java:603) ... 41 more Caused by: java.lang.NumberFormatException: For input string: "true" at java.base/java.lang.NumberFormatException.forInputString(NumberFormatException.java:65) at java.base/java.lang.Integer.parseInt(Integer.java:652) at java.base/java.lang.Integer.valueOf(Integer.java:957) at java.base/java.lang.Integer.decode(Integer.java:1436) at org.apache.flink.shaded.clickhouse.ru.yandex.clickhouse.response.parser.ClickHouseValueParser$ClickHouseValueParserFunctionWrapper.parse(ClickHouseValueParser.java:240) ... 43 more
Thank you for the explanation. Are there any plans for adding support for newer versions of clickhouse server?
Thank you
Hi @MarekMaj
I had misunderstood this error before.
This error is raised by clickhouse-jdbc 0.3.1
, which version doesn't support the boolean type.
The clickhouse-jdbc 0.3.1
treats the boolean type as an integer, and uses Integer::decode
to deserialize "true"
, you can get more info from ClickHouseColumnInfo
, ClickHouseDataType
and ClickHouseValueParser
.
A newer version of clickhouse-jdbc
supports boolean type but it has many breaking changes, and more effort is needed to update the version, I don't have time to do it for now.
You can add boolean type support to clickhouse-jdbc 0.3.1
, it only needs a few changes in ClickHouseDataType
, and it's easier.
The master branch has upgraded the version of clickhouse-jdbc
to 0.6.0, this issue should be solved.
If there are no more questions here, I will close this issue in a week.
cc @czy006
@MarekMaj new version fixed it
Hi All, We are leveraging flink-connector-clickhouse to read and write data to clickhouse from custom written DataStream Api KeyedCoProcessFunction. Implementation uses AbstractClickHouseOutputFormat builder to initialize RichOutputFormat from inside custom function.
It works well with most of the types (long, int, BigDecimal). However recently we encountered unexpected ClassCastException when trying to write to clickhouse a row with a column of Boolean type:
When I follow the stacktrace and look at the source I can find this piece of code:
My question is what is a reasoning behind that casting to Number? I can see similar behavior for Short and Byte types where there is a cast to Integer prior to value extraction
Thank you