slinkydeveloper commented on a change in pull request #18063:
URL: https://github.com/apache/flink/pull/18063#discussion_r765770613



##########
File path: 
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/AnyToCharVarchar.java
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.flink.table.planner.functions.casting;
+
+import org.apache.flink.table.data.StringData;
+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.VarCharType;
+import org.apache.flink.table.types.logical.utils.LogicalTypeChecks;
+
+import static 
org.apache.flink.table.planner.functions.casting.CastRuleUtils.methodCall;
+import static 
org.apache.flink.table.planner.functions.casting.CastRuleUtils.shouldPossiblyTrim;
+import static 
org.apache.flink.table.types.logical.utils.LogicalTypeChecks.isStringType;
+
+/**
+ * {@link LogicalTypeFamily#CHARACTER_STRING} to {@link 
LogicalTypeFamily#BINARY_STRING} cast rule.
+ */

Review comment:
       Please more than using the standard javadoc for cast rules explain the 
purpose of this rule separately

##########
File path: 
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/AnyToCharVarchar.java
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.flink.table.planner.functions.casting;
+
+import org.apache.flink.table.data.StringData;
+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.VarCharType;
+import org.apache.flink.table.types.logical.utils.LogicalTypeChecks;
+
+import static 
org.apache.flink.table.planner.functions.casting.CastRuleUtils.methodCall;
+import static 
org.apache.flink.table.planner.functions.casting.CastRuleUtils.shouldPossiblyTrim;
+import static 
org.apache.flink.table.types.logical.utils.LogicalTypeChecks.isStringType;
+
+/**
+ * {@link LogicalTypeFamily#CHARACTER_STRING} to {@link 
LogicalTypeFamily#BINARY_STRING} cast rule.
+ */
+class AnyToCharVarchar extends 
AbstractNullAwareCodeGeneratorCastRule<StringData, StringData> {
+
+    static final AnyToCharVarchar INSTANCE = new AnyToCharVarchar();
+
+    private AnyToCharVarchar() {
+        super(
+                CastRulePredicate.builder()
+                        .predicate(
+                                (inputType, targetType) ->
+                                        !inputType.is(LogicalTypeRoot.ARRAY)
+                                                && 
targetType.is(LogicalTypeFamily.CHARACTER_STRING)
+                                                && !isStringType(targetType))
+                        .build());
+    }
+
+    /* Example generated code for STRING() -> VARCHAR(4) cast
+
+    isNull$0 = _myInputIsNull;
+    if (!isNull$0) {
+        if (_myInput.numChars() > 4) {
+            result$1 = (_myInput).substring(0, 4);
+        } else {
+            result$1 = _myInput;
+        }
+        isNull$0 = result$1 == null;
+    } else {
+        result$1 = 
org.apache.flink.table.data.binary.BinaryStringData.EMPTY_UTF8;
+    }
+
+    */
+    @Override
+    protected String generateCodeBlockInternal(
+            CodeGeneratorCastRule.Context context,
+            String inputTerm,
+            String returnVariable,
+            LogicalType inputLogicalType,
+            LogicalType targetLogicalType) {
+        final int precision = LogicalTypeChecks.getLength(targetLogicalType);
+        CastRule<?, ?> castRule =
+                CastRuleProvider.resolve(inputLogicalType, 
VarCharType.STRING_TYPE);
+
+        // Only used for non-Constructed types - for constructed, the 
trimming/padding is applied
+        // on each individual rule, i.e.: ArrayToStringCastRule
+        if (castRule instanceof ExpressionCodeGeneratorCastRule) {
+            @SuppressWarnings("rawtypes")
+            final String stringExpr =
+                    ((ExpressionCodeGeneratorCastRule) castRule)
+                            .generateExpression(
+                                    context, inputTerm, inputLogicalType, 
targetLogicalType);
+
+            CastRuleUtils.CodeWriter writer = new CastRuleUtils.CodeWriter();
+            if (context.legacyBehaviour() || !shouldPossiblyTrim(precision)) {
+                return writer.assignStmt(returnVariable, 
stringExpr).toString();
+            }
+            return writer.ifStmt(
+                            methodCall(stringExpr, "numChars") + " > " + 
precision,
+                            thenWriter ->
+                                    thenWriter.assignStmt(
+                                            returnVariable,
+                                            methodCall(
+                                                    "(" + stringExpr + ")",
+                                                    "substring",
+                                                    0,
+                                                    precision)),
+                            elseWriter -> 
elseWriter.assignStmt(returnVariable, stringExpr))
+                    .toString();
+        } else {
+            throw new IllegalStateException("This is a bug. Please file an 
issue.");

Review comment:
       I think here you still need to support 
non-`ExpressionCodeGeneratorCastRule`, for row, raw and multiset/map cast 
rules. For invoking "plain" `CodeGeneratorCastRule` perhaps just create a new 
context, where each new variable is added in `CodeWriter` here rather than on 
the context passed to this function

##########
File path: 
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/AnyToCharVarchar.java
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.flink.table.planner.functions.casting;
+
+import org.apache.flink.table.data.StringData;
+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.VarCharType;
+import org.apache.flink.table.types.logical.utils.LogicalTypeChecks;
+
+import static 
org.apache.flink.table.planner.functions.casting.CastRuleUtils.methodCall;
+import static 
org.apache.flink.table.planner.functions.casting.CastRuleUtils.shouldPossiblyTrim;
+import static 
org.apache.flink.table.types.logical.utils.LogicalTypeChecks.isStringType;
+
+/**
+ * {@link LogicalTypeFamily#CHARACTER_STRING} to {@link 
LogicalTypeFamily#BINARY_STRING} cast rule.
+ */
+class AnyToCharVarchar extends 
AbstractNullAwareCodeGeneratorCastRule<StringData, StringData> {

Review comment:
       Perhaps rename `CharacterFamilyTrimmingCastRule`, or something else? 
`AnyToCharVarchar` is unclear to me, and it's not really any as constructed 
types are not covered by this rule.

##########
File path: 
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/CastRuleUtils.java
##########
@@ -163,6 +166,25 @@ static String binaryWriterWriteNull(
         return CodeGenUtils.binaryWriterWriteNull(indexTerm, writerTerm, 
logicalType);
     }
 
+    static String lengthExceedsPrecision(String strTerm, int precision) {
+        return methodCall(strTerm, "length") + " > " + precision;
+    }
+
+    /**
+     * For complex types such as {@link LogicalTypeRoot#ARRAY}, use {@link 
VarCharType#STRING_TYPE}
+     * to avoid trimming/padding when casting the inner elements to a {@link
+     * LogicalTypeRoot#VARCHAR} or {@link LogicalTypeRoot#CHAR}.
+     */
+    static LogicalType getLogicalTypeForInnerCast(LogicalType logicalType) {
+        return logicalType.is(LogicalTypeFamily.CHARACTER_STRING)
+                ? VarCharType.STRING_TYPE
+                : logicalType;
+    }
+
+    static boolean shouldPossiblyTrim(int precision) {
+        return precision < VarCharType.MAX_LENGTH;
+    }

Review comment:
       This class has only code gen methods, not predicates. Can we place these 
somewhere else?

##########
File path: 
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/CastRuleUtils.java
##########
@@ -296,6 +318,35 @@ public CodeWriter appendBlock(String codeBlock) {
             return this;
         }
 
+        public CodeWriter appendAndTrimStringIfNeeded(
+                boolean legacyBehaviour,
+                int precision,
+                String resultStringTerm,
+                String builderTerm) {
+            CodeWriter writer =
+                    declStmt(String.class, resultStringTerm)
+                            .assignStmt(resultStringTerm, 
methodCall(builderTerm, "toString"));
+
+            // Trim if needed
+            if (!legacyBehaviour && shouldPossiblyTrim(precision)) {
+                writer.ifStmt(
+                        lengthExceedsPrecision(builderTerm, precision),
+                        thenWriter ->
+                                thenWriter.assignStmt(
+                                        resultStringTerm,
+                                        methodCall(
+                                                builderTerm,
+                                                "substring",
+                                                0,
+                                                staticCall(
+                                                        Math.class,
+                                                        "min",
+                                                        
methodCall(builderTerm, "length"),
+                                                        precision))));
+            }
+            return writer;
+        }

Review comment:
       I don't like much this method, it's doing way too many things, in 
particular in this context or `CodeWriter` instance methods where my goal is to 
just have basic java constructs, and not our own cast related logic.
   
   Perhaps move this to `AnyToCharVarchar` as static method, and make the 
length predicate a parameter?

##########
File path: 
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/AnyToCharVarchar.java
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.flink.table.planner.functions.casting;
+
+import org.apache.flink.table.data.StringData;
+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.VarCharType;
+import org.apache.flink.table.types.logical.utils.LogicalTypeChecks;
+
+import static 
org.apache.flink.table.planner.functions.casting.CastRuleUtils.methodCall;
+import static 
org.apache.flink.table.planner.functions.casting.CastRuleUtils.shouldPossiblyTrim;
+import static 
org.apache.flink.table.types.logical.utils.LogicalTypeChecks.isStringType;
+
+/**
+ * {@link LogicalTypeFamily#CHARACTER_STRING} to {@link 
LogicalTypeFamily#BINARY_STRING} cast rule.
+ */
+class AnyToCharVarchar extends 
AbstractNullAwareCodeGeneratorCastRule<StringData, StringData> {

Review comment:
       I like the idea of this class, composing more than inheriting the 
behaviour!

##########
File path: 
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/CastRuleUtils.java
##########
@@ -163,6 +166,25 @@ static String binaryWriterWriteNull(
         return CodeGenUtils.binaryWriterWriteNull(indexTerm, writerTerm, 
logicalType);
     }
 
+    static String lengthExceedsPrecision(String strTerm, int precision) {
+        return methodCall(strTerm, "length") + " > " + precision;
+    }
+
+    /**
+     * For complex types such as {@link LogicalTypeRoot#ARRAY}, use {@link 
VarCharType#STRING_TYPE}
+     * to avoid trimming/padding when casting the inner elements to a {@link
+     * LogicalTypeRoot#VARCHAR} or {@link LogicalTypeRoot#CHAR}.
+     */
+    static LogicalType getLogicalTypeForInnerCast(LogicalType logicalType) {
+        return logicalType.is(LogicalTypeFamily.CHARACTER_STRING)
+                ? VarCharType.STRING_TYPE
+                : logicalType;
+    }

Review comment:
       Why you need this? In every call site you use it, you're always casting 
to `CHARACTER_STRING`, so you'll never fall back to the original type




-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to