You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2021/10/11 11:35:41 UTC

[GitHub] [flink] matriv opened a new pull request #17450: [FLINK-24418][table-planner] Add support for casting RAW to BINARY

matriv opened a new pull request #17450:
URL: https://github.com/apache/flink/pull/17450


   Implement casting from generic RAW type into BINARY/VARBINARY/BYTES.
   
   ## What is the purpose of the change
   
   Add support for explicit casting from `RAW(<class>)` to BINARY type, so
   basically convert to a `byte[]`.
   
   
   ## Brief change log
   
     - Enable the cast in `LogicalTypeCasts`
     - Add conversion in `ScalarOperatorGens`
   
   ## Verifying this change
   
   This change added tests and can be verified as follows:
   
     - Test validity of conversion in LogicalTypeCastsTest
     - Integration tests added to CastFunctionITCase
    
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): no
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: no
     - The serializers: no
     - The runtime per-record code paths (performance sensitive): no
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn, ZooKeeper: no
     - The S3 file system connector: no
   
   ## Documentation
   
     - Does this pull request introduce a new feature? no
     - If yes, how is the feature documented? not applicable
   


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] matriv commented on a change in pull request #17450: [FLINK-24418][table-planner] Add support for casting RAW to BINARY

Posted by GitBox <gi...@apache.org>.
matriv commented on a change in pull request #17450:
URL: https://github.com/apache/flink/pull/17450#discussion_r727126762



##########
File path: flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/utils/LogicalTypeCasts.java
##########
@@ -401,6 +405,8 @@ private static boolean supportsConstructedCasting(
                 }
             }
             return true;
+        } else if (sourceRoot == ARRAY && (targetRoot == CHAR || targetRoot == VARCHAR)) {

Review comment:
       Ah, thx, just a leftover for some playing around with casting to ARRAY().




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #17450: [FLINK-24418][table-planner] Add support for casting RAW to BINARY

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #17450:
URL: https://github.com/apache/flink/pull/17450#issuecomment-939949291


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "a27b45f90f31067656d64a3ec0a9d20d0c60402e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=24950",
       "triggerID" : "a27b45f90f31067656d64a3ec0a9d20d0c60402e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "459111838bfa4cd32499535d6d89636f4aa8a024",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=24952",
       "triggerID" : "459111838bfa4cd32499535d6d89636f4aa8a024",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e17cd221fe36ee6064cfff125b0a9b025c5310bd",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=24989",
       "triggerID" : "e17cd221fe36ee6064cfff125b0a9b025c5310bd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e17cd221fe36ee6064cfff125b0a9b025c5310bd Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=24989) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #17450: [FLINK-24418][table-planner] Add support for casting RAW to BINARY

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #17450:
URL: https://github.com/apache/flink/pull/17450#issuecomment-939949291


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "a27b45f90f31067656d64a3ec0a9d20d0c60402e",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=24950",
       "triggerID" : "a27b45f90f31067656d64a3ec0a9d20d0c60402e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a27b45f90f31067656d64a3ec0a9d20d0c60402e Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=24950) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] matriv commented on a change in pull request #17450: [FLINK-24418][table-planner] Add support for casting RAW to BINARY

Posted by GitBox <gi...@apache.org>.
matriv commented on a change in pull request #17450:
URL: https://github.com/apache/flink/pull/17450#discussion_r727132320



##########
File path: flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala
##########
@@ -40,15 +40,14 @@ import org.apache.flink.table.types.logical.utils.LogicalTypeCasts.supportsExpli
 import org.apache.flink.table.types.logical.utils.LogicalTypeChecks.getFieldTypes
 import org.apache.flink.table.types.logical.utils.LogicalTypeMerging.findCommonType
 import org.apache.flink.util.Preconditions.checkArgument
-
 import org.apache.calcite.avatica.util.DateTimeUtils.MILLIS_PER_DAY
 import org.apache.calcite.avatica.util.{DateTimeUtils, TimeUnitRange}
 import org.apache.calcite.util.BuiltInMethod
+import org.apache.flink.table.api.DataTypes.BYTES
 
 import java.lang.{StringBuilder => JStringBuilder}
 import java.nio.charset.StandardCharsets
 import java.util.Arrays.asList
-

Review comment:
       I will, Indeed this is automatically done by IntelliJ, need to fix.




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #17450: [FLINK-24418][table-planner] Add support for casting RAW to BINARY

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #17450:
URL: https://github.com/apache/flink/pull/17450#issuecomment-939949291


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "a27b45f90f31067656d64a3ec0a9d20d0c60402e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=24950",
       "triggerID" : "a27b45f90f31067656d64a3ec0a9d20d0c60402e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "459111838bfa4cd32499535d6d89636f4aa8a024",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=24952",
       "triggerID" : "459111838bfa4cd32499535d6d89636f4aa8a024",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e17cd221fe36ee6064cfff125b0a9b025c5310bd",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=24989",
       "triggerID" : "e17cd221fe36ee6064cfff125b0a9b025c5310bd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e17cd221fe36ee6064cfff125b0a9b025c5310bd Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=24989) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] twalthr commented on a change in pull request #17450: [FLINK-24418][table-planner] Add support for casting RAW to BINARY

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #17450:
URL: https://github.com/apache/flink/pull/17450#discussion_r727097560



##########
File path: flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/utils/LogicalTypeCasts.java
##########
@@ -319,7 +322,8 @@ private static boolean supportsCasting(
         } else if (sourceRoot == STRUCTURED_TYPE || targetRoot == STRUCTURED_TYPE) {
             return supportsStructuredCasting(
                     sourceType, targetType, (s, t) -> supportsCasting(s, t, allowExplicit));
-        } else if (sourceRoot == RAW || targetRoot == RAW) {
+        } else if ((sourceRoot == RAW && !targetRoot.getFamilies().contains(BINARY_STRING))

Review comment:
       very nit: We use `hasFamily` for better code readability. Can we rewrite this to `(sourceRoot == RAW && !hasFamily(targetType, BINARY_STRING)`?

##########
File path: flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/LogicalTypeCastsTest.java
##########
@@ -253,7 +256,15 @@
                                 .build(),
                         false,
                         true
-                    }
+                    },
+
+                    // raw to binary
+                    {
+                        new RawType(Integer.class, new ValueSerializer(Integer.class)),

Review comment:
       nit: use `IntSeriaizer.INSTANCE`? should be more stable once we remove ValueSerializer

##########
File path: flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/utils/LogicalTypeCasts.java
##########
@@ -401,6 +405,8 @@ private static boolean supportsConstructedCasting(
                 }
             }
             return true;
