You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "Satyam Shekhar (Jira)" <ji...@apache.org> on 2020/10/01 17:31:00 UTC

[jira] [Updated] (FLINK-19488) Failed compilation of generated class

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

Satyam Shekhar updated FLINK-19488:
-----------------------------------
    Attachment: code.java

> Failed compilation of generated class
> -------------------------------------
>
>                 Key: FLINK-19488
>                 URL: https://issues.apache.org/jira/browse/FLINK-19488
>             Project: Flink
>          Issue Type: Bug
>          Components: Table SQL / Runtime
>    Affects Versions: 1.11.1
>         Environment: Flink version: 1.11.1
> The Environment has the following configuration parameters -
> {{env.setParallelism(Integer.getInteger("flinkParallelism", 2));}}
> {{env.getConfig().enableObjectReuse();}}
> {{var settings = EnvironmentSettings.newInstance()}}
> {{  .useBlinkPlanner()}}
> {{  .inBatchMode()}}
> {{  .build();}}
>            Reporter: Satyam Shekhar
>            Priority: Major
>         Attachments: code.java
>
>
> I have a table T0 with the following schema -
> {{root}}
> {{ |-- C0: BIGINT}}
> {{ |-- C1: STRING}}
> {{ |-- blaze_itime: TIMESTAMP(3)}}
>  
> The following SQL query fails for the above table - 
> {{SELECT A.C0 AS output, A.C1 AS dim0 FROM T0 AS A WHERE (A.blaze_itime BETWEEN NOW() - INTERVAL '10' MINUTE AND NOW());}}
>  
> {{The generated code for the above query tries to assign a long value to timestamp type and fails to compile with the following exception -}}
> {{java.lang.RuntimeException: Could not instantiate generated class 'BatchCalc$14'java.lang.RuntimeException: Could not instantiate generated class 'BatchCalc$14' at org.apache.flink.table.runtime.generated.GeneratedClass.newInstance(GeneratedClass.java:67) at org.apache.flink.table.runtime.operators.CodeGenOperatorFactory.createStreamOperator(CodeGenOperatorFactory.java:40) at org.apache.flink.streaming.api.operators.StreamOperatorFactoryUtil.createOperator(StreamOperatorFactoryUtil.java:70) at org.apache.flink.streaming.runtime.tasks.OperatorChain.createChainedOperator(OperatorChain.java:470) at }}
> {{...}}
> {{Caused by: org.codehaus.commons.compiler.CompileException: Line 55, Column 21: Assignment conversion not possible from type "long" to type "org.apache.flink.table.data.TimestampData"Caused by: org.codehaus.commons.compiler.CompileException: Line 55, Column 21: Assignment conversion not possible from type "long" to type "org.apache.flink.table.data.TimestampData" at org.codehaus.janino.UnitCompiler.compileError(UnitCompiler.java:12124) at org.codehaus.janino.UnitCompiler.assignmentConversion(UnitCompiler.java:10975) at org.codehaus.janino.UnitCompiler.compile2(UnitCompiler.java:3788) at org.codehaus.janino.UnitCompiler.access$6100(UnitCompiler.java:215)}}
>  
> {{Here is the generated code -}}
> /* 1 */
> /* 2 */ public class BatchCalc$14 extends org.apache.flink.table.runtime.operators.TableStreamOperator
> /* 3 */ implements org.apache.flink.streaming.api.operators.OneInputStreamOperator {
> /* 4 */
> /* 5 */ private final Object[] references;
> /* 6 */ org.apache.flink.table.data.BoxedWrapperRowData out = new org.apache.flink.table.data.BoxedWrapperRowData(2);
> /* 7 */ private final org.apache.flink.streaming.runtime.streamrecord.StreamRecord outElement = new org.apache.flink.streaming.runtime.streamrecord.StreamRecord(null);
> /* 8 */
> /* 9 */ public BatchCalc$14(
> /* 10 */ Object[] references,
> /* 11 */ org.apache.flink.streaming.runtime.tasks.StreamTask task,
> /* 12 */ org.apache.flink.streaming.api.graph.StreamConfig config,
> /* 13 */ org.apache.flink.streaming.api.operators.Output output,
> /* 14 */ org.apache.flink.streaming.runtime.tasks.ProcessingTimeService processingTimeService) throws Exception {
> /* 15 */ this.references = references;
> /* 16 */ 
> /* 17 */ this.setup(task, config, output);
> /* 18 */ if (this instanceof org.apache.flink.streaming.api.operators.AbstractStreamOperator) {
> /* 19 */ ((org.apache.flink.streaming.api.operators.AbstractStreamOperator) this)
> /* 20 */ .setProcessingTimeService(processingTimeService);
> /* 21 */ }
> /* 22 */ }
> /* 23 */
> /* 24 */ @Override
> /* 25 */ public void open() throws Exception {
> /* 26 */ super.open();
> /* 27 */ 
> /* 28 */ }
> /* 29 */
> /* 30 */ @Override
> /* 31 */ public void processElement(org.apache.flink.streaming.runtime.streamrecord.StreamRecord element) throws Exception {
> /* 32 */ org.apache.flink.table.data.RowData in1 = (org.apache.flink.table.data.RowData) element.getValue();
> /* 33 */ 
> /* 34 */ org.apache.flink.table.data.TimestampData field$2;
> /* 35 */ boolean isNull$2;
> /* 36 */ boolean isNull$3;
> /* 37 */ org.apache.flink.table.data.TimestampData result$4;
> /* 38 */ boolean isNull$5;
> /* 39 */ org.apache.flink.table.data.TimestampData result$6;
> /* 40 */ boolean isNull$7;
> /* 41 */ boolean result$8;
> /* 42 */ boolean isNull$9;
> /* 43 */ org.apache.flink.table.data.TimestampData result$10;
> /* 44 */ boolean isNull$11;
> /* 45 */ boolean result$12;
> /* 46 */ 
> /* 47 */ 
> /* 48 */ isNull$2 = in1.isNullAt(2);
> /* 49 */ field$2 = null;
> /* 50 */ if (!isNull$2) {
> /* 51 */ field$2 = in1.getTimestamp(2, 3);
> /* 52 */ }
> /* 53 */ isNull$3 = false;
> /* 54 */ 
> /* 55 */ result$4 = 
> /* 56 */ org.apache.flink.table.runtime.functions.SqlDateTimeUtils.now()
> /* 57 */ ;
> /* 58 */ 
> /* 59 */ 
> /* 60 */ 
> /* 61 */ isNull$5 = isNull$3 || false;
> /* 62 */ result$6 = null;
> /* 63 */ if (!isNull$5) {
> /* 64 */ 
> /* 65 */ result$6 = org.apache.flink.table.data.TimestampData.fromEpochMillis(result$4.getMillisecond() - ((long) 600000L), result$4.getNanoOfMillisecond());
> /* 66 */ 
> /* 67 */ }
> /* 68 */ 
> /* 69 */ isNull$7 = isNull$2 || isNull$5;
> /* 70 */ result$8 = false;
> /* 71 */ if (!isNull$7) {
> /* 72 */ 
> /* 73 */ result$8 = field$2.compareTo(result$6) >= 0;
> /* 74 */ 
> /* 75 */ }
> /* 76 */ 
> /* 77 */ 
> /* 78 */ boolean result$13 = false;
> /* 79 */ boolean isNull$13 = false;
> /* 80 */ if (!isNull$7 && !result$8) {
> /* 81 */ // left expr is false, skip right expr
> /* 82 */ } else {
> /* 83 */ 
> /* 84 */ 
> /* 85 */ 
> /* 86 */ 
> /* 87 */ isNull$9 = false;
> /* 88 */ 
> /* 89 */ result$10 = 
> /* 90 */ org.apache.flink.table.runtime.functions.SqlDateTimeUtils.now()
> /* 91 */ ;
> /* 92 */ 
> /* 93 */ 
> /* 94 */ isNull$11 = isNull$2 || isNull$9;
> /* 95 */ result$12 = false;
> /* 96 */ if (!isNull$11) {
> /* 97 */ 
> /* 98 */ result$12 = field$2.compareTo(result$10) <= 0;
> /* 99 */ 
> /* 100 */ }
> /* 101 */ 
> /* 102 */ 
> /* 103 */ if (!isNull$7 && !isNull$11) {
> /* 104 */ result$13 = result$8 && result$12;
> /* 105 */ isNull$13 = false;
> /* 106 */ }
> /* 107 */ else if (!isNull$7 && result$8 && isNull$11) {
> /* 108 */ result$13 = false;
> /* 109 */ isNull$13 = true;
> /* 110 */ }
> /* 111 */ else if (!isNull$7 && !result$8 && isNull$11) {
> /* 112 */ result$13 = false;
> /* 113 */ isNull$13 = false;
> /* 114 */ }
> /* 115 */ else if (isNull$7 && !isNull$11 && result$12) {
> /* 116 */ result$13 = false;
> /* 117 */ isNull$13 = true;
> /* 118 */ }
> /* 119 */ else if (isNull$7 && !isNull$11 && !result$12) {
> /* 120 */ result$13 = false;
> /* 121 */ isNull$13 = false;
> /* 122 */ }
> /* 123 */ else {
> /* 124 */ result$13 = false;
> /* 125 */ isNull$13 = true;
> /* 126 */ }
> /* 127 */ }
> /* 128 */ if (result$13) {
> /* 129 */ 
> /* 130 */ 
> /* 131 */ 
> /* 132 */ 
> /* 133 */ 
> /* 134 */ 
> /* 135 */ 
> /* 136 */ if (in1.isNullAt(0)) {
> /* 137 */ out.setNullAt(0);
> /* 138 */ } else {
> /* 139 */ out.setLong(0, in1.getLong(0));
> /* 140 */ }
> /* 141 */ 
> /* 142 */ 
> /* 143 */ 
> /* 144 */ if (in1.isNullAt(1)) {
> /* 145 */ out.setNullAt(1);
> /* 146 */ } else {
> /* 147 */ out.setNonPrimitiveValue(1, ((org.apache.flink.table.data.binary.BinaryStringData) in1.getString(1)));
> /* 148 */ }
> /* 149 */ 
> /* 150 */ 
> /* 151 */ output.collect(outElement.replace(out));
> /* 152 */ 
> /* 153 */ }
> /* 154 */ 
> /* 155 */ }
> /* 156 */
> /* 157 */ 
> /* 158 */
> /* 159 */ @Override
> /* 160 */ public void close() throws Exception {
> /* 161 */ super.close();
> /* 162 */ 
> /* 163 */ }
> /* 164 */
> /* 165 */ 
> /* 166 */ }
> /* 167 */



--
This message was sent by Atlassian Jira
(v8.3.4#803005)