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

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

Satyam Shekhar created FLINK-19488:
--------------------------------------

             Summary: 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


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)