+        } else if (sourceRoot == ARRAY && (targetRoot == CHAR || targetRoot == VARCHAR)) {

Review comment:
       why this change?

##########
File path: flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala
##########
@@ -40,15 +40,14 @@ import org.apache.flink.table.types.logical.utils.LogicalTypeCasts.supportsExpli
 import org.apache.flink.table.types.logical.utils.LogicalTypeChecks.getFieldTypes
 import org.apache.flink.table.types.logical.utils.LogicalTypeMerging.findCommonType
 import org.apache.flink.util.Preconditions.checkArgument
-
 import org.apache.calcite.avatica.util.DateTimeUtils.MILLIS_PER_DAY
 import org.apache.calcite.avatica.util.{DateTimeUtils, TimeUnitRange}
 import org.apache.calcite.util.BuiltInMethod
+import org.apache.flink.table.api.DataTypes.BYTES
 
 import java.lang.{StringBuilder => JStringBuilder}
 import java.nio.charset.StandardCharsets
 import java.util.Arrays.asList
-

Review comment:
       make sure that you have configured your imports correctly, Scala needs special settings. but it should be the same order as in our code style guides.

##########
File path: flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/CastFunctionMiscITCase.java
##########
@@ -26,9 +26,20 @@
 
 import org.junit.runners.Parameterized;
 
+import java.nio.ByteBuffer;
 import java.util.Arrays;
 import java.util.List;
 
+import static org.apache.flink.table.api.DataTypes.BIGINT;
+import static org.apache.flink.table.api.DataTypes.BINARY;
+import static org.apache.flink.table.api.DataTypes.BOOLEAN;
+import static org.apache.flink.table.api.DataTypes.BYTES;
+import static org.apache.flink.table.api.DataTypes.FIELD;
+import static org.apache.flink.table.api.DataTypes.INT;
+import static org.apache.flink.table.api.DataTypes.ROW;
+import static org.apache.flink.table.api.DataTypes.STRING;
+import static org.apache.flink.table.api.DataTypes.VARBINARY;
+import static org.apache.flink.table.api.DataTypes.of;

Review comment:
       nit: static imports of functions that are called `of` don't contribute to code readability, I would not import it, other types are fine




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] twalthr closed pull request #17450: [FLINK-24418][table-planner] Add support for casting RAW to BINARY

