leonardBang commented on a change in pull request #12436:
URL: https://github.com/apache/flink/pull/12436#discussion_r433724667
##########
File path:
flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala
##########
@@ -1643,19 +1644,35 @@ object ScalarOperatorGens {
val resultTypeTerm = primitiveTypeTermForType(componentInfo)
val defaultTerm = primitiveDefaultValue(componentInfo)
+ if (index.literalValue.isDefined &&
+ index.literalValue.get.isInstanceOf[Int] &&
+ index.literalValue.get.asInstanceOf[Int] < 1) {
+ throw new ValidationException(s"Array element access needs an index
starting at 1 but was " +
+ s"${index.literalValue.get.asInstanceOf[Int]}.")
+ }
val idxStr = s"${index.resultTerm} - 1"
val arrayIsNull = s"${array.resultTerm}.isNullAt($idxStr)"
val arrayGet =
rowFieldReadAccess(ctx, idxStr, array.resultTerm, componentInfo)
+ /**
+ * Return null when array index out of bounds which follows Calcite's
behaviour.
+ * @see org.apache.calcite.sql.fun.SqlStdOperatorTable
+ */
val arrayAccessCode =
- s"""
- |${array.code}
- |${index.code}
- |boolean $nullTerm = ${array.nullTerm} || ${index.nullTerm} ||
$arrayIsNull;
- |$resultTypeTerm $resultTerm = $nullTerm ? $defaultTerm : $arrayGet;
- |""".stripMargin
-
+ s"""
+ |${array.code}
+ |${index.code}
+ |$resultTypeTerm $resultTerm;
+ |boolean $nullTerm;
+ |if (${idxStr} < 0 || ${idxStr} >= ${array.resultTerm}.size()) {
Review comment:
I think the `idxStr` has a certain value in runtime, both for `idx < 0`
and 'idx >= array.size' we should return null.
----------------------------------------------------------------
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.
For queries about this service, please contact Infrastructure at:
[email protected]