[ 
https://issues.apache.org/jira/browse/FLINK-27212?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17522183#comment-17522183
 ] 

Jark Wu commented on FLINK-27212:
---------------------------------

>From my point of view, casting string to binary should use the UTF8 encoding. 

It seems PG will inspect the format of the character string. If it is hex 
representation, PG will decode the string into bytes using hex encoding. 
Otherwise, PG will decode the string using UTF8 encoding. 

{code}
postgres=# create table t (a varchar(20));
CREATE TABLE
postgres=#
postgres=# insert into t values ('\x00011c57'), ('00011c57');
INSERT 0 2
postgres=#
postgres=# select a, pg_typeof(a), a::bytea from t;
     a      |     pg_typeof     |         a
------------+-------------------+--------------------
 \x00011c57 | character varying | \x00011c57
 00011c57   | character varying | \x3030303131633537
(2 rows)
{code}

> Failed to CAST('abcde', VARBINARY)
> ----------------------------------
>
>                 Key: FLINK-27212
>                 URL: https://issues.apache.org/jira/browse/FLINK-27212
>             Project: Flink
>          Issue Type: Bug
>          Components: Table SQL / Runtime
>    Affects Versions: 1.16.0
>            Reporter: Shengkai Fang
>            Assignee: Marios Trivyzas
>            Priority: Blocker
>
> Please add test in the CalcITCase
> {code:scala}
> @Test
>   def testCalc(): Unit = {
>     val sql =
>       """
>         |SELECT CAST('abcde' AS VARBINARY(6))
>         |""".stripMargin
>     val result = tEnv.executeSql(sql)
>     print(result.getResolvedSchema)
>     result.print()
>   }
> {code}
> The exception is 
> {code:java}
> Caused by: org.apache.flink.table.api.TableException: Odd number of 
> characters.
>       at 
> org.apache.flink.table.utils.EncodingUtils.decodeHex(EncodingUtils.java:203)
>       at StreamExecCalc$33.processElement(Unknown Source)
>       at 
> org.apache.flink.streaming.runtime.tasks.ChainingOutput.pushToOperator(ChainingOutput.java:99)
>       at 
> org.apache.flink.streaming.runtime.tasks.ChainingOutput.collect(ChainingOutput.java:80)
>       at 
> org.apache.flink.streaming.runtime.tasks.ChainingOutput.collect(ChainingOutput.java:39)
>       at 
> org.apache.flink.streaming.api.operators.CountingOutput.collect(CountingOutput.java:56)
>       at 
> org.apache.flink.streaming.api.operators.CountingOutput.collect(CountingOutput.java:29)
>       at 
> org.apache.flink.streaming.api.operators.StreamSourceContexts$ManualWatermarkContext.processAndCollect(StreamSourceContexts.java:418)
>       at 
> org.apache.flink.streaming.api.operators.StreamSourceContexts$WatermarkContext.collect(StreamSourceContexts.java:513)
>       at 
> org.apache.flink.streaming.api.operators.StreamSourceContexts$SwitchingOnClose.collect(StreamSourceContexts.java:103)
>       at 
> org.apache.flink.streaming.api.functions.source.InputFormatSourceFunction.run(InputFormatSourceFunction.java:92)
>       at 
> org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:110)
>       at 
> org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:67)
>       at 
> org.apache.flink.streaming.runtime.tasks.SourceStreamTask$LegacySourceFunctionThread.run(SourceStreamTask.java:332)
> {code}



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

Reply via email to