xuyangzhong commented on code in PR #27330:
URL: https://github.com/apache/flink/pull/27330#discussion_r2726271959


##########
flink-table/flink-sql-parser/src/main/java/org/apache/calcite/sql/fun/SqlItemOperator.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperandCountRange;
+import org.apache.calcite.sql.SqlOperatorBinding;
+import org.apache.calcite.sql.SqlSpecialOperator;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.SqlOperandCountRanges;
+import org.apache.calcite.sql.type.SqlSingleOperandTypeChecker;
+import org.apache.calcite.sql.type.SqlTypeFamily;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+
+import java.util.Arrays;
+
+import static java.util.Objects.requireNonNull;
+import static 
org.apache.calcite.sql.type.NonNullableAccessors.getComponentTypeOrThrow;
+import static 
org.apache.calcite.sql.validate.SqlNonNullableAccessors.getOperandLiteralValueOrThrow;
+
+/**
+ * The item operator {@code [ ... ]}, used to access a given element of an 
array, map or struct. For
+ * example, {@code myArray[3]}, {@code "myMap['foo']"}, {@code myStruct[2]} or 
{@code
+ * myStruct['fieldName']}.
+ *
+ * <p>This class was copied over from Calcite 1.39.0 version to support access 
variant
+ * (FLINK-37924).
+ *
+ * <p>Line 148, CALCITE-7325, should be removed after upgrading Calcite to 
1.42.0.
+ */
+public class SqlItemOperator extends SqlSpecialOperator {
+    public final int offset;
+    public final boolean safe;
+
+    public SqlItemOperator(
+            String name, SqlSingleOperandTypeChecker operandTypeChecker, int 
offset, boolean safe) {
+        super(name, SqlKind.ITEM, 100, true, null, null, operandTypeChecker);
+        this.offset = offset;
+        this.safe = safe;
+    }
+
+    @Override
+    public ReduceResult reduceExpr(int ordinal, TokenSequence list) {
+        SqlNode left = list.node(ordinal - 1);
+        SqlNode right = list.node(ordinal + 1);
+        return new ReduceResult(
+                ordinal - 1,
+                ordinal + 2,
+                createCall(
+                        SqlParserPos.sum(
+                                Arrays.asList(
+                                        requireNonNull(left, 
"left").getParserPosition(),
+                                        requireNonNull(right, 
"right").getParserPosition(),
+                                        list.pos(ordinal))),
+                        left,
+                        right));
+    }
+
+    @Override
+    public void unparse(SqlWriter writer, SqlCall call, int leftPrec, int 
rightPrec) {
+        call.operand(0).unparse(writer, leftPrec, 0);
+        final SqlWriter.Frame frame = writer.startList("[", "]");
+        if (!this.getName().equals("ITEM")) {
+            final SqlWriter.Frame offsetFrame = 
writer.startFunCall(this.getName());
+            call.operand(1).unparse(writer, 0, 0);
+            writer.endFunCall(offsetFrame);
+        } else {
+            call.operand(1).unparse(writer, 0, 0);
+        }
+        writer.endList(frame);
+    }
+
+    @Override
+    public SqlOperandCountRange getOperandCountRange() {
+        return SqlOperandCountRanges.of(2);
+    }
+
+    @Override
+    public boolean checkOperandTypes(SqlCallBinding callBinding, boolean 
throwOnFailure) {
+        final SqlNode left = callBinding.operand(0);
+        final SqlNode right = callBinding.operand(1);
+        if (!getOperandTypeChecker().checkSingleOperandType(callBinding, left, 
0, throwOnFailure)) {
+            return false;
+        }
+        final SqlSingleOperandTypeChecker checker = getChecker(callBinding);
+        return checker.checkSingleOperandType(callBinding, right, 0, 
throwOnFailure);
+    }
+
+    @Override
+    public SqlSingleOperandTypeChecker getOperandTypeChecker() {
+        return (SqlSingleOperandTypeChecker)
+                requireNonNull(super.getOperandTypeChecker(), 
"operandTypeChecker");
+    }
+
+    private static SqlSingleOperandTypeChecker getChecker(SqlCallBinding 
callBinding) {
+        final RelDataType operandType = callBinding.getOperandType(0);
+        switch (operandType.getSqlTypeName()) {
+            case ARRAY:
+                return OperandTypes.family(SqlTypeFamily.INTEGER);
+            case MAP:
+                RelDataType keyType =
+                        requireNonNull(operandType.getKeyType(), 
"operandType.getKeyType()");
+                SqlTypeName sqlTypeName = keyType.getSqlTypeName();
+                return OperandTypes.family(
+                        requireNonNull(
+                                sqlTypeName.getFamily(),
+                                () ->
+                                        "keyType.getSqlTypeName().getFamily() 
null, type is "
+                                                + sqlTypeName));
+            case ROW:
+            case ANY:
+            case DYNAMIC_STAR:
+            case VARIANT:
+                return OperandTypes.family(SqlTypeFamily.INTEGER)
+                        .or(OperandTypes.family(SqlTypeFamily.CHARACTER));
+            default:
+                throw callBinding.newValidationSignatureError();
+        }
+    }
+
+    @Override
+    public String getAllowedSignatures(String name) {
+        if (name.equals("ITEM")) {
+            return "<ARRAY>[<INTEGER>]\n"

Review Comment:
   nit replace it with the following and update the changed line number scope 
in class annotation.
   ```
   // FLINK MODIFICATION BEGIN
   return "<ARRAY>[<INTEGER>]\n"
                       + "<MAP>[<ANY>]\n"
                       + "<ROW>[<CHARACTER>|<INTEGER>]\n"
                       + "<VARIANT>[<CHARACTER>|<INTEGER>]";
   // FLINK MODIFICATION END
   ```



##########
flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/ItemAtIndexArgumentTypeStrategy.java:
##########
@@ -41,7 +41,10 @@
  * or {@link LogicalTypeRoot#MULTISET}
  *
  * <p>the type to be equal to the key type of {@link LogicalTypeRoot#MAP} if 
the first argument is a
- * map.
+ * map
+ *
+ * <p>a {@link LogicalTypeFamily#NUMERIC} type or 
{LogicalTypeFamily.CHARACTER_STRING} type if the

Review Comment:
   nit `{@link LogicalTypeFamily#CHARACTER_STRING}`



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala:
##########
@@ -1668,6 +1668,100 @@ object ScalarOperatorGens {
     generateUnaryOperatorIfNotNull(ctx, resultType, map)(_ => 
s"${map.resultTerm}.size()")
   }
 
+  def generateVariantGet(
+      ctx: CodeGeneratorContext,
+      variant: GeneratedExpression,
+      key: GeneratedExpression): GeneratedExpression = {
+    val Seq(resultTerm, nullTerm) = newNames(ctx, "result", "isNull")
+    val tmpValue = newName(ctx, "tmpValue")
+
+    val variantType = variant.resultType.asInstanceOf[VariantType]
+    val variantTerm = variant.resultTerm
+    val variantTypeTerm = primitiveTypeTermForType(variantType)
+    val variantDefaultTerm = primitiveDefaultValue(variantType)
+
+    val keyTerm = key.resultTerm
+    val keyType = key.resultType
+
+    val accessCode = if (isInteger(keyType)) {

Review Comment:
   Should we align this with 
`indexType.getLogicalType().is(LogicalTypeFamily.INTEGER_NUMERIC)` in 
`ItemAtIndexArgumentTypeStrategy`?  Some `INTEGER_NUMERIC` types like smallint 
and bigint will pass the check in type strategy but fail here.



##########
docs/data/sql_functions.yml:
##########
@@ -1167,6 +1167,16 @@ variant:
       parser will keep the last occurrence of all fields with the same key, 
otherwise when 
       `allowDuplicateKeys` is false it will throw an error. The default value 
of 
       `allowDuplicateKeys` is false.
+  - sql: variant '[' INT ']'
+    table: VARIANT.at(INT)
+    description: |
+      If the VARIANT is an ARRAY value, returns a VARIANT whose value is the 
element at

Review Comment:
   I think we should also mention the index starts from 1 like `array '[' INT 
']'` because most developers might think they're starting from 0.



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