Shuai Xu created FLINK-36654:
--------------------------------

             Summary: Decimal divide Integer reports Null pointer exception
                 Key: FLINK-36654
                 URL: https://issues.apache.org/jira/browse/FLINK-36654
             Project: Flink
          Issue Type: Bug
          Components: Table SQL / Planner
    Affects Versions: 2.0-preview
            Reporter: Shuai Xu


paste the test code in SqlExpressionTest.

 
{code:java}
// code placeholder
// Decimal(2,1) / Integer => Decimal(13,12)
testSqlApi("1.0/400", "0.002500000000")

// Decimal(2,1) / BigInteger => Decimal(22,21)
testSqlApi("1.0/100000000000", "0.000000000010000000000")

// Decimal(2,1) / TinyInt => Decimal(7,6)
testSqlApi("1.0/cast(100 as TINYINT)", "0.010000")

// Decimal(2,1) / SmallInt => Decimal(8,7)
testSqlApi("1.0/cast(10000 as SMALLINT)", "0.0001000") {code}
and get exception 

 

 
{code:java}
org.apache.flink.table.planner.expressions.SqlExpressionTest,testDivide
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@61150d94                public 
class TestFunction$20          extends 
org.apache.flink.api.common.functions.RichMapFunction {
                org.apache.flink.table.data.DecimalData decimal$0 = 
org.apache.flink.table.data.DecimalDataUtils.castFrom(        "1.0",        2,  
      1);                        org.apache.flink.table.data.DecimalData 
decimal$5 = org.apache.flink.table.data.DecimalDataUtils.castFrom(        
"1.0",        2,        1);                        
org.apache.flink.table.data.DecimalData decimal$10 = 
org.apache.flink.table.data.DecimalDataUtils.castFrom(        "1.0",        2,  
      1);                        org.apache.flink.table.data.DecimalData 
decimal$15 = org.apache.flink.table.data.DecimalDataUtils.castFrom(        
"1.0",        2,        1);                
org.apache.flink.table.data.binary.BinaryRowData out = new 
org.apache.flink.table.data.binary.BinaryRowData(4);        
org.apache.flink.table.data.writer.BinaryRowWriter outWriter = new 
org.apache.flink.table.data.writer.BinaryRowWriter(out);
        public TestFunction$20(Object[] references) throws Exception {          
        }
        
        @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;                    boolean 
isNull$1;          org.apache.flink.table.data.DecimalData result$2;          
boolean isNull$3;          org.apache.flink.table.data.binary.BinaryStringData 
result$4;          boolean isNull$6;          
org.apache.flink.table.data.DecimalData result$7;          boolean isNull$8;    
      org.apache.flink.table.data.binary.BinaryStringData result$9;          
boolean isNull$11;          org.apache.flink.table.data.DecimalData result$12;  
        boolean isNull$13;          
org.apache.flink.table.data.binary.BinaryStringData result$14;          boolean 
isNull$16;          org.apache.flink.table.data.DecimalData result$17;          
boolean isNull$18;          org.apache.flink.table.data.binary.BinaryStringData 
result$19;                                                  outWriter.reset();  
                                                isNull$1 = false || false;      
    result$2 = null;          if (!isNull$1) {                                
result$2 = 
org.apache.flink.table.data.DecimalDataUtils.divide(((org.apache.flink.table.data.DecimalData)
 decimal$0), 
org.apache.flink.table.data.DecimalDataUtils.castFrom(((long)(((int) 400))), 
13, 12), 13, 12);                      isNull$1 = (result$2 == null);          
}                     // --- Cast section generated by 
org.apache.flink.table.planner.functions.casting.CharVarCharTrimPadCastRule     
      isNull$3 = isNull$1;          if (!isNull$3) {          result$4 = 
org.apache.flink.table.data.binary.BinaryStringData.fromString("" + result$2);  
        isNull$3 = result$4 == null;          } else {          result$4 = 
org.apache.flink.table.data.binary.BinaryStringData.EMPTY_UTF8;          }      
               // --- End cast section                                   if 
(isNull$3) {            outWriter.setNullAt(0);          } else {            
outWriter.writeString(0, result$4);          }                                  
                                     isNull$6 = false || false;          
result$7 = null;          if (!isNull$6) {                                
result$7 = 
org.apache.flink.table.data.DecimalDataUtils.divide(((org.apache.flink.table.data.DecimalData)
 decimal$5), 
org.apache.flink.table.data.DecimalDataUtils.castFrom(((long)(((long) 
100000000000L))), 22, 21), 22, 21);                      isNull$6 = (result$7 
== null);          }                     // --- Cast section generated by 
org.apache.flink.table.planner.functions.casting.CharVarCharTrimPadCastRule     
      isNull$8 = isNull$6;          if (!isNull$8) {          result$9 = 
org.apache.flink.table.data.binary.BinaryStringData.fromString("" + result$7);  
        isNull$8 = result$9 == null;          } else {          result$9 = 
org.apache.flink.table.data.binary.BinaryStringData.EMPTY_UTF8;          }      
               // --- End cast section                                   if 
(isNull$8) {            outWriter.setNullAt(1);          } else {            
outWriter.writeString(1, result$9);          }                                  
                                     isNull$11 = false || false;          
result$12 = null;          if (!isNull$11) {                                
result$12 = 
org.apache.flink.table.data.DecimalDataUtils.divide(((org.apache.flink.table.data.DecimalData)
 decimal$10), 
org.apache.flink.table.data.DecimalDataUtils.castFrom(((long)(((byte) 
((byte)100)))), 7, 6), 7, 6);                      isNull$11 = (result$12 == 
null);          }                     // --- Cast section generated by 
org.apache.flink.table.planner.functions.casting.CharVarCharTrimPadCastRule     
      isNull$13 = isNull$11;          if (!isNull$13) {          result$14 = 
org.apache.flink.table.data.binary.BinaryStringData.fromString("" + result$12); 
         isNull$13 = result$14 == null;          } else {          result$14 = 
org.apache.flink.table.data.binary.BinaryStringData.EMPTY_UTF8;          }      
               // --- End cast section                                   if 
(isNull$13) {            outWriter.setNullAt(2);          } else {            
outWriter.writeString(2, result$14);          }                                 
                                      isNull$16 = false || false;          
result$17 = null;          if (!isNull$16) {                                
result$17 = 
org.apache.flink.table.data.DecimalDataUtils.divide(((org.apache.flink.table.data.DecimalData)
 decimal$15), 
org.apache.flink.table.data.DecimalDataUtils.castFrom(((long)(((short) 
((short)10000)))), 8, 7), 8, 7);                      isNull$16 = (result$17 == 
null);          }                     // --- Cast section generated by 
org.apache.flink.table.planner.functions.casting.CharVarCharTrimPadCastRule     
      isNull$18 = isNull$16;          if (!isNull$18) {          result$19 = 
org.apache.flink.table.data.binary.BinaryStringData.fromString("" + result$17); 
         isNull$18 = result$19 == null;          } else {          result$19 = 
org.apache.flink.table.data.binary.BinaryStringData.EMPTY_UTF8;          }      
               // --- End cast section                                   if 
(isNull$18) {            outWriter.setNullAt(3);          } else {            
outWriter.writeString(3, result$19);          }                               
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.evaluateExprs(ExpressionTestBase.scala:141)
      at java.base/java.lang.reflect.Method.invoke(Method.java:566)   at 
java.base/java.util.concurrent.RecursiveAction.exec(RecursiveAction.java:189)   
     at 
java.base/java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java:290)    at 
java.base/java.util.concurrent.ForkJoinPool$WorkQueue.topLevelExec(ForkJoinPool.java:1020)
   at java.base/java.util.concurrent.ForkJoinPool.scan(ForkJoinPool.java:1656)  
   at 
java.base/java.util.concurrent.ForkJoinPool.runWorker(ForkJoinPool.java:1594)   
     at 
java.base/java.util.concurrent.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:177)Caused
 by: java.lang.NullPointerException      at 
org.apache.flink.table.data.DecimalDataUtils.divide(DecimalDataUtils.java:146)  
     at TestFunction$20.mapImpl_split2(Unknown Source)       at 
TestFunction$20.mapImpl(Unknown Source)      at TestFunction$20.map(Unknown 
Source)  at 
org.apache.flink.table.planner.expressions.utils.ExpressionTestBase.evaluateFunctionResult(ExpressionTestBase.scala:260)
     ... 9 more

Process finished with exit code 255
 {code}
 



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to