[
https://issues.apache.org/jira/browse/FLINK-35916?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Evgeniy updated FLINK-35916:
----------------------------
Issue Type: Bug (was: Improvement)
Priority: Critical (was: Major)
Summary: AbstractJdbcRowConverter: type conversion to string
(ClassCastException) (was: AbstractJdbcRowConverter: type conversion to string)
> AbstractJdbcRowConverter: type conversion to string (ClassCastException)
> ------------------------------------------------------------------------
>
> Key: FLINK-35916
> URL: https://issues.apache.org/jira/browse/FLINK-35916
> Project: Flink
> Issue Type: Bug
> Components: Connectors / JDBC
> Affects Versions: jdbc-3.2.0
> Reporter: Evgeniy
> Priority: Critical
> Fix For: jdbc-3.2.0, jdbc-3.1.2
>
> Original Estimate: 1m
> Remaining Estimate: 1m
>
> *Problem:* When converting a type to a string, Java types are explicitly
> cast, which generates a ClassCastException.
>
> {*}Effects{*}: Most Java types, one way or another, have the ability to be
> converted to a String type. However, not all types with this capability
> inherit a class of type String or similar, where the type can be cast to type
> String. As java suggests, any object can be cast to the String:
> Object#toString type if its implementation does not imply the opposite.
> However, explicit casting to the String type makes this impossible for 99% of
> custom Java objects.
>
> {*}Goals{*}: Calling the Object#toString method for all objects that are
> supposed to be converted to a string.
>
>
> {*}Solution{*}:
> To begin with, I suggest you pay attention to the class that deals with type
> conversion - AbstractJdbcRowConverter. Pay attention to the method that deals
> with type conversion - createInternalConverter, and specifically to the
> lambda expression that deals with converting a type to a String type (CHAR,
> VARCHAR).
> {code:java}
> switch (type.getTypeRoot()) {
> ...
> case CHAR:
> case VARCHAR:
> return val -> StringData.fromString((String) val);
> ...
> } {code}
> You can observe (String) val, which is problematic.
> My intended solution is the following change.
> {code:java}
> return val -> StringData.fromString(val.toString()); {code}
> val.toString() can give a chance for a new life to most objects that cannot
> be processed by Flink by default, and this is very cool because it solves at
> least 1 problem.
>
>
> {*}For further research{*}:
> For example, pay attention to the type that is used in the podtgres driver to
> package PostgreSQL types other than primitive ones - PGobject.
> {code:java}
> public class PGobject implements Serializable, Cloneable {
> protected @Nullable String type;
> protected @Nullable String value;
> ...
> ...
> public String toString() {
> return this.getValue();
> }
> ...
> } {code}
> As you may have noticed, PostgreSQL itself is engaged in converting types to
> a string, which is an excellent argument for using val.toString().
> Perhaps in the future, you can add PGobject type processing, as was done for
> PGarray, but it seems to me that the solution described above is the fastest
> and most optimal. Thanks!
--
This message was sent by Atlassian Jira
(v8.20.10#820010)