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 2022/01/26 14:17:39 UTC

[GitHub] [flink] slinkydeveloper opened a new pull request #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

slinkydeveloper opened a new pull request #18524:
URL: https://github.com/apache/flink/pull/18524


   ## What is the purpose of the change
   
   The goal of this PR is to allow performing casting to string of complex types
   
   ## Brief change log
   
   * Add patched `SqlTypeUtil` from calcite. This is required as calcite extension point for casting rules (see `SqlTypeCoercionRule`) is limited to atomic types. For constructed types, the logic is still hardcoded in `SqlTypeUtil`. This is the diff of my patched `SqlTypeUtil`:
   
   ![Screenshot from 2022-01-26 15-16-08](https://user-images.githubusercontent.com/6706544/151179245-9acee484-5ebd-422d-9dd1-ad6183239d35.png)
   
   * Add test cases
   
   ## Verifying this change
   
   Added test cases
   
   ## 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)`: yes
   
   ## Documentation
   
     - Does this pull request introduce a new feature? yes
     - If yes, how is the feature documented? not documented
   


-- 
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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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



##########
File path: flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/LogicalTypeCastsTest.java
##########
@@ -254,39 +243,34 @@
                                                 new StructuredAttribute("diff", new TinyIntType())))
                                 .build(),
                         false,
-                        true
-                    },
+                        true),
 
-                    // raw to binary
-                    {
-                        new RawType(Integer.class, IntSerializer.INSTANCE),
+                // raw to binary
+                Arguments.of(
+                        new RawType<>(Integer.class, IntSerializer.INSTANCE),
                         new BinaryType(),
                         false,
-                        true
-                    },
-                });
+                        true),
+                // raw to binary
+                Arguments.of(
+                        new RawType<>(Integer.class, IntSerializer.INSTANCE),
+                        VarCharType.STRING_TYPE,
+                        false,
+                        true));
     }
 
-    @Parameter public LogicalType sourceType;
-
-    @Parameter(1)
-    public LogicalType targetType;
-
-    @Parameter(2)
-    public boolean supportsImplicit;
-
-    @Parameter(3)
-    public boolean supportsExplicit;
-
-    @Test
-    public void testImplicitCasting() {
+    @ParameterizedTest(name = "{index}: [From: {0}, To: {1}, Implicit: {2}, Explicit: {3}]")

Review comment:
       I know, again that's just my like, no need to change, consistency is more important.




-- 
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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30242",
       "triggerID" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252",
       "triggerID" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e7d824ef0fe09b4b3576fde918a80cfb99069c1b Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252) 
   * 5219cc9984b4392eff658fdf342b1784e9b17aed 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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30242",
       "triggerID" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252",
       "triggerID" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278",
       "triggerID" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e7d824ef0fe09b4b3576fde918a80cfb99069c1b Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252) 
   * 5219cc9984b4392eff658fdf342b1784e9b17aed Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278) 
   * f48082f47413dc6d3ace4b452afb0d7886e35621 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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30242",
       "triggerID" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252",
       "triggerID" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278",
       "triggerID" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288",
       "triggerID" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30296",
       "triggerID" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5219cc9984b4392eff658fdf342b1784e9b17aed Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278) 
   * f48082f47413dc6d3ace4b452afb0d7886e35621 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288) 
   * 8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30296) 
   
   <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] slinkydeveloper commented on a change in pull request #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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



##########
File path: flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/CastFunctionITCase.java
##########
@@ -1332,12 +1350,12 @@ private static boolean isTimestampToNumeric(LogicalType srcType, LogicalType trg
         return srcType.is(LogicalTypeFamily.TIMESTAMP) && trgType.is(LogicalTypeFamily.NUMERIC);
     }
 
-    private static <K, V> Map.Entry<K, V> entry(K k, V v) {
+    static <K, V> Map.Entry<K, V> entry(K k, V v) {

Review comment:
       I like it




-- 
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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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


   


-- 
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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30242",
       "triggerID" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252",
       "triggerID" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278",
       "triggerID" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288",
       "triggerID" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e7d824ef0fe09b4b3576fde918a80cfb99069c1b Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252) 
   * 5219cc9984b4392eff658fdf342b1784e9b17aed Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278) 
   * f48082f47413dc6d3ace4b452afb0d7886e35621 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288) 
   * 8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616 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] slinkydeveloper commented on a change in pull request #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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



