[ 
https://issues.apache.org/jira/browse/FLINK-30499?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Runkang He updated FLINK-30499:
-------------------------------
    Description: 
There is a bug in batch over window operator codegen, the comparator codegen 
logic is wrong that member code incorrectly used in method body. This causes 
over window code compile failed when operator initialization.

The incorrect generated code for RangeBoundComparator is below, and the wrong 
place is *private static final java.util.TimeZone timeZone = 
java.util.TimeZone.getTimeZone("Asia/Shanghai");*

which is a member statments that should be placed in class body, not method 
body.
{code:java}
      public class RangeBoundComparator$40 implements 
org.apache.flink.table.runtime.generated.RecordComparator {

        private final Object[] references;
        

        public RangeBoundComparator$40(Object[] references) {
          this.references = references;
          
          
        }

        @Override
        public int compare(org.apache.flink.table.data.RowData in1, 
org.apache.flink.table.data.RowData in2) {
          
                  org.apache.flink.table.data.TimestampData field$41;
                  boolean isNull$41;
                  org.apache.flink.table.data.TimestampData field$42;
                  boolean isNull$42;
                  isNull$41 = in1.isNullAt(0);
                  field$41 = null;
                  if (!isNull$41) {
                    field$41 = in1.getTimestamp(0, 3);
                  }
                  isNull$42 = in2.isNullAt(0);
                  field$42 = null;
                  if (!isNull$42) {
                    field$42 = in2.getTimestamp(0, 3);
                  }
                  if (isNull$41 && isNull$42) {
                     return 1;
                  } else if (isNull$41 || isNull$42) {
                     return -1;
                  } else {
                     
                            private static final java.util.TimeZone timeZone =
                                             
java.util.TimeZone.getTimeZone("Asia/Shanghai");
                            org.apache.flink.table.data.TimestampData result$43;
                            boolean isNull$43;
                            boolean isNull$44;
                            org.apache.flink.table.data.binary.BinaryStringData 
result$45;
                            boolean isNull$46;
                            long result$47;
                            org.apache.flink.table.data.TimestampData result$48;
                            boolean isNull$48;
                            boolean isNull$49;
                            org.apache.flink.table.data.binary.BinaryStringData 
result$50;
                            boolean isNull$51;
                            long result$52;
                            boolean isNull$53;
                            long result$54;
                            boolean isNull$55;
                            boolean result$56;
                            isNull$43 = 
(org.apache.flink.table.data.TimestampData) field$41 == null;
                            result$43 = null;
                            if (!isNull$43) {
                              result$43 = 
(org.apache.flink.table.data.TimestampData) field$41;
                            }
                            isNull$48 = 
(org.apache.flink.table.data.TimestampData) field$42 == null;
                            result$48 = null;
                            if (!isNull$48) {
                              result$48 = 
(org.apache.flink.table.data.TimestampData) field$42;
                            }
                            
                            
                            
                            
                            
                            isNull$44 = isNull$43;
                            result$45 = 
org.apache.flink.table.data.binary.BinaryStringData.EMPTY_UTF8;
                            
                            if (!isNull$44) {
                              
                              result$45 = 
org.apache.flink.table.data.binary.BinaryStringData.fromString(org.apache.flink.table.runtime.functions.SqlDateTimeUtils.timestampToString(result$43,
 6));
                              isNull$44 = (result$45 == null);
                            }
                            
                            isNull$46 = isNull$44;
                            result$47 = -1L;
                            
                            if (!isNull$46) {
                              
                              result$47 = 
                            
org.apache.flink.table.runtime.functions.SqlDateTimeUtils.unixTimestamp(result$45.toString(),
 timeZone)
                                       ;
                              
                            }
                            
                            
                            
                            
                            isNull$49 = isNull$48;
                            result$50 = 
org.apache.flink.table.data.binary.BinaryStringData.EMPTY_UTF8;
                            
                            if (!isNull$49) {
                              
                              result$50 = 
org.apache.flink.table.data.binary.BinaryStringData.fromString(org.apache.flink.table.runtime.functions.SqlDateTimeUtils.timestampToString(result$48,
 6));
                              isNull$49 = (result$50 == null);
                            }
                            
                            isNull$51 = isNull$49;
                            result$52 = -1L;
                            
                            if (!isNull$51) {
                              
                              result$52 = 
                            
org.apache.flink.table.runtime.functions.SqlDateTimeUtils.unixTimestamp(result$50.toString(),
 timeZone)
                                       ;
                              
                            }
                            
                            isNull$53 = isNull$46 || isNull$51;
                            result$54 = -1L;
                            
                            if (!isNull$53) {
                              
                              result$54 = (long) (result$47 - result$52);
                              
                            }
                            
                            
                            isNull$55 = isNull$53 || false;
                            result$56 = false;
                            
                            if (!isNull$55) {
                              
                              result$56 = result$54 >= ((int) -600000);
                              
                            }
                            
                            if (result$56) {
                              return 1;
                            } else {
                              return -1;
                            }
                          
                  }
               
        }
      }
       {code}

  was:
There is a bug in batch over window operator codegen, the comparator codegen 
logic is wrong that member code incorrectly used in method body. This causes 
over window code compile failed when operator initialization.

 

The incorrect generated code for RangeBoundComparator is below, and the wrong 
place is *private static final java.util.TimeZone timeZone = 
java.util.TimeZone.getTimeZone("Asia/Shanghai");*

which is a member statments that should be placed in class body, not method 
body.
{code:java}
      public class RangeBoundComparator$40 implements 
org.apache.flink.table.runtime.generated.RecordComparator {

        private final Object[] references;
        

        public RangeBoundComparator$40(Object[] references) {
          this.references = references;
          
          
        }

        @Override
        public int compare(org.apache.flink.table.data.RowData in1, 
org.apache.flink.table.data.RowData in2) {
          
                  org.apache.flink.table.data.TimestampData field$41;
                  boolean isNull$41;
                  org.apache.flink.table.data.TimestampData field$42;
                  boolean isNull$42;
                  isNull$41 = in1.isNullAt(0);
                  field$41 = null;
                  if (!isNull$41) {
                    field$41 = in1.getTimestamp(0, 3);
                  }
                  isNull$42 = in2.isNullAt(0);
                  field$42 = null;
                  if (!isNull$42) {
                    field$42 = in2.getTimestamp(0, 3);
                  }
                  if (isNull$41 && isNull$42) {
                     return 1;
                  } else if (isNull$41 || isNull$42) {
                     return -1;
                  } else {
                     
                            private static final java.util.TimeZone timeZone =
                                             
java.util.TimeZone.getTimeZone("Asia/Shanghai");
                            org.apache.flink.table.data.TimestampData result$43;
                            boolean isNull$43;
                            boolean isNull$44;
                            org.apache.flink.table.data.binary.BinaryStringData 
result$45;
                            boolean isNull$46;
                            long result$47;
                            org.apache.flink.table.data.TimestampData result$48;
                            boolean isNull$48;
                            boolean isNull$49;
                            org.apache.flink.table.data.binary.BinaryStringData 
result$50;
                            boolean isNull$51;
                            long result$52;
                            boolean isNull$53;
                            long result$54;
                            boolean isNull$55;
                            boolean result$56;
                            isNull$43 = 
(org.apache.flink.table.data.TimestampData) field$41 == null;
                            result$43 = null;
                            if (!isNull$43) {
                              result$43 = 
(org.apache.flink.table.data.TimestampData) field$41;
                            }
                            isNull$48 = 
(org.apache.flink.table.data.TimestampData) field$42 == null;
                            result$48 = null;
                            if (!isNull$48) {
                              result$48 = 
(org.apache.flink.table.data.TimestampData) field$42;
                            }
                            
                            
                            
                            
                            
                            isNull$44 = isNull$43;
                            result$45 = 
org.apache.flink.table.data.binary.BinaryStringData.EMPTY_UTF8;
                            
                            if (!isNull$44) {
                              
                              result$45 = 
org.apache.flink.table.data.binary.BinaryStringData.fromString(org.apache.flink.table.runtime.functions.SqlDateTimeUtils.timestampToString(result$43,
 6));
                              isNull$44 = (result$45 == null);
                            }
                            
                            isNull$46 = isNull$44;
                            result$47 = -1L;
                            
                            if (!isNull$46) {
                              
                              result$47 = 
                            
org.apache.flink.table.runtime.functions.SqlDateTimeUtils.unixTimestamp(result$45.toString(),
 timeZone)
                                       ;
                              
                            }
                            
                            
                            
                            
                            isNull$49 = isNull$48;
                            result$50 = 
org.apache.flink.table.data.binary.BinaryStringData.EMPTY_UTF8;
                            
                            if (!isNull$49) {
                              
                              result$50 = 
org.apache.flink.table.data.binary.BinaryStringData.fromString(org.apache.flink.table.runtime.functions.SqlDateTimeUtils.timestampToString(result$48,
 6));
                              isNull$49 = (result$50 == null);
                            }
                            
                            isNull$51 = isNull$49;
                            result$52 = -1L;
                            
                            if (!isNull$51) {
                              
                              result$52 = 
                            
org.apache.flink.table.runtime.functions.SqlDateTimeUtils.unixTimestamp(result$50.toString(),
 timeZone)
                                       ;
                              
                            }
                            
                            isNull$53 = isNull$46 || isNull$51;
                            result$54 = -1L;
                            
                            if (!isNull$53) {
                              
                              result$54 = (long) (result$47 - result$52);
                              
                            }
                            
                            
                            isNull$55 = isNull$53 || false;
                            result$56 = false;
                            
                            if (!isNull$55) {
                              
                              result$56 = result$54 >= ((int) -600000);
                              
                            }
                            
                            if (result$56) {
                              return 1;
                            } else {
                              return -1;
                            }
                          
                  }
               
        }
      }
       {code}


> Over window codegen error when member code incorrectly used in method body
> --------------------------------------------------------------------------
>
>                 Key: FLINK-30499
>                 URL: https://issues.apache.org/jira/browse/FLINK-30499
>             Project: Flink
>          Issue Type: Bug
>          Components: Table SQL / Planner
>    Affects Versions: 1.16.0
>            Reporter: Runkang He
>            Priority: Blocker
>
> There is a bug in batch over window operator codegen, the comparator codegen 
> logic is wrong that member code incorrectly used in method body. This causes 
> over window code compile failed when operator initialization.
> The incorrect generated code for RangeBoundComparator is below, and the wrong 
> place is *private static final java.util.TimeZone timeZone = 
> java.util.TimeZone.getTimeZone("Asia/Shanghai");*
> which is a member statments that should be placed in class body, not method 
> body.
> {code:java}
>       public class RangeBoundComparator$40 implements 
> org.apache.flink.table.runtime.generated.RecordComparator {
>         private final Object[] references;
>         
>         public RangeBoundComparator$40(Object[] references) {
>           this.references = references;
>           
>           
>         }
>         @Override
>         public int compare(org.apache.flink.table.data.RowData in1, 
> org.apache.flink.table.data.RowData in2) {
>           
>                   org.apache.flink.table.data.TimestampData field$41;
>                   boolean isNull$41;
>                   org.apache.flink.table.data.TimestampData field$42;
>                   boolean isNull$42;
>                   isNull$41 = in1.isNullAt(0);
>                   field$41 = null;
>                   if (!isNull$41) {
>                     field$41 = in1.getTimestamp(0, 3);
>                   }
>                   isNull$42 = in2.isNullAt(0);
>                   field$42 = null;
>                   if (!isNull$42) {
>                     field$42 = in2.getTimestamp(0, 3);
>                   }
>                   if (isNull$41 && isNull$42) {
>                      return 1;
>                   } else if (isNull$41 || isNull$42) {
>                      return -1;
>                   } else {
>                      
>                             private static final java.util.TimeZone timeZone =
>                                              
> java.util.TimeZone.getTimeZone("Asia/Shanghai");
>                             org.apache.flink.table.data.TimestampData 
> result$43;
>                             boolean isNull$43;
>                             boolean isNull$44;
>                             
> org.apache.flink.table.data.binary.BinaryStringData result$45;
>                             boolean isNull$46;
>                             long result$47;
>                             org.apache.flink.table.data.TimestampData 
> result$48;
>                             boolean isNull$48;
>                             boolean isNull$49;
>                             
> org.apache.flink.table.data.binary.BinaryStringData result$50;
>                             boolean isNull$51;
>                             long result$52;
>                             boolean isNull$53;
>                             long result$54;
>                             boolean isNull$55;
>                             boolean result$56;
>                             isNull$43 = 
> (org.apache.flink.table.data.TimestampData) field$41 == null;
>                             result$43 = null;
>                             if (!isNull$43) {
>                               result$43 = 
> (org.apache.flink.table.data.TimestampData) field$41;
>                             }
>                             isNull$48 = 
> (org.apache.flink.table.data.TimestampData) field$42 == null;
>                             result$48 = null;
>                             if (!isNull$48) {
>                               result$48 = 
> (org.apache.flink.table.data.TimestampData) field$42;
>                             }
>                             
>                             
>                             
>                             
>                             
>                             isNull$44 = isNull$43;
>                             result$45 = 
> org.apache.flink.table.data.binary.BinaryStringData.EMPTY_UTF8;
>                             
>                             if (!isNull$44) {
>                               
>                               result$45 = 
> org.apache.flink.table.data.binary.BinaryStringData.fromString(org.apache.flink.table.runtime.functions.SqlDateTimeUtils.timestampToString(result$43,
>  6));
>                               isNull$44 = (result$45 == null);
>                             }
>                             
>                             isNull$46 = isNull$44;
>                             result$47 = -1L;
>                             
>                             if (!isNull$46) {
>                               
>                               result$47 = 
>                             
> org.apache.flink.table.runtime.functions.SqlDateTimeUtils.unixTimestamp(result$45.toString(),
>  timeZone)
>                                        ;
>                               
>                             }
>                             
>                             
>                             
>                             
>                             isNull$49 = isNull$48;
>                             result$50 = 
> org.apache.flink.table.data.binary.BinaryStringData.EMPTY_UTF8;
>                             
>                             if (!isNull$49) {
>                               
>                               result$50 = 
> org.apache.flink.table.data.binary.BinaryStringData.fromString(org.apache.flink.table.runtime.functions.SqlDateTimeUtils.timestampToString(result$48,
>  6));
>                               isNull$49 = (result$50 == null);
>                             }
>                             
>                             isNull$51 = isNull$49;
>                             result$52 = -1L;
>                             
>                             if (!isNull$51) {
>                               
>                               result$52 = 
>                             
> org.apache.flink.table.runtime.functions.SqlDateTimeUtils.unixTimestamp(result$50.toString(),
>  timeZone)
>                                        ;
>                               
>                             }
>                             
>                             isNull$53 = isNull$46 || isNull$51;
>                             result$54 = -1L;
>                             
>                             if (!isNull$53) {
>                               
>                               result$54 = (long) (result$47 - result$52);
>                               
>                             }
>                             
>                             
>                             isNull$55 = isNull$53 || false;
>                             result$56 = false;
>                             
>                             if (!isNull$55) {
>                               
>                               result$56 = result$54 >= ((int) -600000);
>                               
>                             }
>                             
>                             if (result$56) {
>                               return 1;
>                             } else {
>                               return -1;
>                             }
>                           
>                   }
>                
>         }
>       }
>        {code}



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

Reply via email to