Posted by GitBox <gi...@apache.org>.
twalthr closed pull request #17450:
URL: https://github.com/apache/flink/pull/17450


   


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #17450: [FLINK-24418][table-planner] Add support for casting RAW to BINARY

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #17450:
URL: https://github.com/apache/flink/pull/17450#issuecomment-939949291


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "a27b45f90f31067656d64a3ec0a9d20d0c60402e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=24950",
       "triggerID" : "a27b45f90f31067656d64a3ec0a9d20d0c60402e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "459111838bfa4cd32499535d6d89636f4aa8a024",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=24952",
       "triggerID" : "459111838bfa4cd32499535d6d89636f4aa8a024",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e17cd221fe36ee6064cfff125b0a9b025c5310bd",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e17cd221fe36ee6064cfff125b0a9b025c5310bd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 459111838bfa4cd32499535d6d89636f4aa8a024 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=24952) 
   * e17cd221fe36ee6064cfff125b0a9b025c5310bd UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #17450: [FLINK-24418][table-planner] Add support for casting RAW to BINARY

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #17450:
URL: https://github.com/apache/flink/pull/17450#issuecomment-939949291


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "a27b45f90f31067656d64a3ec0a9d20d0c60402e",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=24950",
       "triggerID" : "a27b45f90f31067656d64a3ec0a9d20d0c60402e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "459111838bfa4cd32499535d6d89636f4aa8a024",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=24952",
       "triggerID" : "459111838bfa4cd32499535d6d89636f4aa8a024",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a27b45f90f31067656d64a3ec0a9d20d0c60402e Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=24950) 
   * 459111838bfa4cd32499535d6d89636f4aa8a024 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=24952) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot commented on pull request #17450: [FLINK-24418][table-planner] Add support for casting RAW to BINARY

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #17450:
URL: https://github.com/apache/flink/pull/17450#issuecomment-939949291


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "a27b45f90f31067656d64a3ec0a9d20d0c60402e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a27b45f90f31067656d64a3ec0a9d20d0c60402e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a27b45f90f31067656d64a3ec0a9d20d0c60402e UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] matriv commented on pull request #17450: [FLINK-24418][table-planner] Add support for casting RAW to BINARY

