This is an automated email from the ASF dual-hosted git repository.

jark pushed a commit to branch release-1.10
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/release-1.10 by this push:
     new e99a929  [FLINK-15478][table-planner-blink] Fix FROM_BASE64 code 
generation exception because of the wrong result type (#10766)
e99a929 is described below

commit e99a929a9b9da7eb56fbf63fda7de386f27f4844
Author: Benchao Li <[email protected]>
AuthorDate: Tue Jan 7 15:13:35 2020 +0800

    [FLINK-15478][table-planner-blink] Fix FROM_BASE64 code generation 
exception because of the wrong result type (#10766)
---
 .../org/apache/flink/table/planner/codegen/calls/StringCallGen.scala  | 2 +-
 .../apache/flink/table/planner/expressions/ScalarFunctionsTest.scala  | 4 ++++
 .../org/apache/flink/table/runtime/functions/SqlFunctionUtils.java    | 4 ++--
 3 files changed, 7 insertions(+), 3 deletions(-)

diff --git 
a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/StringCallGen.scala
 
b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/StringCallGen.scala
index b12581b..0755b0e 100644
--- 
a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/StringCallGen.scala
+++ 
b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/StringCallGen.scala
@@ -636,7 +636,7 @@ object StringCallGen {
     ctx: CodeGeneratorContext,
     operands: Seq[GeneratedExpression]): GeneratedExpression = {
     val className = classOf[SqlFunctionUtils].getCanonicalName
-    generateCallIfArgsNotNull(ctx, new 
VarBinaryType(VarBinaryType.MAX_LENGTH), operands) {
+    generateCallIfArgsNotNull(ctx, new VarCharType(VarCharType.MAX_LENGTH), 
operands) {
       terms => s"$className.fromBase64(${terms.head})"
     }
   }
diff --git 
a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/ScalarFunctionsTest.scala
 
b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/ScalarFunctionsTest.scala
index 618eaa4..f4c6dd5 100644
--- 
a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/ScalarFunctionsTest.scala
+++ 
b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/ScalarFunctionsTest.scala
@@ -893,6 +893,10 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
       "FROM_BASE64('aGVsbG8gd29ybGQ=')",
       "hello world")
 
+    testSqlApi(
+      "CONCAT(FROM_BASE64('5L2g5aW9'), ' flink')",
+      "你好 flink")
+
     //null test
     testSqlApi(
       "FROM_BASE64(f33)",
diff --git 
a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/functions/SqlFunctionUtils.java
 
b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/functions/SqlFunctionUtils.java
index 30d944e..20c0c8f 100644
--- 
a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/functions/SqlFunctionUtils.java
+++ 
b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/functions/SqlFunctionUtils.java
@@ -1063,8 +1063,8 @@ public class SqlFunctionUtils {
                return Base64.getEncoder().encodeToString(bytes);
        }
 
-       public static byte[] fromBase64(BinaryString bs){
-               return Base64.getDecoder().decode(bs.getBytes());
+       public static BinaryString fromBase64(BinaryString bs) {
+               return 
BinaryString.fromBytes(Base64.getDecoder().decode(bs.getBytes()));
        }
 
        public static String uuid(){

Reply via email to