##########
File path: flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/fun/SqlCastFunction.java
##########
@@ -0,0 +1,213 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.sql.fun;
+
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeFamily;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+import org.apache.flink.table.types.logical.utils.LogicalTypeCasts;
+
+import com.google.common.collect.ImmutableSetMultimap;
+import com.google.common.collect.SetMultimap;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFamily;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlDynamicParam;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlIntervalQualifier;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperandCountRange;
+import org.apache.calcite.sql.SqlOperatorBinding;
+import org.apache.calcite.sql.SqlSyntax;
+import org.apache.calcite.sql.SqlUtil;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.type.InferTypes;
+import org.apache.calcite.sql.type.SqlOperandCountRanges;
+import org.apache.calcite.sql.type.SqlTypeFamily;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.sql.validate.SqlMonotonicity;
+import org.apache.calcite.sql.validate.SqlValidatorImpl;
+
+import java.text.Collator;
+import java.util.Objects;
+
+import static org.apache.calcite.util.Static.RESOURCE;
+
+/**
+ * SqlCastFunction. Note that the std functions are really singleton objects, because they always
+ * get fetched via the StdOperatorTable. So you can't store any local info in the class and hence
+ * the return type data is maintained in operand[1] through the validation phase.
+ *
+ * <p>Can be used for both {@link SqlCall} and {@link org.apache.calcite.rex.RexCall}. Note that the
+ * {@code SqlCall} has two operands (expression and type), while the {@code RexCall} has one operand
+ * (expression) and the type is obtained from {@link org.apache.calcite.rex.RexNode#getType()}.
+ *
+ * <p>The class was copied over because of CALCITE-XXXX, in order to workaround the method {@link
+ * SqlTypeUtil#canCastFrom(RelDataType, RelDataType, boolean)}. Line 141 in {@link
+ * #checkOperandTypes(SqlCallBinding, boolean)} and new method {@link #canCastFrom(RelDataType,
+ * RelDataType)}.
+ *
+ * @see SqlCastOperator
+ */
+public class SqlCastFunction extends SqlFunction {
+    // ~ Instance fields --------------------------------------------------------
+
+    /** Map of all casts that do not preserve monotonicity. */
+    private final SetMultimap<SqlTypeFamily, SqlTypeFamily> nonMonotonicCasts =
+            ImmutableSetMultimap.<SqlTypeFamily, SqlTypeFamily>builder()
+                    .put(SqlTypeFamily.EXACT_NUMERIC, SqlTypeFamily.CHARACTER)
+                    .put(SqlTypeFamily.NUMERIC, SqlTypeFamily.CHARACTER)
+                    .put(SqlTypeFamily.APPROXIMATE_NUMERIC, SqlTypeFamily.CHARACTER)
+                    .put(SqlTypeFamily.DATETIME_INTERVAL, SqlTypeFamily.CHARACTER)
+                    .put(SqlTypeFamily.CHARACTER, SqlTypeFamily.EXACT_NUMERIC)
+                    .put(SqlTypeFamily.CHARACTER, SqlTypeFamily.NUMERIC)
+                    .put(SqlTypeFamily.CHARACTER, SqlTypeFamily.APPROXIMATE_NUMERIC)
+                    .put(SqlTypeFamily.CHARACTER, SqlTypeFamily.DATETIME_INTERVAL)
+                    .put(SqlTypeFamily.DATETIME, SqlTypeFamily.TIME)
+                    .put(SqlTypeFamily.TIMESTAMP, SqlTypeFamily.TIME)
+                    .put(SqlTypeFamily.TIME, SqlTypeFamily.DATETIME)
+                    .put(SqlTypeFamily.TIME, SqlTypeFamily.TIMESTAMP)
+                    .build();
+
+    // ~ Constructors -----------------------------------------------------------
+
+    public SqlCastFunction() {
+        super("CAST", SqlKind.CAST, null, InferTypes.FIRST_KNOWN, null, SqlFunctionCategory.SYSTEM);
+    }
+
+    // ~ Methods ----------------------------------------------------------------
+
+    public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
+        assert opBinding.getOperandCount() == 2;
+        RelDataType ret = opBinding.getOperandType(1);
+        RelDataType firstType = opBinding.getOperandType(0);
+        ret = opBinding.getTypeFactory().createTypeWithNullability(ret, firstType.isNullable());
+        if (opBinding instanceof SqlCallBinding) {
+            SqlCallBinding callBinding = (SqlCallBinding) opBinding;
+            SqlNode operand0 = callBinding.operand(0);
+
+            // dynamic parameters and null constants need their types assigned
+            // to them using the type they are casted to.
+            if (((operand0 instanceof SqlLiteral) && (((SqlLiteral) operand0).getValue() == null))
+                    || (operand0 instanceof SqlDynamicParam)) {
+                final SqlValidatorImpl validator = (SqlValidatorImpl) callBinding.getValidator();
+                validator.setValidatedNodeType(operand0, ret);
+            }
+        }
+        return ret;
+    }
+
+    public String getSignatureTemplate(final int operandsCount) {
+        assert operandsCount == 2;
+        return "{0}({1} AS {2})";
+    }
+
+    public SqlOperandCountRange getOperandCountRange() {
+        return SqlOperandCountRanges.of(2);
+    }
+
+    /**
+     * Makes sure that the number and types of arguments are allowable. Operators (such as "ROW" and
+     * "AS") which do not check their arguments can override this method.
+     */
+    public boolean checkOperandTypes(SqlCallBinding callBinding, boolean throwOnFailure) {
+        final SqlNode left = callBinding.operand(0);
+        final SqlNode right = callBinding.operand(1);
+        if (SqlUtil.isNullLiteral(left, false) || left instanceof SqlDynamicParam) {
+            return true;
+        }
+        RelDataType validatedNodeType = callBinding.getValidator().getValidatedNodeType(left);
+        RelDataType returnType = SqlTypeUtil.deriveType(callBinding, right);
+        if (!canCastFrom(returnType, validatedNodeType)) {
+            if (throwOnFailure) {
+                throw callBinding.newError(
+                        RESOURCE.cannotCastValue(
+                                validatedNodeType.toString(), returnType.toString()));
+            }
+            return false;
+        }
+        if (SqlTypeUtil.areCharacterSetsMismatched(validatedNodeType, returnType)) {
+            if (throwOnFailure) {
+                // Include full type string to indicate character
+                // set mismatch.
+                throw callBinding.newError(
+                        RESOURCE.cannotCastValue(
+                                validatedNodeType.getFullTypeString(),
+                                returnType.getFullTypeString()));
+            }
+            return false;
+        }
+        return true;
+    }
+
+    private boolean canCastFrom(RelDataType toType, RelDataType fromType) {
+        LogicalType from = FlinkTypeFactory.toLogicalType(fromType);
+        if (from.is(LogicalTypeFamily.CONSTRUCTED)
+                || from.is(LogicalTypeRoot.RAW)
+                || from.is(LogicalTypeRoot.STRUCTURED_TYPE)) {
+            return LogicalTypeCasts.supportsExplicitCast(
+                    from, FlinkTypeFactory.toLogicalType(toType));
+        }

Review comment:
       Add comment and check without doing the conversion of `from`, as it's expensive




-- 
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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30242",
       "triggerID" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252",
       "triggerID" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278",
       "triggerID" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288",
       "triggerID" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e7d824ef0fe09b4b3576fde918a80cfb99069c1b Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252) 
   * 5219cc9984b4392eff658fdf342b1784e9b17aed Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278) 
   * f48082f47413dc6d3ace4b452afb0d7886e35621 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288) 
   
   <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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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