Posted by GitBox <gi...@apache.org>.
matriv commented on pull request #17450:
URL: https://github.com/apache/flink/pull/17450#issuecomment-941009407


   @twalthr Thank you for your comments, please check again.


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #17450: [FLINK-24418][table-planner] Add support for casting RAW to BINARY

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #17450:
URL: https://github.com/apache/flink/pull/17450#issuecomment-939949291


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "a27b45f90f31067656d64a3ec0a9d20d0c60402e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=24950",
       "triggerID" : "a27b45f90f31067656d64a3ec0a9d20d0c60402e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "459111838bfa4cd32499535d6d89636f4aa8a024",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=24952",
       "triggerID" : "459111838bfa4cd32499535d6d89636f4aa8a024",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 459111838bfa4cd32499535d6d89636f4aa8a024 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=24952) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #17450: [FLINK-24418][table-planner] Add support for casting RAW to BINARY

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #17450:
URL: https://github.com/apache/flink/pull/17450#issuecomment-939949291


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "a27b45f90f31067656d64a3ec0a9d20d0c60402e",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=24950",
       "triggerID" : "a27b45f90f31067656d64a3ec0a9d20d0c60402e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "459111838bfa4cd32499535d6d89636f4aa8a024",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=24952",
       "triggerID" : "459111838bfa4cd32499535d6d89636f4aa8a024",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a27b45f90f31067656d64a3ec0a9d20d0c60402e Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=24950) 
   * 459111838bfa4cd32499535d6d89636f4aa8a024 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=24952) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot commented on pull request #17450: [FLINK-24418][table-planner] Add support for casting RAW to BINARY

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #17450:
URL: https://github.com/apache/flink/pull/17450#issuecomment-939948619


   Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
   to review your pull request. We will use this comment to track the progress of the review.
   
   
   ## Automated Checks
   Last check on commit 8295f025da07db6142407192a872440d446a6e0f (Mon Oct 11 11:38:40 UTC 2021)
   
   **Warnings:**
    * No documentation files were touched! Remember to keep the Flink docs up to date!
   
   
   <sub>Mention the bot in a comment to re-run the automated checks.</sub>
   ## Review Progress
   
   * ❓ 1. The [description] looks good.
   * ❓ 2. There is [consensus] that the contribution should go into to Flink.
   * ❓ 3. Needs [attention] from.
   * ❓ 4. The change fits into the overall [architecture].
   * ❓ 5. Overall code [quality] is good.
   
   Please see the [Pull Request Review Guide](https://flink.apache.org/contributing/reviewing-prs.html) for a full explanation of the review process.<details>
    The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot approve description` to approve one or more aspects (aspects: `description`, `consensus`, `architecture` and `quality`)
    - `@flinkbot approve all` to approve all aspects
    - `@flinkbot approve-until architecture` to approve everything until `architecture`
    - `@flinkbot attention @username1 [@username2 ..]` to require somebody's attention
    - `@flinkbot disapprove architecture` to remove an approval you gave earlier
   </details>


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #17450: [FLINK-24418][table-planner] Add support for casting RAW to BINARY

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #17450:
URL: https://github.com/apache/flink/pull/17450#issuecomment-939949291


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "a27b45f90f31067656d64a3ec0a9d20d0c60402e",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=24950",
       "triggerID" : "a27b45f90f31067656d64a3ec0a9d20d0c60402e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "459111838bfa4cd32499535d6d89636f4aa8a024",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "459111838bfa4cd32499535d6d89636f4aa8a024",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a27b45f90f31067656d64a3ec0a9d20d0c60402e Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=24950) 
   * 459111838bfa4cd32499535d6d89636f4aa8a024 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #17450: [FLINK-24418][table-planner] Add support for casting RAW to BINARY

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #17450:
URL: https://github.com/apache/flink/pull/17450#issuecomment-939949291


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "a27b45f90f31067656d64a3ec0a9d20d0c60402e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=24950",
       "triggerID" : "a27b45f90f31067656d64a3ec0a9d20d0c60402e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "459111838bfa4cd32499535d6d89636f4aa8a024",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=24952",
       "triggerID" : "459111838bfa4cd32499535d6d89636f4aa8a024",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e17cd221fe36ee6064cfff125b0a9b025c5310bd",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=24989",
       "triggerID" : "e17cd221fe36ee6064cfff125b0a9b025c5310bd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 459111838bfa4cd32499535d6d89636f4aa8a024 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=24952) 
   * e17cd221fe36ee6064cfff125b0a9b025c5310bd Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=24989) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] matriv commented on a change in pull request #17450: [FLINK-24418][table-planner] Add support for casting RAW to BINARY

Posted by GitBox <gi...@apache.org>.
matriv commented on a change in pull request #17450:
URL: https://github.com/apache/flink/pull/17450#discussion_r727127467



##########
File path: flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/utils/LogicalTypeCasts.java
##########
@@ -319,7 +322,8 @@ private static boolean supportsCasting(
         } else if (sourceRoot == STRUCTURED_TYPE || targetRoot == STRUCTURED_TYPE) {
             return supportsStructuredCasting(
                     sourceType, targetType, (s, t) -> supportsCasting(s, t, allowExplicit));
-        } else if (sourceRoot == RAW || targetRoot == RAW) {
+        } else if ((sourceRoot == RAW && !targetRoot.getFamilies().contains(BINARY_STRING))

Review comment:
       Not nit, the method is there, I just missed it, thx!




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org