nateab commented on code in PR #26719:
URL: https://github.com/apache/flink/pull/26719#discussion_r2178469627


##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala:
##########
@@ -1423,7 +1423,7 @@ object ScalarOperatorGens {
          |    $resultTerm = $nullTerm ? $defaultValue : $arrayGet;
          |    break;
          |  default:
-         |    throw new RuntimeException("Array has more than one element.");
+         |    throw new TableRuntimeException("Array has more than one 
element.");

Review Comment:
   We should use TableRuntimeException then I believe, since we don't know the 
actual size of the array until runtime. 
   
   I tried adding a test, but it's not straightforward to me how to test this 
runtime exception without breaking the testing framework
   
   ```
   +++ 
b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/ArrayTypeTest.scala
   @@ -21,7 +21,6 @@ import 
org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches
    import org.apache.flink.table.api._
    import org.apache.flink.table.planner.expressions.utils.ArrayTypeTestBase
    import org.apache.flink.table.planner.utils.DateTimeTestUtil.{localDate, 
localDateTime, localTime => gLocalTime}
   -
    import org.assertj.core.api.Assertions.assertThatThrownBy
    import org.junit.jupiter.api.Test
   
   @@ -242,6 +241,13 @@ class ArrayTypeTest extends ArrayTypeTestBase {
          "Array element access needs an index starting at 1 but was 0.")
      }
   
   +  @Test
   +  def testElement(): Unit = {
   +    testExpectedSqlException(
   +      "ELEMENT(f2)",
   +      "Array has more than one element.")
   +  }
   +
      @Test
      def testReturnNullWhenArrayIndexOutOfBounds(): Unit = {
        // ARRAY<INT NOT NULL>
   @@ -250,4 +256,11 @@ class ArrayTypeTest extends ArrayTypeTestBase {
        // ARRAY<INT>
        testAllApis('f11.at(3), "f11[4]", "NULL")
      }
   +
   +  @Test
   +  def testElementFailsOnMultiElementArray(): Unit = {
   +    testExpectedSqlException(
   +      "ELEMENT(ARRAY[1, 2])",
   +      "Array has more than one element.")
   +  }
    }
   ```
   
   These tests would fail with something like:
   ```
   java.lang.AssertionError: 
   Expecting actual throwable to be an instance of:
     org.apache.flink.table.api.ValidationException
   but was:
     java.lang.AssertionError: Error when executing the expression. Expression 
code:
   
         
         // Using option 'table.exec.legacy-cast-behaviour':'false'
         // Timezone: org.apache.flink.table.api.TableConfig@610fe5b7
             
         public class TestFunction$6
             extends org.apache.flink.api.common.functions.RichMapFunction {
   
           private transient 
org.apache.flink.table.runtime.typeutils.ArrayDataSerializer typeSerializer$1;
           org.apache.flink.table.data.binary.BinaryRowData out = new 
org.apache.flink.table.data.binary.BinaryRowData(1);
           org.apache.flink.table.data.writer.BinaryRowWriter outWriter = new 
org.apache.flink.table.data.writer.BinaryRowWriter(out);
   
           public TestFunction$6(Object[] references) throws Exception {
             typeSerializer$1 = 
(((org.apache.flink.table.runtime.typeutils.ArrayDataSerializer) 
references[0]));
           }
   
           
   
           @Override
           public void open(org.apache.flink.api.common.functions.OpenContext 
openContext) throws Exception {
             
           }
   
           @Override
           public Object map(Object _in1) throws Exception {
             org.apache.flink.table.data.RowData in1 = 
(org.apache.flink.table.data.RowData) _in1;
             
             org.apache.flink.table.data.ArrayData field$0;
             boolean isNull$0;
             org.apache.flink.table.data.ArrayData field$2;
             boolean isNull$4;
             org.apache.flink.table.data.binary.BinaryStringData result$5;
             
             isNull$0 = in1.isNullAt(2);
             field$0 = null;
             if (!isNull$0) {
               field$0 = in1.getArray(2);
             }
             field$2 = field$0;
             if (!isNull$0) {
               field$2 = (org.apache.flink.table.data.ArrayData) 
(typeSerializer$1.copy(field$2));
             }
                     
             
             
             
             outWriter.reset();
             
             
             
             boolean isNull$3;
             int result$3;
             switch (isNull$0 ? 0 : field$2.size()) {
               case 0:
                 isNull$3 = true;
                 result$3 = -1;
                 break;
               case 1:
                 isNull$3 = field$2.isNullAt(0);
                 result$3 = isNull$3 ? -1 : field$2.getInt(0);
                 break;
               default:
                 throw new 
org.apache.flink.table.api.ValidationException("Array has more than one 
element.");
             }
             
              // --- Cast section generated by 
org.apache.flink.table.planner.functions.casting.CharVarCharTrimPadCastRule
              isNull$4 = isNull$3;
             if (!isNull$4) {
             result$5 = 
org.apache.flink.table.data.binary.BinaryStringData.fromString("" + result$3);
             isNull$4 = result$5 == null;
             } else {
             result$5 = 
org.apache.flink.table.data.binary.BinaryStringData.EMPTY_UTF8;
             }
             
              // --- End cast section
                            
             if (isNull$4) {
               outWriter.setNullAt(0);
             } else {
               outWriter.writeString(0, result$5);
             }
                        
             outWriter.complete();
                     
             return out;
                     
           }
   
           @Override
           public void close() throws Exception {
             
           }
   
           
         }
       
        at 
org.apache.flink.table.planner.expressions.utils.ExpressionTestBase.evaluateFunctionResult(ExpressionTestBase.scala:285)
        at 
org.apache.flink.table.planner.expressions.utils.ExpressionTestBase.evaluateGivenExprs(ExpressionTestBase.scala:346)
        at 
org.apache.flink.table.planner.expressions.utils.ExpressionTestBase.$anonfun$evaluateExprs$2(ExpressionTestBase.scala:149)
        ...(73 remaining lines not displayed - this can be changed with 
Assertions.setMaxStackTraceElementsDisplayed)
   
        at 
org.apache.flink.table.planner.expressions.utils.ExpressionTestBase.$anonfun$evaluateExprs$1(ExpressionTestBase.scala:153)
        at 
scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62)
        at 
scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55)
        at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49)
        at 
org.apache.flink.table.planner.expressions.utils.ExpressionTestBase.evaluateExprs(ExpressionTestBase.scala:144)
        at java.base/java.lang.reflect.Method.invoke(Method.java:568)
        at 
java.base/java.util.concurrent.ForkJoinTask.doExec$$$capture(ForkJoinTask.java:373)
        at java.base/java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java)
   ```
   
   Since existing tests pass with this change, could we skip adding a test 
since it is a small 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-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to