##########
File path: flink-core/src/main/java/org/apache/flink/util/CollectionUtil.java
##########
@@ -109,4 +110,22 @@ public static boolean isNullOrEmpty(Map<?, ?> map) {
         iterator.forEachRemaining(list::add);
         return list;
     }
+
+    /** Returns an immutable {@link Map.Entry}. */
+    public static <K, V> Map.Entry<K, V> entry(K k, V v) {

Review comment:
       please put core changes in a hotfix commit

##########
File path: flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/utils/LogicalTypeCasts.java
##########
@@ -318,11 +318,17 @@ private static boolean supportsCasting(
             // cast between interval and exact numeric is only supported if interval has a single
             // field
             return isSingleFieldInterval(targetType);
+        } else if ((sourceType.is(CONSTRUCTED) || sourceType.is(STRUCTURED_TYPE))
+                && targetType.is(CHARACTER_STRING)) {
+            return supportsCollectionAndStructuredToStringCasting(sourceType, allowExplicit);

Review comment:
       can't this also just always be `true`? What casting does not support to string?

##########
File path: flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/fun/SqlCastFunction.java
##########
@@ -0,0 +1,220 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.sql.fun;
+
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.types.logical.utils.LogicalTypeCasts;
+
+import com.google.common.collect.ImmutableSetMultimap;
+import com.google.common.collect.SetMultimap;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFamily;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlDynamicParam;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlIntervalQualifier;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperandCountRange;
+import org.apache.calcite.sql.SqlOperatorBinding;
+import org.apache.calcite.sql.SqlSyntax;
+import org.apache.calcite.sql.SqlUtil;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.type.InferTypes;
+import org.apache.calcite.sql.type.SqlOperandCountRanges;
+import org.apache.calcite.sql.type.SqlTypeFamily;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.sql.validate.SqlMonotonicity;
+import org.apache.calcite.sql.validate.SqlValidatorImpl;
+
+import java.text.Collator;
+import java.util.Objects;
+
+import static org.apache.calcite.util.Static.RESOURCE;
+
+/**
+ * SqlCastFunction. Note that the std functions are really singleton objects, because they always
+ * get fetched via the StdOperatorTable. So you can't store any local info in the class and hence
+ * the return type data is maintained in operand[1] through the validation phase.
+ *
+ * <p>Can be used for both {@link SqlCall} and {@link org.apache.calcite.rex.RexCall}. Note that the
+ * {@code SqlCall} has two operands (expression and type), while the {@code RexCall} has one operand
+ * (expression) and the type is obtained from {@link org.apache.calcite.rex.RexNode#getType()}.
+ *
+ * <p>The class was copied over because of CALCITE-XXXX, in order to workaround the method {@link

Review comment:
       `CALCITE-XXXX`?




-- 
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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30242",
       "triggerID" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252",
       "triggerID" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278",
       "triggerID" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288",
       "triggerID" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30296",
       "triggerID" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff6ad63a811bad6c67be6cec61b2f6dd69f6b572",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30301",
       "triggerID" : "ff6ad63a811bad6c67be6cec61b2f6dd69f6b572",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9e05fba98c2a3b8727b518e19d5beb8b7a649230",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30526",
       "triggerID" : "9e05fba98c2a3b8727b518e19d5beb8b7a649230",
       "triggerType" : "PUSH"
     }, {
       "hash" : "106308aab12dcfb37d9279d51b7bb4d2b076baff",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=31926",
       "triggerID" : "106308aab12dcfb37d9279d51b7bb4d2b076baff",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 106308aab12dcfb37d9279d51b7bb4d2b076baff Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=31926) 
   
   <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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30242",
       "triggerID" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252",
       "triggerID" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b8a6710e7f17a8613051746080e009bc79361f00 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30242) 
   * e7d824ef0fe09b4b3576fde918a80cfb99069c1b Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252) 
   
   <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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30242",
       "triggerID" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252",
       "triggerID" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278",
       "triggerID" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e7d824ef0fe09b4b3576fde918a80cfb99069c1b Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252) 
   * 5219cc9984b4392eff658fdf342b1784e9b17aed Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278) 
   
   <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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30242",
       "triggerID" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252",
       "triggerID" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278",
       "triggerID" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288",
       "triggerID" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e7d824ef0fe09b4b3576fde918a80cfb99069c1b Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252) 
   * 5219cc9984b4392eff658fdf342b1784e9b17aed Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278) 
   * f48082f47413dc6d3ace4b452afb0d7886e35621 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288) 
   * 8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616 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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30242",
       "triggerID" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252",
       "triggerID" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278",
       "triggerID" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288",
       "triggerID" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30296",
       "triggerID" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff6ad63a811bad6c67be6cec61b2f6dd69f6b572",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30301",
       "triggerID" : "ff6ad63a811bad6c67be6cec61b2f6dd69f6b572",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9e05fba98c2a3b8727b518e19d5beb8b7a649230",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30526",
       "triggerID" : "9e05fba98c2a3b8727b518e19d5beb8b7a649230",
       "triggerType" : "PUSH"
     }, {
       "hash" : "106308aab12dcfb37d9279d51b7bb4d2b076baff",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "106308aab12dcfb37d9279d51b7bb4d2b076baff",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9e05fba98c2a3b8727b518e19d5beb8b7a649230 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30526) 
   * 106308aab12dcfb37d9279d51b7bb4d2b076baff 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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30242",
       "triggerID" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252",
       "triggerID" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278",
       "triggerID" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288",
       "triggerID" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30296",
       "triggerID" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff6ad63a811bad6c67be6cec61b2f6dd69f6b572",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30301",
       "triggerID" : "ff6ad63a811bad6c67be6cec61b2f6dd69f6b572",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9e05fba98c2a3b8727b518e19d5beb8b7a649230",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30526",
       "triggerID" : "9e05fba98c2a3b8727b518e19d5beb8b7a649230",
       "triggerType" : "PUSH"
     }, {
       "hash" : "106308aab12dcfb37d9279d51b7bb4d2b076baff",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=31926",
       "triggerID" : "106308aab12dcfb37d9279d51b7bb4d2b076baff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b09273676906d83f35efa59916aabf74e1ffac6",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=31994",
       "triggerID" : "7b09273676906d83f35efa59916aabf74e1ffac6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7b09273676906d83f35efa59916aabf74e1ffac6 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=31994) 
   
   <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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30242",
       "triggerID" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252",
       "triggerID" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278",
       "triggerID" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288",
       "triggerID" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30296",
       "triggerID" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff6ad63a811bad6c67be6cec61b2f6dd69f6b572",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30301",
       "triggerID" : "ff6ad63a811bad6c67be6cec61b2f6dd69f6b572",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f48082f47413dc6d3ace4b452afb0d7886e35621 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288) 
   * 8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30296) 
   * ff6ad63a811bad6c67be6cec61b2f6dd69f6b572 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30301) 
   
   <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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30242",
       "triggerID" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252",
       "triggerID" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278",
       "triggerID" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288",
       "triggerID" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30296",
       "triggerID" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff6ad63a811bad6c67be6cec61b2f6dd69f6b572",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30301",
       "triggerID" : "ff6ad63a811bad6c67be6cec61b2f6dd69f6b572",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30296) 
   * ff6ad63a811bad6c67be6cec61b2f6dd69f6b572 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30301) 
   
   <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] slinkydeveloper commented on a change in pull request #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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



##########
File path: flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/CastFunctionITCase.java
##########
@@ -1229,6 +1242,11 @@ private CastTestSpecBuilder fromCase(DataType dataType, Object src, Object targe
             this.columnTypes.add(dataType);
             this.columnData.add(src);
             this.expectedValues.add(target);
+            assertThat(
+                            LogicalTypeCasts.supportsExplicitCast(

Review comment:
       It is not :rofl: 




-- 
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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30242",
       "triggerID" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252",
       "triggerID" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278",
       "triggerID" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288",
       "triggerID" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30296",
       "triggerID" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff6ad63a811bad6c67be6cec61b2f6dd69f6b572",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ff6ad63a811bad6c67be6cec61b2f6dd69f6b572",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5219cc9984b4392eff658fdf342b1784e9b17aed Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278) 
   * f48082f47413dc6d3ace4b452afb0d7886e35621 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288) 
   * 8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30296) 
   * ff6ad63a811bad6c67be6cec61b2f6dd69f6b572 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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30242",
       "triggerID" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252",
       "triggerID" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e7d824ef0fe09b4b3576fde918a80cfb99069c1b Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252) 
   
   <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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30242",
       "triggerID" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252",
       "triggerID" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278",
       "triggerID" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288",
       "triggerID" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30296",
       "triggerID" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff6ad63a811bad6c67be6cec61b2f6dd69f6b572",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30301",
       "triggerID" : "ff6ad63a811bad6c67be6cec61b2f6dd69f6b572",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5219cc9984b4392eff658fdf342b1784e9b17aed Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278) 
   * f48082f47413dc6d3ace4b452afb0d7886e35621 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288) 
   * 8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30296) 
   * ff6ad63a811bad6c67be6cec61b2f6dd69f6b572 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30301) 
   
   <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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30242",
       "triggerID" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252",
       "triggerID" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278",
       "triggerID" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288",
       "triggerID" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30296",
       "triggerID" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff6ad63a811bad6c67be6cec61b2f6dd69f6b572",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30301",
       "triggerID" : "ff6ad63a811bad6c67be6cec61b2f6dd69f6b572",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9e05fba98c2a3b8727b518e19d5beb8b7a649230",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30526",
       "triggerID" : "9e05fba98c2a3b8727b518e19d5beb8b7a649230",
       "triggerType" : "PUSH"
     }, {
       "hash" : "106308aab12dcfb37d9279d51b7bb4d2b076baff",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=31926",
       "triggerID" : "106308aab12dcfb37d9279d51b7bb4d2b076baff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b09273676906d83f35efa59916aabf74e1ffac6",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=31994",
       "triggerID" : "7b09273676906d83f35efa59916aabf74e1ffac6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 106308aab12dcfb37d9279d51b7bb4d2b076baff Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=31926) 
   * 7b09273676906d83f35efa59916aabf74e1ffac6 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=31994) 
   
   <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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30242",
       "triggerID" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252",
       "triggerID" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278",
       "triggerID" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288",
       "triggerID" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e7d824ef0fe09b4b3576fde918a80cfb99069c1b Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252) 
   * 5219cc9984b4392eff658fdf342b1784e9b17aed Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278) 
   * f48082f47413dc6d3ace4b452afb0d7886e35621 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288) 
   
   <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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30242",
       "triggerID" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252",
       "triggerID" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278",
       "triggerID" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288",
       "triggerID" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e7d824ef0fe09b4b3576fde918a80cfb99069c1b Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252) 
   * 5219cc9984b4392eff658fdf342b1784e9b17aed Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278) 
   * f48082f47413dc6d3ace4b452afb0d7886e35621 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288) 
   
   <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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30242",
       "triggerID" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252",
       "triggerID" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b8a6710e7f17a8613051746080e009bc79361f00 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30242) 
   * e7d824ef0fe09b4b3576fde918a80cfb99069c1b Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252) 
   
   <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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30242",
       "triggerID" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252",
       "triggerID" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278",
       "triggerID" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288",
       "triggerID" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30296",
       "triggerID" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5219cc9984b4392eff658fdf342b1784e9b17aed Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278) 
   * f48082f47413dc6d3ace4b452afb0d7886e35621 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288) 
   * 8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30296) 
   
   <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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30242",
       "triggerID" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252",
       "triggerID" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278",
       "triggerID" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288",
       "triggerID" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30296",
       "triggerID" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff6ad63a811bad6c67be6cec61b2f6dd69f6b572",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ff6ad63a811bad6c67be6cec61b2f6dd69f6b572",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5219cc9984b4392eff658fdf342b1784e9b17aed Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278) 
   * f48082f47413dc6d3ace4b452afb0d7886e35621 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288) 
   * 8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30296) 
   * ff6ad63a811bad6c67be6cec61b2f6dd69f6b572 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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30242",
       "triggerID" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252",
       "triggerID" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278",
       "triggerID" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288",
       "triggerID" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e7d824ef0fe09b4b3576fde918a80cfb99069c1b Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252) 
   * 5219cc9984b4392eff658fdf342b1784e9b17aed Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278) 
   * f48082f47413dc6d3ace4b452afb0d7886e35621 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288) 
   * 8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616 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] slinkydeveloper commented on a change in pull request #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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



##########
File path: flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/CastFunctionITCase.java
##########
@@ -1332,12 +1350,12 @@ private static boolean isTimestampToNumeric(LogicalType srcType, LogicalType trg
         return srcType.is(LogicalTypeFamily.TIMESTAMP) && trgType.is(LogicalTypeFamily.NUMERIC);
     }
 
-    private static <K, V> Map.Entry<K, V> entry(K k, V v) {
+    static <K, V> Map.Entry<K, V> entry(K k, V v) {

Review comment:
       Not sure where?




-- 
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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30242",
       "triggerID" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252",
       "triggerID" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278",
       "triggerID" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e7d824ef0fe09b4b3576fde918a80cfb99069c1b Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252) 
   * 5219cc9984b4392eff658fdf342b1784e9b17aed Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278) 
   * f48082f47413dc6d3ace4b452afb0d7886e35621 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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30242",
       "triggerID" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252",
       "triggerID" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278",
       "triggerID" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e7d824ef0fe09b4b3576fde918a80cfb99069c1b Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252) 
   * 5219cc9984b4392eff658fdf342b1784e9b17aed Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278) 
   
   <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] slinkydeveloper commented on a change in pull request #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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



##########
File path: flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/utils/LogicalTypeCasts.java
##########
@@ -318,11 +318,17 @@ private static boolean supportsCasting(
             // cast between interval and exact numeric is only supported if interval has a single
             // field
             return isSingleFieldInterval(targetType);
+        } else if ((sourceType.is(CONSTRUCTED) || sourceType.is(STRUCTURED_TYPE))
+                && targetType.is(CHARACTER_STRING)) {
+            return supportsCollectionAndStructuredToStringCasting(sourceType, allowExplicit);

Review comment:
       Now we should be good to go, although I still had to do a couple of changes in this method to ensure the proper if-else branch :(




-- 
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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30242",
       "triggerID" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252",
       "triggerID" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278",
       "triggerID" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e7d824ef0fe09b4b3576fde918a80cfb99069c1b Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252) 
   * 5219cc9984b4392eff658fdf342b1784e9b17aed Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278) 
   
   <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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30242",
       "triggerID" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252",
       "triggerID" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278",
       "triggerID" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288",
       "triggerID" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e7d824ef0fe09b4b3576fde918a80cfb99069c1b Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252) 
   * 5219cc9984b4392eff658fdf342b1784e9b17aed Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278) 
   * f48082f47413dc6d3ace4b452afb0d7886e35621 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288) 
   
   <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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30242",
       "triggerID" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252",
       "triggerID" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278",
       "triggerID" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288",
       "triggerID" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30296",
       "triggerID" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5219cc9984b4392eff658fdf342b1784e9b17aed Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278) 
   * f48082f47413dc6d3ace4b452afb0d7886e35621 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288) 
   * 8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30296) 
   
   <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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30242",
       "triggerID" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252",
       "triggerID" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278",
       "triggerID" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288",
       "triggerID" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30296",
       "triggerID" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff6ad63a811bad6c67be6cec61b2f6dd69f6b572",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ff6ad63a811bad6c67be6cec61b2f6dd69f6b572",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5219cc9984b4392eff658fdf342b1784e9b17aed Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278) 
   * f48082f47413dc6d3ace4b452afb0d7886e35621 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288) 
   * 8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30296) 
   * ff6ad63a811bad6c67be6cec61b2f6dd69f6b572 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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30242",
       "triggerID" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252",
       "triggerID" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278",
       "triggerID" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288",
       "triggerID" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30296",
       "triggerID" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff6ad63a811bad6c67be6cec61b2f6dd69f6b572",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30301",
       "triggerID" : "ff6ad63a811bad6c67be6cec61b2f6dd69f6b572",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9e05fba98c2a3b8727b518e19d5beb8b7a649230",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9e05fba98c2a3b8727b518e19d5beb8b7a649230",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ff6ad63a811bad6c67be6cec61b2f6dd69f6b572 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30301) 
   * 9e05fba98c2a3b8727b518e19d5beb8b7a649230 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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30242",
       "triggerID" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252",
       "triggerID" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278",
       "triggerID" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288",
       "triggerID" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30296",
       "triggerID" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff6ad63a811bad6c67be6cec61b2f6dd69f6b572",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30301",
       "triggerID" : "ff6ad63a811bad6c67be6cec61b2f6dd69f6b572",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9e05fba98c2a3b8727b518e19d5beb8b7a649230",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30526",
       "triggerID" : "9e05fba98c2a3b8727b518e19d5beb8b7a649230",
       "triggerType" : "PUSH"
     }, {
       "hash" : "106308aab12dcfb37d9279d51b7bb4d2b076baff",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=31926",
       "triggerID" : "106308aab12dcfb37d9279d51b7bb4d2b076baff",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9e05fba98c2a3b8727b518e19d5beb8b7a649230 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30526) 
   * 106308aab12dcfb37d9279d51b7bb4d2b076baff Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=31926) 
   
   <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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30242",
       "triggerID" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252",
       "triggerID" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278",
       "triggerID" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288",
       "triggerID" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30296",
       "triggerID" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff6ad63a811bad6c67be6cec61b2f6dd69f6b572",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30301",
       "triggerID" : "ff6ad63a811bad6c67be6cec61b2f6dd69f6b572",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9e05fba98c2a3b8727b518e19d5beb8b7a649230",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30526",
       "triggerID" : "9e05fba98c2a3b8727b518e19d5beb8b7a649230",
       "triggerType" : "PUSH"
     }, {
       "hash" : "106308aab12dcfb37d9279d51b7bb4d2b076baff",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=31926",
       "triggerID" : "106308aab12dcfb37d9279d51b7bb4d2b076baff",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b09273676906d83f35efa59916aabf74e1ffac6",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7b09273676906d83f35efa59916aabf74e1ffac6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 106308aab12dcfb37d9279d51b7bb4d2b076baff Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=31926) 
   * 7b09273676906d83f35efa59916aabf74e1ffac6 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] slinkydeveloper commented on a change in pull request #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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



##########
File path: flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/LogicalTypeCastsTest.java
##########
@@ -254,39 +243,34 @@
                                                 new StructuredAttribute("diff", new TinyIntType())))
                                 .build(),
                         false,
-                        true
-                    },
+                        true),
 
-                    // raw to binary
-                    {
-                        new RawType(Integer.class, IntSerializer.INSTANCE),
+                // raw to binary
+                Arguments.of(
+                        new RawType<>(Integer.class, IntSerializer.INSTANCE),
                         new BinaryType(),
                         false,
-                        true
-                    },
-                });
+                        true),
+                // raw to binary
+                Arguments.of(
+                        new RawType<>(Integer.class, IntSerializer.INSTANCE),
+                        VarCharType.STRING_TYPE,
+                        false,
+                        true));
     }
 
-    @Parameter public LogicalType sourceType;
-
-    @Parameter(1)
-    public LogicalType targetType;
-
-    @Parameter(2)
-    public boolean supportsImplicit;
-
-    @Parameter(3)
-    public boolean supportsExplicit;
-
-    @Test
-    public void testImplicitCasting() {
+    @ParameterizedTest(name = "{index}: [From: {0}, To: {1}, Implicit: {2}, Explicit: {3}]")

Review comment:
       TBH I prefer to keep it this way, as this is what we do with all the parametrized other tests (e.g. see `CastRulesTest`)




-- 
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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b8a6710e7f17a8613051746080e009bc79361f00 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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30242",
       "triggerID" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252",
       "triggerID" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278",
       "triggerID" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e7d824ef0fe09b4b3576fde918a80cfb99069c1b Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252) 
   * 5219cc9984b4392eff658fdf342b1784e9b17aed Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278) 
   * f48082f47413dc6d3ace4b452afb0d7886e35621 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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30242",
       "triggerID" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252",
       "triggerID" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278",
       "triggerID" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288",
       "triggerID" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e7d824ef0fe09b4b3576fde918a80cfb99069c1b Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252) 
   * 5219cc9984b4392eff658fdf342b1784e9b17aed Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278) 
   * f48082f47413dc6d3ace4b452afb0d7886e35621 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288) 
   * 8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616 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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30242",
       "triggerID" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252",
       "triggerID" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278",
       "triggerID" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288",
       "triggerID" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30296",
       "triggerID" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e7d824ef0fe09b4b3576fde918a80cfb99069c1b Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252) 
   * 5219cc9984b4392eff658fdf342b1784e9b17aed Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278) 
   * f48082f47413dc6d3ace4b452afb0d7886e35621 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288) 
   * 8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30296) 
   
   <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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30242",
       "triggerID" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252",
       "triggerID" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278",
       "triggerID" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288",
       "triggerID" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30296",
       "triggerID" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff6ad63a811bad6c67be6cec61b2f6dd69f6b572",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30301",
       "triggerID" : "ff6ad63a811bad6c67be6cec61b2f6dd69f6b572",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9e05fba98c2a3b8727b518e19d5beb8b7a649230",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30526",
       "triggerID" : "9e05fba98c2a3b8727b518e19d5beb8b7a649230",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ff6ad63a811bad6c67be6cec61b2f6dd69f6b572 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30301) 
   * 9e05fba98c2a3b8727b518e19d5beb8b7a649230 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30526) 
   
   <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] slinkydeveloper commented on a change in pull request #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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



##########
File path: flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/utils/LogicalTypeCasts.java
##########
@@ -318,11 +318,17 @@ private static boolean supportsCasting(
             // cast between interval and exact numeric is only supported if interval has a single
             // field
             return isSingleFieldInterval(targetType);
+        } else if ((sourceType.is(CONSTRUCTED) || sourceType.is(STRUCTURED_TYPE))
+                && targetType.is(CHARACTER_STRING)) {
+            return supportsCollectionAndStructuredToStringCasting(sourceType, allowExplicit);

Review comment:
       We don't have any at the moment, at least looking at the matrix I wrote here https://github.com/apache/flink/pull/18813. Nevertheless, isn't it better to keep this code to avoid eventual future bugs with new types?




-- 
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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30242",
       "triggerID" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252",
       "triggerID" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278",
       "triggerID" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288",
       "triggerID" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30296",
       "triggerID" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff6ad63a811bad6c67be6cec61b2f6dd69f6b572",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30301",
       "triggerID" : "ff6ad63a811bad6c67be6cec61b2f6dd69f6b572",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9e05fba98c2a3b8727b518e19d5beb8b7a649230",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30526",
       "triggerID" : "9e05fba98c2a3b8727b518e19d5beb8b7a649230",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9e05fba98c2a3b8727b518e19d5beb8b7a649230 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30526) 
   
   <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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30242",
       "triggerID" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252",
       "triggerID" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e7d824ef0fe09b4b3576fde918a80cfb99069c1b Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252) 
   
   <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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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



##########
File path: flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/CastFunctionITCase.java
##########
@@ -1332,12 +1350,12 @@ private static boolean isTimestampToNumeric(LogicalType srcType, LogicalType trg
         return srcType.is(LogicalTypeFamily.TIMESTAMP) && trgType.is(LogicalTypeFamily.NUMERIC);
     }
 
-    private static <K, V> Map.Entry<K, V> entry(K k, V v) {
+    static <K, V> Map.Entry<K, V> entry(K k, V v) {

Review comment:
       Maybe even in `CollectionUtil`, but then maybe it's too much.
   Anyways it was just a quick thought.




-- 
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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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



##########
File path: flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/CastFunctionITCase.java
##########
@@ -1229,6 +1242,11 @@ private CastTestSpecBuilder fromCase(DataType dataType, Object src, Object targe
             this.columnTypes.add(dataType);
             this.columnData.add(src);
             this.expectedValues.add(target);
+            assertThat(
+                            LogicalTypeCasts.supportsExplicitCast(

Review comment:
       The formatting here, seems a bit off, but might be wrong.

##########
File path: flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/CastFunctionITCase.java
##########
@@ -1332,12 +1350,12 @@ private static boolean isTimestampToNumeric(LogicalType srcType, LogicalType trg
         return srcType.is(LogicalTypeFamily.TIMESTAMP) && trgType.is(LogicalTypeFamily.NUMERIC);
     }
 
-    private static <K, V> Map.Entry<K, V> entry(K k, V v) {
+    static <K, V> Map.Entry<K, V> entry(K k, V v) {

Review comment:
       Maybe we can add those 2 methods in some utility class.

##########
File path: flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/CastFunctionMiscITCase.java
##########
@@ -206,7 +211,34 @@ protected Configuration configuration() {
                         .testSqlResult(
                                 "CAST(CAST(x'68656C6C6F2063617374' AS BINARY(10)) AS VARCHAR)",
                                 "68656c6c6f2063617374",
-                                STRING().notNull()));
+                                STRING().notNull()),
+                // Test cases that can't be added to CastFunctionITCase because they need to
+                // workaround the limitations of fromValues
+                TestSpec.forFunction(BuiltInFunctionDefinitions.CAST, "cast STRUCTURED to STRING")
+                        .onFieldsWithData(123456, "Flink")
+                        .andDataTypes(INT(), STRING())
+                        .withFunction(StructuredTypeConstructor.class)
+                        .testTableApiResult(
+                                call("StructuredTypeConstructor", row($("f0"), $("f1")))
+                                        .cast(STRING()),
+                                "(i=123456, s=Flink)",
+                                STRING()),
+                TestSpec.forFunction(BuiltInFunctionDefinitions.CAST, "cast MULTISET to STRING")
+                        .onFieldsWithData(map(entry("a", 1), entry("b", 1)))

Review comment:
       nit: use `2` for the second entry.




-- 
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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30242",
       "triggerID" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252",
       "triggerID" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278",
       "triggerID" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288",
       "triggerID" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e7d824ef0fe09b4b3576fde918a80cfb99069c1b Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252) 
   * 5219cc9984b4392eff658fdf342b1784e9b17aed Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278) 
   * f48082f47413dc6d3ace4b452afb0d7886e35621 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288) 
   
   <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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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


   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 b8a6710e7f17a8613051746080e009bc79361f00 (Wed Jan 26 14:22:51 UTC 2022)
   
   **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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30242",
       "triggerID" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252",
       "triggerID" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e7d824ef0fe09b4b3576fde918a80cfb99069c1b Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252) 
   
   <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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30242",
       "triggerID" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252",
       "triggerID" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278",
       "triggerID" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288",
       "triggerID" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30296",
       "triggerID" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff6ad63a811bad6c67be6cec61b2f6dd69f6b572",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ff6ad63a811bad6c67be6cec61b2f6dd69f6b572",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5219cc9984b4392eff658fdf342b1784e9b17aed Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278) 
   * f48082f47413dc6d3ace4b452afb0d7886e35621 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288) 
   * 8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30296) 
   * ff6ad63a811bad6c67be6cec61b2f6dd69f6b572 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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30242",
       "triggerID" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252",
       "triggerID" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278",
       "triggerID" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288",
       "triggerID" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30296",
       "triggerID" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5219cc9984b4392eff658fdf342b1784e9b17aed Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278) 
   * f48082f47413dc6d3ace4b452afb0d7886e35621 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288) 
   * 8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30296) 
   
   <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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30242",
       "triggerID" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b8a6710e7f17a8613051746080e009bc79361f00 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30242) 
   
   <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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30242",
       "triggerID" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252",
       "triggerID" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278",
       "triggerID" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288",
       "triggerID" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30296",
       "triggerID" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff6ad63a811bad6c67be6cec61b2f6dd69f6b572",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30301",
       "triggerID" : "ff6ad63a811bad6c67be6cec61b2f6dd69f6b572",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ff6ad63a811bad6c67be6cec61b2f6dd69f6b572 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30301) 
   
   <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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30242",
       "triggerID" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252",
       "triggerID" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278",
       "triggerID" : "5219cc9984b4392eff658fdf342b1784e9b17aed",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288",
       "triggerID" : "f48082f47413dc6d3ace4b452afb0d7886e35621",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e7d824ef0fe09b4b3576fde918a80cfb99069c1b Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30252) 
   * 5219cc9984b4392eff658fdf342b1784e9b17aed Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30278) 
   * f48082f47413dc6d3ace4b452afb0d7886e35621 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30288) 
   * 8d482c84f6d1b73acc9fc6bcfadd4825fb3c0616 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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30242",
       "triggerID" : "b8a6710e7f17a8613051746080e009bc79361f00",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e7d824ef0fe09b4b3576fde918a80cfb99069c1b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b8a6710e7f17a8613051746080e009bc79361f00 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30242) 
   * e7d824ef0fe09b4b3576fde918a80cfb99069c1b 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] twalthr commented on a change in pull request #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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



##########
File path: flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/utils/LogicalTypeCasts.java
##########
@@ -318,11 +318,17 @@ private static boolean supportsCasting(
             // cast between interval and exact numeric is only supported if interval has a single
             // field
             return isSingleFieldInterval(targetType);
+        } else if ((sourceType.is(CONSTRUCTED) || sourceType.is(STRUCTURED_TYPE))
+                && targetType.is(CHARACTER_STRING)) {
+            return supportsCollectionAndStructuredToStringCasting(sourceType, allowExplicit);

Review comment:
       if we would introduce new types, we need to check all utilities anyway. it is pretty obvious that a `toString` will be supported by those as well. I would suggest to keep the code simple.




-- 
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 #18524: [FLINK-25428][table-common][table-planner] Expose string casting for map, multiset, structured, row and raw

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



##########
File path: flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/LogicalTypeCastsTest.java
##########
@@ -254,39 +243,34 @@
                                                 new StructuredAttribute("diff", new TinyIntType())))
                                 .build(),
                         false,
-                        true
-                    },
+                        true),
 
-                    // raw to binary
-                    {
-                        new RawType(Integer.class, IntSerializer.INSTANCE),
+                // raw to binary
+                Arguments.of(
+                        new RawType<>(Integer.class, IntSerializer.INSTANCE),
                         new BinaryType(),
                         false,
-                        true
-                    },
-                });
+                        true),
+                // raw to binary
+                Arguments.of(
+                        new RawType<>(Integer.class, IntSerializer.INSTANCE),
+                        VarCharType.STRING_TYPE,
+                        false,
+                        true));
     }
 
-    @Parameter public LogicalType sourceType;
-
-    @Parameter(1)
-    public LogicalType targetType;
-
-    @Parameter(2)
-    public boolean supportsImplicit;
-
-    @Parameter(3)
-    public boolean supportsExplicit;
-
-    @Test
-    public void testImplicitCasting() {
+    @ParameterizedTest(name = "{index}: [From: {0}, To: {1}, Implicit: {2}, Explicit: {3}]")

Review comment:
       This is 100% personal preference: How about moving the actual test method on top of the class before the constructions of the parameterised stream? It just feels more natural to see first the actual test, but again that's my preference, no need to change.




-- 
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