You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by ja...@apache.org on 2020/06/04 14:45:23 UTC

[flink] branch master updated: [FLINK-17847][table-planner-blink] Fix runtime ArrayIndexOutOfBoundsException when accessing out-of-bounds elements of ARRAY data

This is an automated email from the ASF dual-hosted git repository.

jark pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
     new 3ee4c1f  [FLINK-17847][table-planner-blink] Fix runtime ArrayIndexOutOfBoundsException when accessing out-of-bounds elements of ARRAY data
3ee4c1f is described below

commit 3ee4c1f0a094d7562f4188b9fc93a2313e872cde
Author: Leonard Xu <xb...@gmail.com>
AuthorDate: Tue Jun 2 16:18:32 2020 +0800

    [FLINK-17847][table-planner-blink] Fix runtime ArrayIndexOutOfBoundsException when accessing out-of-bounds elements of ARRAY data
    
    This changes the behavior of ARRAY element accessing:
    - if the accessing index is a literal which is less than 1, then an error will be thrown before job is submitted with readable exception message
    - if the accessing index is out-of-bound during runtime, then return null instead of throwing exception
    
    This closes #12436
---
 .../planner/codegen/calls/ScalarOperatorGens.scala | 25 ++++++++++++-----
 .../table/planner/expressions/ArrayTypeTest.scala  | 31 ++++++++++++++++++++++
 2 files changed, 49 insertions(+), 7 deletions(-)

diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala
index ff91ed2..e5e229c 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala
@@ -18,6 +18,7 @@
 
 package org.apache.flink.table.planner.codegen.calls
 
+import org.apache.flink.table.api.ValidationException
 import org.apache.flink.table.data.binary.BinaryArrayData
 import org.apache.flink.table.data.util.{DataFormatConverters, MapDataUtil}
 import org.apache.flink.table.data.writer.{BinaryArrayWriter, BinaryRowWriter}
@@ -1634,6 +1635,10 @@ object ScalarOperatorGens {
     GeneratedExpression(arrayTerm, GeneratedExpression.NEVER_NULL, code, arrayType)
   }
 
+  /**
+   * Return null when array index out of bounds which follows Calcite's behaviour.
+   * @see [[org.apache.calcite.sql.fun.SqlStdOperatorTable.ITEM]]
+   */
   def generateArrayElementAt(
       ctx: CodeGeneratorContext,
       array: GeneratedExpression,
@@ -1643,19 +1648,25 @@ object ScalarOperatorGens {
     val resultTypeTerm = primitiveTypeTermForType(componentInfo)
     val defaultTerm = primitiveDefaultValue(componentInfo)
 
+    index.literalValue match {
+      case Some(v: Int) if v < 1 =>
+        throw new ValidationException(
+          s"Array element access needs an index starting at 1 but was $v.")
+      case _ => //nothing
+    }
     val idxStr = s"${index.resultTerm} - 1"
     val arrayIsNull = s"${array.resultTerm}.isNullAt($idxStr)"
     val arrayGet =
       rowFieldReadAccess(ctx, idxStr, array.resultTerm, componentInfo)
 
     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}
+        |boolean $nullTerm = ${array.nullTerm} || ${index.nullTerm} ||
+        |   $idxStr < 0 || $idxStr >= ${array.resultTerm}.size() || $arrayIsNull;
+        |$resultTypeTerm $resultTerm = $nullTerm ? $defaultTerm : $arrayGet;
+        |""".stripMargin
     GeneratedExpression(resultTerm, nullTerm, arrayAccessCode, componentInfo)
   }
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/ArrayTypeTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/ArrayTypeTest.scala
index cfc798d..0c13200 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/ArrayTypeTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/ArrayTypeTest.scala
@@ -375,4 +375,35 @@ class ArrayTypeTest extends ArrayTypeTestBase {
       "[1984-03-12, 1984-02-10]"
     )
   }
+
+  @Test
+  def testArrayIndexStaticCheckForTable(): Unit = {
+    thrown.expect(classOf[ValidationException])
+    thrown.expectMessage("Array element access needs an index starting at 1 but was 0.")
+    testTableApi('f2.at(0), "1")
+  }
+
+  @Test
+  def testArrayIndexStaticCheckForSql(): Unit = {
+    thrown.expect(classOf[ValidationException])
+    thrown.expectMessage("Array element access needs an index starting at 1 but was 0.")
+    testSqlApi("f2[0]", "1")
+  }
+
+  @Test
+  def testReturnNullWhenArrayIndexOutOfBounds(): Unit = {
+    // ARRAY<INT NOT NULL>
+    testAllApis(
+      'f2.at(4),
+      "f2.at(4)",
+      "f2[4]",
+      "null")
+
+    // ARRAY<INT>
+    testAllApis(
+      'f11.at(3),
+      "f11.at(3)",
+      "f11[4]",
+      "null")
+  }
 }